github.com/parquet-go/parquet-go@v0.20.0/parquet_test.go (about)

     1  package parquet_test
     2  
     3  import (
     4  	"bytes"
     5  	"fmt"
     6  	"io"
     7  	"log"
     8  	"math/rand"
     9  	"os"
    10  	"reflect"
    11  	"strings"
    12  	"testing"
    13  	"time"
    14  
    15  	"github.com/google/uuid"
    16  	"google.golang.org/protobuf/types/known/structpb"
    17  
    18  	"github.com/parquet-go/parquet-go"
    19  	"github.com/parquet-go/parquet-go/deprecated"
    20  	"github.com/parquet-go/parquet-go/internal/quick"
    21  )
    22  
    23  const (
    24  	benchmarkNumRows     = 10_000
    25  	benchmarkRowsPerStep = 1000
    26  )
    27  
    28  func ExampleReadFile() {
    29  	type Row struct {
    30  		ID   int64  `parquet:"id"`
    31  		Name string `parquet:"name,zstd"`
    32  	}
    33  
    34  	ExampleWriteFile()
    35  
    36  	rows, err := parquet.ReadFile[Row]("/tmp/file.parquet")
    37  	if err != nil {
    38  		log.Fatal(err)
    39  	}
    40  
    41  	for _, row := range rows {
    42  		fmt.Printf("%d: %q\n", row.ID, row.Name)
    43  	}
    44  
    45  	// Output:
    46  	// 0: "Bob"
    47  	// 1: "Alice"
    48  	// 2: "Franky"
    49  }
    50  
    51  func ExampleWriteFile() {
    52  	type Row struct {
    53  		ID   int64  `parquet:"id"`
    54  		Name string `parquet:"name,zstd"`
    55  	}
    56  
    57  	if err := parquet.WriteFile("/tmp/file.parquet", []Row{
    58  		{ID: 0, Name: "Bob"},
    59  		{ID: 1, Name: "Alice"},
    60  		{ID: 2, Name: "Franky"},
    61  	}); err != nil {
    62  		log.Fatal(err)
    63  	}
    64  
    65  	// Output:
    66  }
    67  
    68  func ExampleRead_any() {
    69  	type Row struct{ FirstName, LastName string }
    70  
    71  	buf := new(bytes.Buffer)
    72  	err := parquet.Write(buf, []Row{
    73  		{FirstName: "Luke", LastName: "Skywalker"},
    74  		{FirstName: "Han", LastName: "Solo"},
    75  		{FirstName: "R2", LastName: "D2"},
    76  	})
    77  	if err != nil {
    78  		log.Fatal(err)
    79  	}
    80  
    81  	file := bytes.NewReader(buf.Bytes())
    82  
    83  	rows, err := parquet.Read[any](file, file.Size())
    84  	if err != nil {
    85  		log.Fatal(err)
    86  	}
    87  
    88  	for _, row := range rows {
    89  		fmt.Printf("%q\n", row)
    90  	}
    91  
    92  	// Output:
    93  	// map["FirstName":"Luke" "LastName":"Skywalker"]
    94  	// map["FirstName":"Han" "LastName":"Solo"]
    95  	// map["FirstName":"R2" "LastName":"D2"]
    96  }
    97  
    98  func ExampleWrite_any() {
    99  	schema := parquet.SchemaOf(struct {
   100  		FirstName string
   101  		LastName  string
   102  	}{})
   103  
   104  	buf := new(bytes.Buffer)
   105  	err := parquet.Write[any](
   106  		buf,
   107  		[]any{
   108  			map[string]string{"FirstName": "Luke", "LastName": "Skywalker"},
   109  			map[string]string{"FirstName": "Han", "LastName": "Solo"},
   110  			map[string]string{"FirstName": "R2", "LastName": "D2"},
   111  		},
   112  		schema,
   113  	)
   114  	if err != nil {
   115  		log.Fatal(err)
   116  	}
   117  
   118  	file := bytes.NewReader(buf.Bytes())
   119  
   120  	rows, err := parquet.Read[any](file, file.Size())
   121  	if err != nil {
   122  		log.Fatal(err)
   123  	}
   124  
   125  	for _, row := range rows {
   126  		fmt.Printf("%q\n", row)
   127  	}
   128  
   129  	// Output:
   130  	// map["FirstName":"Luke" "LastName":"Skywalker"]
   131  	// map["FirstName":"Han" "LastName":"Solo"]
   132  	// map["FirstName":"R2" "LastName":"D2"]
   133  }
   134  
   135  func ExampleSearch() {
   136  	type Row struct{ FirstName, LastName string }
   137  
   138  	buf := new(bytes.Buffer)
   139  	// The column being searched should be sorted to avoid a full scan of the
   140  	// column. See the section of the readme on sorting for how to sort on
   141  	// insertion into the parquet file using parquet.SortingColumns
   142  	rows := []Row{
   143  		{FirstName: "C", LastName: "3PO"},
   144  		{FirstName: "Han", LastName: "Solo"},
   145  		{FirstName: "Leia", LastName: "Organa"},
   146  		{FirstName: "Luke", LastName: "Skywalker"},
   147  		{FirstName: "R2", LastName: "D2"},
   148  	}
   149  	// The tiny page buffer size ensures we get multiple pages out of the example above.
   150  	w := parquet.NewGenericWriter[Row](buf, parquet.PageBufferSize(12), parquet.WriteBufferSize(0))
   151  	// Need to write 1 row at a time here as writing many at once disregards PageBufferSize option.
   152  	for _, row := range rows {
   153  		_, err := w.Write([]Row{row})
   154  		if err != nil {
   155  			log.Fatal(err)
   156  		}
   157  	}
   158  	err := w.Close()
   159  	if err != nil {
   160  		log.Fatal(err)
   161  	}
   162  
   163  	reader := bytes.NewReader(buf.Bytes())
   164  	file, err := parquet.OpenFile(reader, reader.Size())
   165  	if err != nil {
   166  		log.Fatal(err)
   167  	}
   168  
   169  	// Search is scoped to a single RowGroup/ColumnChunk
   170  	rowGroup := file.RowGroups()[0]
   171  	firstNameColChunk := rowGroup.ColumnChunks()[0]
   172  
   173  	columnIndex, err := firstNameColChunk.ColumnIndex()
   174  	if err != nil {
   175  		log.Fatal(err)
   176  	}
   177  	found := parquet.Search(columnIndex, parquet.ValueOf("Luke"), parquet.ByteArrayType)
   178  	offsetIndex, _ := firstNameColChunk.OffsetIndex()
   179  	fmt.Printf("numPages: %d\n", offsetIndex.NumPages())
   180  	fmt.Printf("result found in page: %d\n", found)
   181  	if found < offsetIndex.NumPages() {
   182  		r := parquet.NewGenericReader[Row](file)
   183  		defer r.Close()
   184  		// Seek to the first row in the page the result was found
   185  		r.SeekToRow(offsetIndex.FirstRowIndex(found))
   186  		result := make([]Row, 2)
   187  		_, _ = r.Read(result)
   188  		// Leia is in index 0 for the page.
   189  		for _, row := range result {
   190  			if row.FirstName == "Luke" {
   191  				fmt.Printf("%q\n", row)
   192  			}
   193  		}
   194  	}
   195  
   196  	// Output:
   197  	// numPages: 3
   198  	// result found in page: 1
   199  	// {"Luke" "Skywalker"}
   200  }
   201  
   202  func TestIssue360(t *testing.T) {
   203  	type TestType struct {
   204  		Key []int
   205  	}
   206  
   207  	schema := parquet.SchemaOf(TestType{})
   208  	buffer := parquet.NewGenericBuffer[any](schema)
   209  
   210  	data := make([]any, 1)
   211  	data[0] = TestType{Key: []int{1}}
   212  	_, err := buffer.Write(data)
   213  	if err != nil {
   214  		fmt.Println("Exiting with error: ", err)
   215  		return
   216  	}
   217  
   218  	var out bytes.Buffer
   219  	writer := parquet.NewGenericWriter[any](&out, schema)
   220  
   221  	_, err = parquet.CopyRows(writer, buffer.Rows())
   222  	if err != nil {
   223  		fmt.Println("Exiting with error: ", err)
   224  		return
   225  	}
   226  	writer.Close()
   227  
   228  	br := bytes.NewReader(out.Bytes())
   229  	rows, _ := parquet.Read[any](br, br.Size())
   230  
   231  	expect := []any{
   232  		map[string]any{
   233  			"Key": []any{
   234  				int64(1),
   235  			},
   236  		},
   237  	}
   238  
   239  	assertRowsEqual(t, expect, rows)
   240  }
   241  
   242  func TestIssue362ParquetReadFromGenericReaders(t *testing.T) {
   243  	path := "testdata/dms_test_table_LOAD00000001.parquet"
   244  	fp, err := os.Open(path)
   245  	if err != nil {
   246  		t.Fatal(err)
   247  	}
   248  	defer fp.Close()
   249  
   250  	r1 := parquet.NewGenericReader[any](fp)
   251  	rows1 := make([]any, r1.NumRows())
   252  	_, err = r1.Read(rows1)
   253  	if err != nil && err != io.EOF {
   254  		t.Fatal(err)
   255  	}
   256  
   257  	r2 := parquet.NewGenericReader[any](fp)
   258  	rows2 := make([]any, r2.NumRows())
   259  	_, err = r2.Read(rows2)
   260  	if err != nil && err != io.EOF {
   261  		t.Fatal(err)
   262  	}
   263  }
   264  
   265  func TestIssue362ParquetReadFile(t *testing.T) {
   266  	rows1, err := parquet.ReadFile[any]("testdata/dms_test_table_LOAD00000001.parquet")
   267  	if err != nil {
   268  		t.Fatal(err)
   269  	}
   270  
   271  	rows2, err := parquet.ReadFile[any]("testdata/dms_test_table_LOAD00000001.parquet")
   272  	if err != nil {
   273  		t.Fatal(err)
   274  	}
   275  
   276  	assertRowsEqual(t, rows1, rows2)
   277  }
   278  
   279  func TestIssue368(t *testing.T) {
   280  	f, err := os.Open("testdata/issue368.parquet")
   281  	if err != nil {
   282  		t.Fatal(err)
   283  	}
   284  	defer f.Close()
   285  
   286  	info, err := f.Stat()
   287  	if err != nil {
   288  		t.Fatal(err)
   289  	}
   290  
   291  	pf, err := parquet.OpenFile(f, info.Size())
   292  	if err != nil {
   293  		t.Fatal(err)
   294  	}
   295  
   296  	reader := parquet.NewGenericReader[any](pf)
   297  	defer reader.Close()
   298  
   299  	trs := make([]any, 1)
   300  	for {
   301  		_, err := reader.Read(trs)
   302  		if err != nil {
   303  			break
   304  		}
   305  	}
   306  }
   307  
   308  func TestIssue377(t *testing.T) {
   309  	type People struct {
   310  		Name string
   311  		Age  int
   312  	}
   313  
   314  	type Nested struct {
   315  		P  []People
   316  		F  string
   317  		GF string
   318  	}
   319  	row1 := Nested{P: []People{
   320  		{
   321  			Name: "Bob",
   322  			Age:  10,
   323  		}}}
   324  	ods := []Nested{
   325  		row1,
   326  	}
   327  	buf := new(bytes.Buffer)
   328  	w := parquet.NewGenericWriter[Nested](buf)
   329  	_, err := w.Write(ods)
   330  	if err != nil {
   331  		t.Fatal("write error: ", err)
   332  	}
   333  	w.Close()
   334  
   335  	file := bytes.NewReader(buf.Bytes())
   336  	rows, err := parquet.Read[Nested](file, file.Size())
   337  	if err != nil {
   338  		t.Fatal("read error: ", err)
   339  	}
   340  
   341  	assertRowsEqual(t, rows, ods)
   342  }
   343  
   344  func TestIssue423(t *testing.T) {
   345  	type Inner struct {
   346  		Value string `parquet:","`
   347  	}
   348  	type Outer struct {
   349  		Label string  `parquet:","`
   350  		Inner Inner   `parquet:",json"`
   351  		Slice []Inner `parquet:",json"`
   352  		// This is the only tricky situation. Because we're delegating to json Marshaler/Unmarshaler
   353  		// We use the json tags for optionality.
   354  		Ptr *Inner `json:",omitempty" parquet:",json"`
   355  
   356  		// This tests BC behavior that slices of bytes and json strings still get written/read in a BC way.
   357  		String        string                     `parquet:",json"`
   358  		Bytes         []byte                     `parquet:",json"`
   359  		MapOfStructPb map[string]*structpb.Value `parquet:",json"`
   360  		StructPB      *structpb.Value            `parquet:",json"`
   361  	}
   362  
   363  	writeRows := []Outer{
   364  		{
   365  			Label: "welp",
   366  			Inner: Inner{
   367  				Value: "this is a string",
   368  			},
   369  			Slice: []Inner{
   370  				{
   371  					Value: "in a slice",
   372  				},
   373  			},
   374  			Ptr:    nil,
   375  			String: `{"hello":"world"}`,
   376  			Bytes:  []byte(`{"goodbye":"world"}`),
   377  			MapOfStructPb: map[string]*structpb.Value{
   378  				"answer": structpb.NewNumberValue(42.00),
   379  			},
   380  			StructPB: structpb.NewBoolValue(true),
   381  		},
   382  		{
   383  			Label: "foxes",
   384  			Inner: Inner{
   385  				Value: "the quick brown fox jumped over the yellow lazy dog.",
   386  			},
   387  			Slice: []Inner{
   388  				{
   389  					Value: "in a slice",
   390  				},
   391  			},
   392  			Ptr: &Inner{
   393  				Value: "not nil",
   394  			},
   395  			String: `{"hello":"world"}`,
   396  			Bytes:  []byte(`{"goodbye":"world"}`),
   397  			MapOfStructPb: map[string]*structpb.Value{
   398  				"doubleAnswer": structpb.NewNumberValue(84.00),
   399  			},
   400  			StructPB: structpb.NewBoolValue(false),
   401  		},
   402  	}
   403  
   404  	schema := parquet.SchemaOf(new(Outer))
   405  	fmt.Println(schema.String())
   406  	buf := new(bytes.Buffer)
   407  	w := parquet.NewGenericWriter[Outer](buf, schema)
   408  	_, err := w.Write(writeRows)
   409  	if err != nil {
   410  		t.Fatal("write error: ", err)
   411  	}
   412  	w.Close()
   413  
   414  	file := bytes.NewReader(buf.Bytes())
   415  	readRows, err := parquet.Read[Outer](file, file.Size())
   416  	if err != nil {
   417  		t.Fatal("read error: ", err)
   418  	}
   419  
   420  	assertRowsEqual(t, writeRows, readRows)
   421  }
   422  
   423  func TestReadFileGenericMultipleRowGroupsMultiplePages(t *testing.T) {
   424  	type MyRow struct {
   425  		ID    [16]byte `parquet:"id,delta,uuid"`
   426  		File  string   `parquet:"file,dict,zstd"`
   427  		Index int64    `parquet:"index,delta,zstd"`
   428  	}
   429  
   430  	numRows := 20_000
   431  	maxPageBytes := 5000
   432  
   433  	tmp, err := os.CreateTemp("/tmp", "*.parquet")
   434  	if err != nil {
   435  		t.Fatal("os.CreateTemp: ", err)
   436  	}
   437  	path := tmp.Name()
   438  	defer os.Remove(path)
   439  	t.Log("file:", path)
   440  
   441  	// The page buffer size ensures we get multiple pages out of this example.
   442  	w := parquet.NewGenericWriter[MyRow](tmp, parquet.PageBufferSize(maxPageBytes))
   443  	// Need to write 1 row at a time here as writing many at once disregards PageBufferSize option.
   444  	for i := 0; i < numRows; i++ {
   445  		row := MyRow{
   446  			ID:    [16]byte{15: byte(i)},
   447  			File:  "hi" + fmt.Sprint(i),
   448  			Index: int64(i),
   449  		}
   450  		_, err := w.Write([]MyRow{row})
   451  		if err != nil {
   452  			t.Fatal("w.Write: ", err)
   453  		}
   454  		// Flush writes rows as row group. 4 total (20k/5k) in this file.
   455  		if (i+1)%maxPageBytes == 0 {
   456  			err = w.Flush()
   457  			if err != nil {
   458  				t.Fatal("w.Flush: ", err)
   459  			}
   460  		}
   461  	}
   462  	err = w.Close()
   463  	if err != nil {
   464  		t.Fatal("w.Close: ", err)
   465  	}
   466  	err = tmp.Close()
   467  	if err != nil {
   468  		t.Fatal("tmp.Close: ", err)
   469  	}
   470  
   471  	rows, err := parquet.ReadFile[MyRow](path)
   472  	if err != nil {
   473  		t.Fatal("parquet.ReadFile: ", err)
   474  	}
   475  
   476  	if len(rows) != numRows {
   477  		t.Fatalf("not enough values were read: want=%d got=%d", len(rows), numRows)
   478  	}
   479  	for i, row := range rows {
   480  		id := [16]byte{15: byte(i)}
   481  		file := "hi" + fmt.Sprint(i)
   482  		index := int64(i)
   483  
   484  		if row.ID != id || row.File != file || row.Index != index {
   485  			t.Fatalf("rows mismatch at index: %d got: %+v", i, row)
   486  		}
   487  	}
   488  }
   489  
   490  func assertRowsEqual[T any](t *testing.T, rows1, rows2 []T) {
   491  	if !reflect.DeepEqual(rows1, rows2) {
   492  		t.Error("rows mismatch")
   493  
   494  		t.Log("want:")
   495  		logRows(t, rows1)
   496  
   497  		t.Log("got:")
   498  		logRows(t, rows2)
   499  	}
   500  }
   501  
   502  func logRows[T any](t *testing.T, rows []T) {
   503  	for _, row := range rows {
   504  		t.Logf(". %#v\n", row)
   505  	}
   506  }
   507  
   508  func TestNestedPointer(t *testing.T) {
   509  	type InnerStruct struct {
   510  		InnerField string
   511  	}
   512  
   513  	type SliceElement struct {
   514  		Inner *InnerStruct
   515  	}
   516  
   517  	type Outer struct {
   518  		Slice []*SliceElement
   519  	}
   520  	value := "inner-string"
   521  	in := &Outer{
   522  		Slice: []*SliceElement{
   523  			{
   524  				Inner: &InnerStruct{
   525  					InnerField: value,
   526  				},
   527  			},
   528  		},
   529  	}
   530  
   531  	var f bytes.Buffer
   532  
   533  	pw := parquet.NewGenericWriter[*Outer](&f)
   534  	_, err := pw.Write([]*Outer{in})
   535  	if err != nil {
   536  		t.Fatal(err)
   537  	}
   538  
   539  	err = pw.Close()
   540  	if err != nil {
   541  		t.Fatal(err)
   542  	}
   543  
   544  	pr := parquet.NewGenericReader[*Outer](bytes.NewReader(f.Bytes()))
   545  
   546  	out := make([]*Outer, 1)
   547  	_, err = pr.Read(out)
   548  	if err != nil {
   549  		t.Fatal(err)
   550  	}
   551  	pr.Close()
   552  	if want, got := value, out[0].Slice[0].Inner.InnerField; want != got {
   553  		t.Error("failed to set inner field pointer")
   554  	}
   555  }
   556  
   557  type benchmarkRowType struct {
   558  	ID    [16]byte `parquet:"id,uuid"`
   559  	Value float64  `parquet:"value"`
   560  }
   561  
   562  func (row benchmarkRowType) generate(prng *rand.Rand) benchmarkRowType {
   563  	prng.Read(row.ID[:])
   564  	row.Value = prng.Float64()
   565  	return row
   566  }
   567  
   568  type paddedBooleanColumn struct {
   569  	Value bool
   570  	_     [3]byte
   571  }
   572  
   573  func (row paddedBooleanColumn) generate(prng *rand.Rand) paddedBooleanColumn {
   574  	return paddedBooleanColumn{Value: prng.Int()%2 == 0}
   575  }
   576  
   577  type booleanColumn struct {
   578  	Value bool
   579  }
   580  
   581  func (row booleanColumn) generate(prng *rand.Rand) booleanColumn {
   582  	return booleanColumn{Value: prng.Int()%2 == 0}
   583  }
   584  
   585  type int32Column struct {
   586  	Value int32 `parquet:",delta"`
   587  }
   588  
   589  func (row int32Column) generate(prng *rand.Rand) int32Column {
   590  	return int32Column{Value: prng.Int31n(100)}
   591  }
   592  
   593  type int64Column struct {
   594  	Value int64 `parquet:",delta"`
   595  }
   596  
   597  func (row int64Column) generate(prng *rand.Rand) int64Column {
   598  	return int64Column{Value: prng.Int63n(100)}
   599  }
   600  
   601  type int96Column struct {
   602  	Value deprecated.Int96
   603  }
   604  
   605  func (row int96Column) generate(prng *rand.Rand) int96Column {
   606  	row.Value[0] = prng.Uint32()
   607  	row.Value[1] = prng.Uint32()
   608  	row.Value[2] = prng.Uint32()
   609  	return row
   610  }
   611  
   612  type floatColumn struct {
   613  	Value float32
   614  }
   615  
   616  func (row floatColumn) generate(prng *rand.Rand) floatColumn {
   617  	return floatColumn{Value: prng.Float32()}
   618  }
   619  
   620  type doubleColumn struct {
   621  	Value float64
   622  }
   623  
   624  func (row doubleColumn) generate(prng *rand.Rand) doubleColumn {
   625  	return doubleColumn{Value: prng.Float64()}
   626  }
   627  
   628  type byteArrayColumn struct {
   629  	Value []byte
   630  }
   631  
   632  func (row byteArrayColumn) generate(prng *rand.Rand) byteArrayColumn {
   633  	row.Value = make([]byte, prng.Intn(10))
   634  	prng.Read(row.Value)
   635  	return row
   636  }
   637  
   638  type fixedLenByteArrayColumn struct {
   639  	Value [10]byte
   640  }
   641  
   642  func (row fixedLenByteArrayColumn) generate(prng *rand.Rand) fixedLenByteArrayColumn {
   643  	prng.Read(row.Value[:])
   644  	return row
   645  }
   646  
   647  type stringColumn struct {
   648  	Value string
   649  }
   650  
   651  func (row stringColumn) generate(prng *rand.Rand) stringColumn {
   652  	return stringColumn{Value: generateString(prng, 10)}
   653  }
   654  
   655  type indexedStringColumn struct {
   656  	Value string `parquet:",dict"`
   657  }
   658  
   659  func (row indexedStringColumn) generate(prng *rand.Rand) indexedStringColumn {
   660  	return indexedStringColumn{Value: generateString(prng, 10)}
   661  }
   662  
   663  type uuidColumn struct {
   664  	Value uuid.UUID `parquet:",delta"`
   665  }
   666  
   667  func (row uuidColumn) generate(prng *rand.Rand) uuidColumn {
   668  	prng.Read(row.Value[:])
   669  	return row
   670  }
   671  
   672  type timeColumn struct {
   673  	Value time.Time
   674  }
   675  
   676  func (row timeColumn) generate(prng *rand.Rand) timeColumn {
   677  	t := time.Unix(0, prng.Int63()).UTC()
   678  	return timeColumn{Value: t}
   679  }
   680  
   681  type timeInMillisColumn struct {
   682  	Value time.Time `parquet:",timestamp(millisecond)"`
   683  }
   684  
   685  func (row timeInMillisColumn) generate(prng *rand.Rand) timeInMillisColumn {
   686  	t := time.Unix(0, prng.Int63()).UTC()
   687  	return timeInMillisColumn{Value: t}
   688  }
   689  
   690  type decimalColumn struct {
   691  	Value int64 `parquet:",decimal(0:3)"`
   692  }
   693  
   694  func (row decimalColumn) generate(prng *rand.Rand) decimalColumn {
   695  	return decimalColumn{Value: prng.Int63()}
   696  }
   697  
   698  type mapColumn struct {
   699  	Value map[utf8string]int
   700  }
   701  
   702  func (row mapColumn) generate(prng *rand.Rand) mapColumn {
   703  	n := prng.Intn(10)
   704  	row.Value = make(map[utf8string]int, n)
   705  	for i := 0; i < n; i++ {
   706  		row.Value[utf8string(generateString(prng, 8))] = prng.Intn(100)
   707  	}
   708  	return row
   709  }
   710  
   711  type addressBook struct {
   712  	Owner             utf8string   `parquet:",plain"`
   713  	OwnerPhoneNumbers []utf8string `parquet:",plain"`
   714  	Contacts          []contact
   715  }
   716  
   717  type contact struct {
   718  	Name        utf8string `parquet:",plain"`
   719  	PhoneNumber utf8string `parquet:",plain"`
   720  }
   721  
   722  func (row contact) generate(prng *rand.Rand) contact {
   723  	return contact{
   724  		Name:        utf8string(generateString(prng, 16)),
   725  		PhoneNumber: utf8string(generateString(prng, 10)),
   726  	}
   727  }
   728  
   729  type optionalInt32Column struct {
   730  	Value int32 `parquet:",optional"`
   731  }
   732  
   733  func (row optionalInt32Column) generate(prng *rand.Rand) optionalInt32Column {
   734  	return optionalInt32Column{Value: prng.Int31n(100)}
   735  }
   736  
   737  type repeatedInt32Column struct {
   738  	Values []int32
   739  }
   740  
   741  func (row repeatedInt32Column) generate(prng *rand.Rand) repeatedInt32Column {
   742  	row.Values = make([]int32, prng.Intn(10))
   743  	for i := range row.Values {
   744  		row.Values[i] = prng.Int31n(10)
   745  	}
   746  	return row
   747  }
   748  
   749  type listColumn2 struct {
   750  	Value utf8string `parquet:",optional"`
   751  }
   752  
   753  type listColumn1 struct {
   754  	List2 []listColumn2 `parquet:",list"`
   755  }
   756  
   757  type listColumn0 struct {
   758  	List1 []listColumn1 `parquet:",list"`
   759  }
   760  
   761  type nestedListColumn1 struct {
   762  	Level3 []utf8string `parquet:"level3"`
   763  }
   764  
   765  type nestedListColumn struct {
   766  	Level1 []nestedListColumn1 `parquet:"level1"`
   767  	Level2 []utf8string        `parquet:"level2"`
   768  }
   769  
   770  type utf8string string
   771  
   772  func (utf8string) Generate(rand *rand.Rand, size int) reflect.Value {
   773  	const characters = "abcdefghijklmnopqrstuvwxyz1234567890"
   774  	const maxSize = 10
   775  	if size > maxSize {
   776  		size = maxSize
   777  	}
   778  	n := rand.Intn(size)
   779  	b := make([]byte, n)
   780  	for i := range b {
   781  		b[i] = characters[rand.Intn(len(characters))]
   782  	}
   783  	return reflect.ValueOf(utf8string(b))
   784  }
   785  
   786  type Contact struct {
   787  	Name        string `parquet:"name"`
   788  	PhoneNumber string `parquet:"phoneNumber,optional,zstd"`
   789  }
   790  
   791  type AddressBook struct {
   792  	Owner             string    `parquet:"owner,zstd"`
   793  	OwnerPhoneNumbers []string  `parquet:"ownerPhoneNumbers,gzip"`
   794  	Contacts          []Contact `parquet:"contacts"`
   795  }
   796  
   797  func forEachLeafColumn(col *parquet.Column, do func(*parquet.Column) error) error {
   798  	children := col.Columns()
   799  
   800  	if len(children) == 0 {
   801  		return do(col)
   802  	}
   803  
   804  	for _, child := range children {
   805  		if err := forEachLeafColumn(child, do); err != nil {
   806  			return err
   807  		}
   808  	}
   809  
   810  	return nil
   811  }
   812  
   813  func forEachPage(pages parquet.PageReader, do func(parquet.Page) error) error {
   814  	doAndReleasePage := func(page parquet.Page) error {
   815  		defer parquet.Release(page)
   816  		return do(page)
   817  	}
   818  
   819  	for {
   820  		p, err := pages.ReadPage()
   821  		if err != nil {
   822  			if err == io.EOF {
   823  				err = nil
   824  			}
   825  			return err
   826  		}
   827  		if err := doAndReleasePage(p); err != nil {
   828  			return err
   829  		}
   830  	}
   831  }
   832  
   833  func forEachValue(values parquet.ValueReader, do func(parquet.Value) error) error {
   834  	buffer := [3]parquet.Value{}
   835  	for {
   836  		n, err := values.ReadValues(buffer[:])
   837  		for _, v := range buffer[:n] {
   838  			if err := do(v); err != nil {
   839  				return err
   840  			}
   841  		}
   842  		if err != nil {
   843  			if err == io.EOF {
   844  				err = nil
   845  			}
   846  			return err
   847  		}
   848  	}
   849  }
   850  
   851  func forEachColumnPage(col *parquet.Column, do func(*parquet.Column, parquet.Page) error) error {
   852  	return forEachLeafColumn(col, func(leaf *parquet.Column) error {
   853  		pages := leaf.Pages()
   854  		defer pages.Close()
   855  		return forEachPage(pages, func(page parquet.Page) error { return do(leaf, page) })
   856  	})
   857  }
   858  
   859  func forEachColumnValue(col *parquet.Column, do func(*parquet.Column, parquet.Value) error) error {
   860  	return forEachColumnPage(col, func(leaf *parquet.Column, page parquet.Page) error {
   861  		return forEachValue(page.Values(), func(value parquet.Value) error { return do(leaf, value) })
   862  	})
   863  }
   864  
   865  func forEachColumnChunk(file *parquet.File, do func(*parquet.Column, parquet.ColumnChunk) error) error {
   866  	return forEachLeafColumn(file.Root(), func(leaf *parquet.Column) error {
   867  		for _, rowGroup := range file.RowGroups() {
   868  			if err := do(leaf, rowGroup.ColumnChunks()[leaf.Index()]); err != nil {
   869  				return err
   870  			}
   871  		}
   872  		return nil
   873  	})
   874  }
   875  
   876  func createParquetFile(rows rows, options ...parquet.WriterOption) (*parquet.File, error) {
   877  	buffer := new(bytes.Buffer)
   878  
   879  	if err := writeParquetFile(buffer, rows, options...); err != nil {
   880  		return nil, err
   881  	}
   882  
   883  	reader := bytes.NewReader(buffer.Bytes())
   884  	return parquet.OpenFile(reader, reader.Size())
   885  }
   886  
   887  func writeParquetFile(w io.Writer, rows rows, options ...parquet.WriterOption) error {
   888  	writer := parquet.NewWriter(w, options...)
   889  
   890  	for _, row := range rows {
   891  		if err := writer.Write(row); err != nil {
   892  			return err
   893  		}
   894  	}
   895  
   896  	return writer.Close()
   897  }
   898  
   899  func writeParquetFileWithBuffer(w io.Writer, rows rows, options ...parquet.WriterOption) error {
   900  	buffer := parquet.NewBuffer()
   901  	for _, row := range rows {
   902  		if err := buffer.Write(row); err != nil {
   903  			return err
   904  		}
   905  	}
   906  
   907  	writer := parquet.NewWriter(w, options...)
   908  	numRows, err := copyRowsAndClose(writer, buffer.Rows())
   909  	if err != nil {
   910  		return err
   911  	}
   912  	if numRows != int64(len(rows)) {
   913  		return fmt.Errorf("wrong number of rows written from buffer to file: want=%d got=%d", len(rows), numRows)
   914  	}
   915  	return writer.Close()
   916  }
   917  
   918  type rows []interface{}
   919  
   920  func makeRows(any interface{}) rows {
   921  	if v, ok := any.([]interface{}); ok {
   922  		return rows(v)
   923  	}
   924  	value := reflect.ValueOf(any)
   925  	slice := make([]interface{}, value.Len())
   926  	for i := range slice {
   927  		slice[i] = value.Index(i).Interface()
   928  	}
   929  	return rows(slice)
   930  }
   931  
   932  func randValueFuncOf(t parquet.Type) func(*rand.Rand) parquet.Value {
   933  	switch k := t.Kind(); k {
   934  	case parquet.Boolean:
   935  		return func(r *rand.Rand) parquet.Value {
   936  			return parquet.ValueOf(r.Float64() < 0.5)
   937  		}
   938  
   939  	case parquet.Int32:
   940  		return func(r *rand.Rand) parquet.Value {
   941  			return parquet.ValueOf(r.Int31())
   942  		}
   943  
   944  	case parquet.Int64:
   945  		return func(r *rand.Rand) parquet.Value {
   946  			return parquet.ValueOf(r.Int63())
   947  		}
   948  
   949  	case parquet.Int96:
   950  		return func(r *rand.Rand) parquet.Value {
   951  			return parquet.ValueOf(deprecated.Int96{
   952  				0: r.Uint32(),
   953  				1: r.Uint32(),
   954  				2: r.Uint32(),
   955  			})
   956  		}
   957  
   958  	case parquet.Float:
   959  		return func(r *rand.Rand) parquet.Value {
   960  			return parquet.ValueOf(r.Float32())
   961  		}
   962  
   963  	case parquet.Double:
   964  		return func(r *rand.Rand) parquet.Value {
   965  			return parquet.ValueOf(r.Float64())
   966  		}
   967  
   968  	case parquet.ByteArray:
   969  		return func(r *rand.Rand) parquet.Value {
   970  			n := r.Intn(49) + 1
   971  			b := make([]byte, n)
   972  			const characters = "1234567890qwertyuiopasdfghjklzxcvbnm "
   973  			for i := range b {
   974  				b[i] = characters[r.Intn(len(characters))]
   975  			}
   976  			return parquet.ValueOf(b)
   977  		}
   978  
   979  	case parquet.FixedLenByteArray:
   980  		arrayType := reflect.ArrayOf(t.Length(), reflect.TypeOf(byte(0)))
   981  		return func(r *rand.Rand) parquet.Value {
   982  			b := make([]byte, arrayType.Len())
   983  			r.Read(b)
   984  			v := reflect.New(arrayType).Elem()
   985  			reflect.Copy(v, reflect.ValueOf(b))
   986  			return parquet.ValueOf(v.Interface())
   987  		}
   988  
   989  	default:
   990  		panic("NOT IMPLEMENTED")
   991  	}
   992  }
   993  
   994  func copyRowsAndClose(w parquet.RowWriter, r parquet.Rows) (int64, error) {
   995  	defer r.Close()
   996  	return parquet.CopyRows(w, r)
   997  }
   998  
   999  func benchmarkRowsPerSecond(b *testing.B, f func() int) {
  1000  	b.ResetTimer()
  1001  	start := time.Now()
  1002  	numRows := int64(0)
  1003  
  1004  	for i := 0; i < b.N; i++ {
  1005  		n := f()
  1006  		numRows += int64(n)
  1007  	}
  1008  
  1009  	seconds := time.Since(start).Seconds()
  1010  	b.ReportMetric(float64(numRows)/seconds, "row/s")
  1011  }
  1012  
  1013  func generateString(r *rand.Rand, n int) string {
  1014  	const characters = "1234567890qwertyuiopasdfghjklzxcvbnm"
  1015  	b := new(strings.Builder)
  1016  	for i := 0; i < n; i++ {
  1017  		b.WriteByte(characters[r.Intn(len(characters))])
  1018  	}
  1019  	return b.String()
  1020  }
  1021  
  1022  var quickCheckConfig = quick.Config{
  1023  	Sizes: []int{
  1024  		0, 1, 2, 3, 4, 5, 6, 7, 8, 9,
  1025  		10, 20, 30, 40, 50, 123,
  1026  		4096 + 1,
  1027  	},
  1028  }
  1029  
  1030  func quickCheck(f interface{}) error {
  1031  	return quickCheckConfig.Check(f)
  1032  }
  1033  
  1034  func TestParquetAnyValueConversions(t *testing.T) {
  1035  	// This test runs conversions to/from any values with edge case schemas.
  1036  
  1037  	type obj = map[string]any
  1038  	type arr = []any
  1039  
  1040  	for _, test := range []struct {
  1041  		name           string
  1042  		input          any
  1043  		explicitOutput any // Only set if we expect a difference from the input
  1044  		schema         parquet.Group
  1045  	}{
  1046  		{
  1047  			name: "simple strings",
  1048  			input: obj{
  1049  				"A": "foo",
  1050  				"B": "bar",
  1051  				"C": "baz",
  1052  			},
  1053  			schema: parquet.Group{
  1054  				"A": parquet.String(),
  1055  				"B": parquet.String(),
  1056  				"C": parquet.String(),
  1057  			},
  1058  		},
  1059  		{
  1060  			name: "simple strings with nil",
  1061  			input: obj{
  1062  				"A": "foo",
  1063  				"B": (*string)(nil),
  1064  				"C": nil,
  1065  			},
  1066  			explicitOutput: obj{
  1067  				"A": "foo",
  1068  				"B": "",
  1069  				"C": "",
  1070  			},
  1071  			schema: parquet.Group{
  1072  				"A": parquet.String(),
  1073  				"B": parquet.String(),
  1074  				"C": parquet.String(),
  1075  			},
  1076  		},
  1077  		{
  1078  			name: "simple groups with nil",
  1079  			input: obj{
  1080  				"A": obj{
  1081  					"AA": "foo",
  1082  				},
  1083  				"B": nil,
  1084  				"C": (*obj)(nil),
  1085  				"D": obj{
  1086  					"DA": "bar",
  1087  				},
  1088  			},
  1089  			explicitOutput: obj{
  1090  				"A": obj{
  1091  					"AA": "foo",
  1092  				},
  1093  				"B": obj{
  1094  					"BA": "",
  1095  				},
  1096  				"C": obj{
  1097  					"CA": "",
  1098  				},
  1099  				"D": obj{
  1100  					"DA": "bar",
  1101  				},
  1102  			},
  1103  			schema: parquet.Group{
  1104  				"A": parquet.Group{
  1105  					"AA": parquet.String(),
  1106  				},
  1107  				"B": parquet.Group{
  1108  					"BA": parquet.String(),
  1109  				},
  1110  				"C": parquet.Group{
  1111  					"CA": parquet.String(),
  1112  				},
  1113  				"D": parquet.Group{
  1114  					"DA": parquet.String(),
  1115  				},
  1116  			},
  1117  		},
  1118  		{
  1119  			name: "simple values",
  1120  			input: obj{
  1121  				"A": "foo",
  1122  				"B": int64(5),
  1123  				"C": 0.5,
  1124  			},
  1125  			schema: parquet.Group{
  1126  				"A": parquet.String(),
  1127  				"B": parquet.Int(64),
  1128  				"C": parquet.Leaf(parquet.DoubleType),
  1129  			},
  1130  		},
  1131  		{
  1132  			name: "repeated values",
  1133  			input: obj{
  1134  				"A": arr{"foo", "bar", "baz"},
  1135  				"B": arr{int64(5), int64(6)},
  1136  				"C": arr{0.5},
  1137  			},
  1138  			schema: parquet.Group{
  1139  				"A": parquet.Repeated(parquet.String()),
  1140  				"B": parquet.Repeated(parquet.Int(64)),
  1141  				"C": parquet.Repeated(parquet.Leaf(parquet.DoubleType)),
  1142  			},
  1143  		},
  1144  		{
  1145  			name: "nested groups",
  1146  			input: obj{
  1147  				"A": obj{
  1148  					"B": obj{
  1149  						"C": "here we are",
  1150  					},
  1151  				},
  1152  			},
  1153  			schema: parquet.Group{
  1154  				"A": parquet.Group{
  1155  					"B": parquet.Group{
  1156  						"C": parquet.String(),
  1157  					},
  1158  				},
  1159  			},
  1160  		},
  1161  		{
  1162  			name: "nested repeated groups",
  1163  			input: obj{
  1164  				"A": arr{
  1165  					obj{
  1166  						"B": arr{
  1167  							obj{"C": arr{"first", "second"}},
  1168  							obj{"C": arr{"third", "fourth"}},
  1169  							obj{"C": arr{"fifth"}},
  1170  						},
  1171  					},
  1172  					obj{
  1173  						"B": arr{
  1174  							obj{"C": arr{"sixth"}},
  1175  						},
  1176  					},
  1177  				},
  1178  			},
  1179  			schema: parquet.Group{
  1180  				"A": parquet.Repeated(parquet.Group{
  1181  					"B": parquet.Repeated(parquet.Group{
  1182  						"C": parquet.Repeated(parquet.String()),
  1183  					}),
  1184  				}),
  1185  			},
  1186  		},
  1187  		{
  1188  			name: "optional values",
  1189  			input: obj{
  1190  				"A": "foo",
  1191  				"B": nil,
  1192  				"C": "baz",
  1193  			},
  1194  			schema: parquet.Group{
  1195  				"A": parquet.Optional(parquet.String()),
  1196  				"B": parquet.Optional(parquet.String()),
  1197  				"C": parquet.Optional(parquet.String()),
  1198  			},
  1199  		},
  1200  		{
  1201  			name: "nested optional groups",
  1202  			input: obj{
  1203  				"A": obj{
  1204  					"B": obj{
  1205  						"C": "here we are",
  1206  					},
  1207  					"D": nil,
  1208  				},
  1209  			},
  1210  			schema: parquet.Group{
  1211  				"A": parquet.Group{
  1212  					"B": parquet.Optional(parquet.Group{
  1213  						"C": parquet.String(),
  1214  					}),
  1215  					"D": parquet.Optional(parquet.Group{
  1216  						"E": parquet.String(),
  1217  					}),
  1218  				},
  1219  			},
  1220  		},
  1221  	} {
  1222  		test := test
  1223  		t.Run(test.name, func(t *testing.T) {
  1224  			var buf bytes.Buffer
  1225  			pWtr := parquet.NewGenericWriter[any](&buf, parquet.NewSchema("", test.schema))
  1226  			if _, err := pWtr.Write([]any{test.input}); err != nil {
  1227  				t.Fatal(err)
  1228  			}
  1229  			if err := pWtr.Close(); err != nil {
  1230  				t.Fatal(err)
  1231  			}
  1232  
  1233  			pRdr := parquet.NewGenericReader[any](bytes.NewReader(buf.Bytes()))
  1234  			outRows := make([]any, 1)
  1235  			if _, err := pRdr.Read(outRows); err != nil {
  1236  				t.Fatal(err)
  1237  			}
  1238  			if err := pRdr.Close(); err != nil {
  1239  				t.Fatal(err)
  1240  			}
  1241  
  1242  			expected := test.input
  1243  			if test.explicitOutput != nil {
  1244  				expected = test.explicitOutput
  1245  			}
  1246  
  1247  			if value1, value2 := expected, outRows[0]; !reflect.DeepEqual(value1, value2) {
  1248  				t.Errorf("value mismatch: want=%+v got=%+v", value1, value2)
  1249  			}
  1250  		})
  1251  	}
  1252  }