github.com/parquet-go/parquet-go@v0.21.1-0.20240501160520-b3c3a0c3ed6f/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  	buf := new(bytes.Buffer)
   406  	w := parquet.NewGenericWriter[Outer](buf, schema)
   407  	_, err := w.Write(writeRows)
   408  	if err != nil {
   409  		t.Fatal("write error: ", err)
   410  	}
   411  	w.Close()
   412  
   413  	file := bytes.NewReader(buf.Bytes())
   414  	readRows, err := parquet.Read[Outer](file, file.Size())
   415  	if err != nil {
   416  		t.Fatal("read error: ", err)
   417  	}
   418  
   419  	assertRowsEqual(t, writeRows, readRows)
   420  }
   421  
   422  func TestReadFileGenericMultipleRowGroupsMultiplePages(t *testing.T) {
   423  	type MyRow struct {
   424  		ID    [16]byte `parquet:"id,delta,uuid"`
   425  		File  string   `parquet:"file,dict,zstd"`
   426  		Index int64    `parquet:"index,delta,zstd"`
   427  	}
   428  
   429  	numRows := 20_000
   430  	maxPageBytes := 5000
   431  
   432  	tmp, err := os.CreateTemp("/tmp", "*.parquet")
   433  	if err != nil {
   434  		t.Fatal("os.CreateTemp: ", err)
   435  	}
   436  	path := tmp.Name()
   437  	defer os.Remove(path)
   438  	t.Log("file:", path)
   439  
   440  	// The page buffer size ensures we get multiple pages out of this example.
   441  	w := parquet.NewGenericWriter[MyRow](tmp, parquet.PageBufferSize(maxPageBytes))
   442  	// Need to write 1 row at a time here as writing many at once disregards PageBufferSize option.
   443  	for i := 0; i < numRows; i++ {
   444  		row := MyRow{
   445  			ID:    [16]byte{15: byte(i)},
   446  			File:  "hi" + fmt.Sprint(i),
   447  			Index: int64(i),
   448  		}
   449  		_, err := w.Write([]MyRow{row})
   450  		if err != nil {
   451  			t.Fatal("w.Write: ", err)
   452  		}
   453  		// Flush writes rows as row group. 4 total (20k/5k) in this file.
   454  		if (i+1)%maxPageBytes == 0 {
   455  			err = w.Flush()
   456  			if err != nil {
   457  				t.Fatal("w.Flush: ", err)
   458  			}
   459  		}
   460  	}
   461  	err = w.Close()
   462  	if err != nil {
   463  		t.Fatal("w.Close: ", err)
   464  	}
   465  	err = tmp.Close()
   466  	if err != nil {
   467  		t.Fatal("tmp.Close: ", err)
   468  	}
   469  
   470  	rows, err := parquet.ReadFile[MyRow](path)
   471  	if err != nil {
   472  		t.Fatal("parquet.ReadFile: ", err)
   473  	}
   474  
   475  	if len(rows) != numRows {
   476  		t.Fatalf("not enough values were read: want=%d got=%d", len(rows), numRows)
   477  	}
   478  	for i, row := range rows {
   479  		id := [16]byte{15: byte(i)}
   480  		file := "hi" + fmt.Sprint(i)
   481  		index := int64(i)
   482  
   483  		if row.ID != id || row.File != file || row.Index != index {
   484  			t.Fatalf("rows mismatch at index: %d got: %+v", i, row)
   485  		}
   486  	}
   487  }
   488  
   489  func assertRowsEqual[T any](t *testing.T, rows1, rows2 []T) {
   490  	if !reflect.DeepEqual(rows1, rows2) {
   491  		t.Error("rows mismatch")
   492  
   493  		t.Log("want:")
   494  		logRows(t, rows1)
   495  
   496  		t.Log("got:")
   497  		logRows(t, rows2)
   498  	}
   499  }
   500  
   501  func logRows[T any](t *testing.T, rows []T) {
   502  	for _, row := range rows {
   503  		t.Logf(". %#v\n", row)
   504  	}
   505  }
   506  
   507  func TestNestedPointer(t *testing.T) {
   508  	type InnerStruct struct {
   509  		InnerField string
   510  	}
   511  
   512  	type SliceElement struct {
   513  		Inner *InnerStruct
   514  	}
   515  
   516  	type Outer struct {
   517  		Slice []*SliceElement
   518  	}
   519  	value := "inner-string"
   520  	in := &Outer{
   521  		Slice: []*SliceElement{
   522  			{
   523  				Inner: &InnerStruct{
   524  					InnerField: value,
   525  				},
   526  			},
   527  		},
   528  	}
   529  
   530  	var f bytes.Buffer
   531  
   532  	pw := parquet.NewGenericWriter[*Outer](&f)
   533  	_, err := pw.Write([]*Outer{in})
   534  	if err != nil {
   535  		t.Fatal(err)
   536  	}
   537  
   538  	err = pw.Close()
   539  	if err != nil {
   540  		t.Fatal(err)
   541  	}
   542  
   543  	pr := parquet.NewGenericReader[*Outer](bytes.NewReader(f.Bytes()))
   544  
   545  	out := make([]*Outer, 1)
   546  	_, err = pr.Read(out)
   547  	if err != nil {
   548  		t.Fatal(err)
   549  	}
   550  	pr.Close()
   551  	if want, got := value, out[0].Slice[0].Inner.InnerField; want != got {
   552  		t.Error("failed to set inner field pointer")
   553  	}
   554  }
   555  
   556  type benchmarkRowType struct {
   557  	ID    [16]byte `parquet:"id,uuid"`
   558  	Value float64  `parquet:"value"`
   559  }
   560  
   561  func (row benchmarkRowType) generate(prng *rand.Rand) benchmarkRowType {
   562  	prng.Read(row.ID[:])
   563  	row.Value = prng.Float64()
   564  	return row
   565  }
   566  
   567  type paddedBooleanColumn struct {
   568  	Value bool
   569  	_     [3]byte
   570  }
   571  
   572  func (row paddedBooleanColumn) generate(prng *rand.Rand) paddedBooleanColumn {
   573  	return paddedBooleanColumn{Value: prng.Int()%2 == 0}
   574  }
   575  
   576  type booleanColumn struct {
   577  	Value bool
   578  }
   579  
   580  func (row booleanColumn) generate(prng *rand.Rand) booleanColumn {
   581  	return booleanColumn{Value: prng.Int()%2 == 0}
   582  }
   583  
   584  type int32Column struct {
   585  	Value int32 `parquet:",delta"`
   586  }
   587  
   588  func (row int32Column) generate(prng *rand.Rand) int32Column {
   589  	return int32Column{Value: prng.Int31n(100)}
   590  }
   591  
   592  type int64Column struct {
   593  	Value int64 `parquet:",delta"`
   594  }
   595  
   596  func (row int64Column) generate(prng *rand.Rand) int64Column {
   597  	return int64Column{Value: prng.Int63n(100)}
   598  }
   599  
   600  type int96Column struct {
   601  	Value deprecated.Int96
   602  }
   603  
   604  func (row int96Column) generate(prng *rand.Rand) int96Column {
   605  	row.Value[0] = prng.Uint32()
   606  	row.Value[1] = prng.Uint32()
   607  	row.Value[2] = prng.Uint32()
   608  	return row
   609  }
   610  
   611  type floatColumn struct {
   612  	Value float32
   613  }
   614  
   615  func (row floatColumn) generate(prng *rand.Rand) floatColumn {
   616  	return floatColumn{Value: prng.Float32()}
   617  }
   618  
   619  type doubleColumn struct {
   620  	Value float64
   621  }
   622  
   623  func (row doubleColumn) generate(prng *rand.Rand) doubleColumn {
   624  	return doubleColumn{Value: prng.Float64()}
   625  }
   626  
   627  type byteArrayColumn struct {
   628  	Value []byte
   629  }
   630  
   631  func (row byteArrayColumn) generate(prng *rand.Rand) byteArrayColumn {
   632  	row.Value = make([]byte, prng.Intn(10))
   633  	prng.Read(row.Value)
   634  	return row
   635  }
   636  
   637  type fixedLenByteArrayColumn struct {
   638  	Value [10]byte
   639  }
   640  
   641  func (row fixedLenByteArrayColumn) generate(prng *rand.Rand) fixedLenByteArrayColumn {
   642  	prng.Read(row.Value[:])
   643  	return row
   644  }
   645  
   646  type stringColumn struct {
   647  	Value string
   648  }
   649  
   650  func (row stringColumn) generate(prng *rand.Rand) stringColumn {
   651  	return stringColumn{Value: generateString(prng, 10)}
   652  }
   653  
   654  type indexedStringColumn struct {
   655  	Value string `parquet:",dict"`
   656  }
   657  
   658  func (row indexedStringColumn) generate(prng *rand.Rand) indexedStringColumn {
   659  	return indexedStringColumn{Value: generateString(prng, 10)}
   660  }
   661  
   662  type uuidColumn struct {
   663  	Value uuid.UUID `parquet:",delta"`
   664  }
   665  
   666  func (row uuidColumn) generate(prng *rand.Rand) uuidColumn {
   667  	prng.Read(row.Value[:])
   668  	return row
   669  }
   670  
   671  type timeColumn struct {
   672  	Value time.Time
   673  }
   674  
   675  func (row timeColumn) generate(prng *rand.Rand) timeColumn {
   676  	t := time.Unix(0, prng.Int63()).UTC()
   677  	return timeColumn{Value: t}
   678  }
   679  
   680  type timeInMillisColumn struct {
   681  	Value time.Time `parquet:",timestamp(millisecond)"`
   682  }
   683  
   684  func (row timeInMillisColumn) generate(prng *rand.Rand) timeInMillisColumn {
   685  	t := time.Unix(0, prng.Int63()).UTC()
   686  	return timeInMillisColumn{Value: t}
   687  }
   688  
   689  type decimalColumn struct {
   690  	Value int64 `parquet:",decimal(0:3)"`
   691  }
   692  
   693  func (row decimalColumn) generate(prng *rand.Rand) decimalColumn {
   694  	return decimalColumn{Value: prng.Int63()}
   695  }
   696  
   697  type mapColumn struct {
   698  	Value map[utf8string]int
   699  }
   700  
   701  func (row mapColumn) generate(prng *rand.Rand) mapColumn {
   702  	n := prng.Intn(10)
   703  	row.Value = make(map[utf8string]int, n)
   704  	for i := 0; i < n; i++ {
   705  		row.Value[utf8string(generateString(prng, 8))] = prng.Intn(100)
   706  	}
   707  	return row
   708  }
   709  
   710  type addressBook struct {
   711  	Owner             utf8string   `parquet:",plain"`
   712  	OwnerPhoneNumbers []utf8string `parquet:",plain"`
   713  	Contacts          []contact
   714  }
   715  
   716  type contact struct {
   717  	Name        utf8string `parquet:",plain"`
   718  	PhoneNumber utf8string `parquet:",plain"`
   719  }
   720  
   721  func (row contact) generate(prng *rand.Rand) contact {
   722  	return contact{
   723  		Name:        utf8string(generateString(prng, 16)),
   724  		PhoneNumber: utf8string(generateString(prng, 10)),
   725  	}
   726  }
   727  
   728  type optionalInt32Column struct {
   729  	Value int32 `parquet:",optional"`
   730  }
   731  
   732  func (row optionalInt32Column) generate(prng *rand.Rand) optionalInt32Column {
   733  	return optionalInt32Column{Value: prng.Int31n(100)}
   734  }
   735  
   736  type repeatedInt32Column struct {
   737  	Values []int32
   738  }
   739  
   740  func (row repeatedInt32Column) generate(prng *rand.Rand) repeatedInt32Column {
   741  	row.Values = make([]int32, prng.Intn(10))
   742  	for i := range row.Values {
   743  		row.Values[i] = prng.Int31n(10)
   744  	}
   745  	return row
   746  }
   747  
   748  type listColumn2 struct {
   749  	Value utf8string `parquet:",optional"`
   750  }
   751  
   752  type listColumn1 struct {
   753  	List2 []listColumn2 `parquet:",list"`
   754  }
   755  
   756  type listColumn0 struct {
   757  	List1 []listColumn1 `parquet:",list"`
   758  }
   759  
   760  type nestedListColumn1 struct {
   761  	Level3 []utf8string `parquet:"level3"`
   762  }
   763  
   764  type nestedListColumn struct {
   765  	Level1 []nestedListColumn1 `parquet:"level1"`
   766  	Level2 []utf8string        `parquet:"level2"`
   767  }
   768  
   769  type utf8string string
   770  
   771  func (utf8string) Generate(rand *rand.Rand, size int) reflect.Value {
   772  	const characters = "abcdefghijklmnopqrstuvwxyz1234567890"
   773  	const maxSize = 10
   774  	if size > maxSize {
   775  		size = maxSize
   776  	}
   777  	n := rand.Intn(size)
   778  	b := make([]byte, n)
   779  	for i := range b {
   780  		b[i] = characters[rand.Intn(len(characters))]
   781  	}
   782  	return reflect.ValueOf(utf8string(b))
   783  }
   784  
   785  type Contact struct {
   786  	Name        string `parquet:"name"`
   787  	PhoneNumber string `parquet:"phoneNumber,optional,zstd"`
   788  }
   789  
   790  type AddressBook struct {
   791  	Owner             string    `parquet:"owner,zstd"`
   792  	OwnerPhoneNumbers []string  `parquet:"ownerPhoneNumbers,gzip"`
   793  	Contacts          []Contact `parquet:"contacts"`
   794  }
   795  
   796  func forEachLeafColumn(col *parquet.Column, do func(*parquet.Column) error) error {
   797  	children := col.Columns()
   798  
   799  	if len(children) == 0 {
   800  		return do(col)
   801  	}
   802  
   803  	for _, child := range children {
   804  		if err := forEachLeafColumn(child, do); err != nil {
   805  			return err
   806  		}
   807  	}
   808  
   809  	return nil
   810  }
   811  
   812  func forEachPage(pages parquet.PageReader, do func(parquet.Page) error) error {
   813  	doAndReleasePage := func(page parquet.Page) error {
   814  		defer parquet.Release(page)
   815  		return do(page)
   816  	}
   817  
   818  	for {
   819  		p, err := pages.ReadPage()
   820  		if err != nil {
   821  			if err == io.EOF {
   822  				err = nil
   823  			}
   824  			return err
   825  		}
   826  		if err := doAndReleasePage(p); err != nil {
   827  			return err
   828  		}
   829  	}
   830  }
   831  
   832  func forEachValue(values parquet.ValueReader, do func(parquet.Value) error) error {
   833  	buffer := [3]parquet.Value{}
   834  	for {
   835  		n, err := values.ReadValues(buffer[:])
   836  		for _, v := range buffer[:n] {
   837  			if err := do(v); err != nil {
   838  				return err
   839  			}
   840  		}
   841  		if err != nil {
   842  			if err == io.EOF {
   843  				err = nil
   844  			}
   845  			return err
   846  		}
   847  	}
   848  }
   849  
   850  func forEachColumnPage(col *parquet.Column, do func(*parquet.Column, parquet.Page) error) error {
   851  	return forEachLeafColumn(col, func(leaf *parquet.Column) error {
   852  		pages := leaf.Pages()
   853  		defer pages.Close()
   854  		return forEachPage(pages, func(page parquet.Page) error { return do(leaf, page) })
   855  	})
   856  }
   857  
   858  func forEachColumnValue(col *parquet.Column, do func(*parquet.Column, parquet.Value) error) error {
   859  	return forEachColumnPage(col, func(leaf *parquet.Column, page parquet.Page) error {
   860  		return forEachValue(page.Values(), func(value parquet.Value) error { return do(leaf, value) })
   861  	})
   862  }
   863  
   864  func forEachColumnChunk(file *parquet.File, do func(*parquet.Column, parquet.ColumnChunk) error) error {
   865  	return forEachLeafColumn(file.Root(), func(leaf *parquet.Column) error {
   866  		for _, rowGroup := range file.RowGroups() {
   867  			if err := do(leaf, rowGroup.ColumnChunks()[leaf.Index()]); err != nil {
   868  				return err
   869  			}
   870  		}
   871  		return nil
   872  	})
   873  }
   874  
   875  func createParquetFile(rows rows, options ...parquet.WriterOption) (*parquet.File, error) {
   876  	buffer := new(bytes.Buffer)
   877  
   878  	if err := writeParquetFile(buffer, rows, options...); err != nil {
   879  		return nil, err
   880  	}
   881  
   882  	reader := bytes.NewReader(buffer.Bytes())
   883  	return parquet.OpenFile(reader, reader.Size())
   884  }
   885  
   886  func writeParquetFile(w io.Writer, rows rows, options ...parquet.WriterOption) error {
   887  	writer := parquet.NewWriter(w, options...)
   888  
   889  	for _, row := range rows {
   890  		if err := writer.Write(row); err != nil {
   891  			return err
   892  		}
   893  	}
   894  
   895  	return writer.Close()
   896  }
   897  
   898  func writeParquetFileWithBuffer(w io.Writer, rows rows, options ...parquet.WriterOption) error {
   899  	buffer := parquet.NewBuffer()
   900  	for _, row := range rows {
   901  		if err := buffer.Write(row); err != nil {
   902  			return err
   903  		}
   904  	}
   905  
   906  	writer := parquet.NewWriter(w, options...)
   907  	numRows, err := copyRowsAndClose(writer, buffer.Rows())
   908  	if err != nil {
   909  		return err
   910  	}
   911  	if numRows != int64(len(rows)) {
   912  		return fmt.Errorf("wrong number of rows written from buffer to file: want=%d got=%d", len(rows), numRows)
   913  	}
   914  	return writer.Close()
   915  }
   916  
   917  type rows []interface{}
   918  
   919  func makeRows(any interface{}) rows {
   920  	if v, ok := any.([]interface{}); ok {
   921  		return rows(v)
   922  	}
   923  	value := reflect.ValueOf(any)
   924  	slice := make([]interface{}, value.Len())
   925  	for i := range slice {
   926  		slice[i] = value.Index(i).Interface()
   927  	}
   928  	return rows(slice)
   929  }
   930  
   931  func randValueFuncOf(t parquet.Type) func(*rand.Rand) parquet.Value {
   932  	switch k := t.Kind(); k {
   933  	case parquet.Boolean:
   934  		return func(r *rand.Rand) parquet.Value {
   935  			return parquet.ValueOf(r.Float64() < 0.5)
   936  		}
   937  
   938  	case parquet.Int32:
   939  		return func(r *rand.Rand) parquet.Value {
   940  			return parquet.ValueOf(r.Int31())
   941  		}
   942  
   943  	case parquet.Int64:
   944  		return func(r *rand.Rand) parquet.Value {
   945  			return parquet.ValueOf(r.Int63())
   946  		}
   947  
   948  	case parquet.Int96:
   949  		return func(r *rand.Rand) parquet.Value {
   950  			return parquet.ValueOf(deprecated.Int96{
   951  				0: r.Uint32(),
   952  				1: r.Uint32(),
   953  				2: r.Uint32(),
   954  			})
   955  		}
   956  
   957  	case parquet.Float:
   958  		return func(r *rand.Rand) parquet.Value {
   959  			return parquet.ValueOf(r.Float32())
   960  		}
   961  
   962  	case parquet.Double:
   963  		return func(r *rand.Rand) parquet.Value {
   964  			return parquet.ValueOf(r.Float64())
   965  		}
   966  
   967  	case parquet.ByteArray:
   968  		return func(r *rand.Rand) parquet.Value {
   969  			n := r.Intn(49) + 1
   970  			b := make([]byte, n)
   971  			const characters = "1234567890qwertyuiopasdfghjklzxcvbnm "
   972  			for i := range b {
   973  				b[i] = characters[r.Intn(len(characters))]
   974  			}
   975  			return parquet.ValueOf(b)
   976  		}
   977  
   978  	case parquet.FixedLenByteArray:
   979  		arrayType := reflect.ArrayOf(t.Length(), reflect.TypeOf(byte(0)))
   980  		return func(r *rand.Rand) parquet.Value {
   981  			b := make([]byte, arrayType.Len())
   982  			r.Read(b)
   983  			v := reflect.New(arrayType).Elem()
   984  			reflect.Copy(v, reflect.ValueOf(b))
   985  			return parquet.ValueOf(v.Interface())
   986  		}
   987  
   988  	default:
   989  		panic("NOT IMPLEMENTED")
   990  	}
   991  }
   992  
   993  func copyRowsAndClose(w parquet.RowWriter, r parquet.Rows) (int64, error) {
   994  	defer r.Close()
   995  	return parquet.CopyRows(w, r)
   996  }
   997  
   998  func benchmarkRowsPerSecond(b *testing.B, f func() int) {
   999  	b.ResetTimer()
  1000  	start := time.Now()
  1001  	numRows := int64(0)
  1002  
  1003  	for i := 0; i < b.N; i++ {
  1004  		n := f()
  1005  		numRows += int64(n)
  1006  	}
  1007  
  1008  	seconds := time.Since(start).Seconds()
  1009  	b.ReportMetric(float64(numRows)/seconds, "row/s")
  1010  }
  1011  
  1012  func generateString(r *rand.Rand, n int) string {
  1013  	const characters = "1234567890qwertyuiopasdfghjklzxcvbnm"
  1014  	b := new(strings.Builder)
  1015  	for i := 0; i < n; i++ {
  1016  		b.WriteByte(characters[r.Intn(len(characters))])
  1017  	}
  1018  	return b.String()
  1019  }
  1020  
  1021  var quickCheckConfig = quick.Config{
  1022  	Sizes: []int{
  1023  		0, 1, 2, 3, 4, 5, 6, 7, 8, 9,
  1024  		10, 20, 30, 40, 50, 123,
  1025  		4096 + 1,
  1026  	},
  1027  }
  1028  
  1029  func quickCheck(f interface{}) error {
  1030  	return quickCheckConfig.Check(f)
  1031  }
  1032  
  1033  func TestParquetAnyValueConversions(t *testing.T) {
  1034  	// This test runs conversions to/from any values with edge case schemas.
  1035  
  1036  	type obj = map[string]any
  1037  	type arr = []any
  1038  
  1039  	for _, test := range []struct {
  1040  		name           string
  1041  		input          any
  1042  		explicitOutput any // Only set if we expect a difference from the input
  1043  		schema         parquet.Group
  1044  	}{
  1045  		{
  1046  			name: "simple strings",
  1047  			input: obj{
  1048  				"A": "foo",
  1049  				"B": "bar",
  1050  				"C": "baz",
  1051  			},
  1052  			schema: parquet.Group{
  1053  				"A": parquet.String(),
  1054  				"B": parquet.String(),
  1055  				"C": parquet.String(),
  1056  			},
  1057  		},
  1058  		{
  1059  			name: "simple strings with nil",
  1060  			input: obj{
  1061  				"A": "foo",
  1062  				"B": (*string)(nil),
  1063  				"C": nil,
  1064  			},
  1065  			explicitOutput: obj{
  1066  				"A": "foo",
  1067  				"B": "",
  1068  				"C": "",
  1069  			},
  1070  			schema: parquet.Group{
  1071  				"A": parquet.String(),
  1072  				"B": parquet.String(),
  1073  				"C": parquet.String(),
  1074  			},
  1075  		},
  1076  		{
  1077  			name: "simple groups with nil",
  1078  			input: obj{
  1079  				"A": obj{
  1080  					"AA": "foo",
  1081  				},
  1082  				"B": nil,
  1083  				"C": (*obj)(nil),
  1084  				"D": obj{
  1085  					"DA": "bar",
  1086  				},
  1087  			},
  1088  			explicitOutput: obj{
  1089  				"A": obj{
  1090  					"AA": "foo",
  1091  				},
  1092  				"B": obj{
  1093  					"BA": "",
  1094  				},
  1095  				"C": obj{
  1096  					"CA": "",
  1097  				},
  1098  				"D": obj{
  1099  					"DA": "bar",
  1100  				},
  1101  			},
  1102  			schema: parquet.Group{
  1103  				"A": parquet.Group{
  1104  					"AA": parquet.String(),
  1105  				},
  1106  				"B": parquet.Group{
  1107  					"BA": parquet.String(),
  1108  				},
  1109  				"C": parquet.Group{
  1110  					"CA": parquet.String(),
  1111  				},
  1112  				"D": parquet.Group{
  1113  					"DA": parquet.String(),
  1114  				},
  1115  			},
  1116  		},
  1117  		{
  1118  			name: "simple values",
  1119  			input: obj{
  1120  				"A": "foo",
  1121  				"B": int64(5),
  1122  				"C": 0.5,
  1123  			},
  1124  			schema: parquet.Group{
  1125  				"A": parquet.String(),
  1126  				"B": parquet.Int(64),
  1127  				"C": parquet.Leaf(parquet.DoubleType),
  1128  			},
  1129  		},
  1130  		{
  1131  			name: "repeated values",
  1132  			input: obj{
  1133  				"A": arr{"foo", "bar", "baz"},
  1134  				"B": arr{int64(5), int64(6)},
  1135  				"C": arr{0.5},
  1136  			},
  1137  			schema: parquet.Group{
  1138  				"A": parquet.Repeated(parquet.String()),
  1139  				"B": parquet.Repeated(parquet.Int(64)),
  1140  				"C": parquet.Repeated(parquet.Leaf(parquet.DoubleType)),
  1141  			},
  1142  		},
  1143  		{
  1144  			name: "nested groups",
  1145  			input: obj{
  1146  				"A": obj{
  1147  					"B": obj{
  1148  						"C": "here we are",
  1149  					},
  1150  				},
  1151  			},
  1152  			schema: parquet.Group{
  1153  				"A": parquet.Group{
  1154  					"B": parquet.Group{
  1155  						"C": parquet.String(),
  1156  					},
  1157  				},
  1158  			},
  1159  		},
  1160  		{
  1161  			name: "nested repeated groups",
  1162  			input: obj{
  1163  				"A": arr{
  1164  					obj{
  1165  						"B": arr{
  1166  							obj{"C": arr{"first", "second"}},
  1167  							obj{"C": arr{"third", "fourth"}},
  1168  							obj{"C": arr{"fifth"}},
  1169  						},
  1170  					},
  1171  					obj{
  1172  						"B": arr{
  1173  							obj{"C": arr{"sixth"}},
  1174  						},
  1175  					},
  1176  				},
  1177  			},
  1178  			schema: parquet.Group{
  1179  				"A": parquet.Repeated(parquet.Group{
  1180  					"B": parquet.Repeated(parquet.Group{
  1181  						"C": parquet.Repeated(parquet.String()),
  1182  					}),
  1183  				}),
  1184  			},
  1185  		},
  1186  		{
  1187  			name: "optional values",
  1188  			input: obj{
  1189  				"A": "foo",
  1190  				"B": nil,
  1191  				"C": "baz",
  1192  			},
  1193  			schema: parquet.Group{
  1194  				"A": parquet.Optional(parquet.String()),
  1195  				"B": parquet.Optional(parquet.String()),
  1196  				"C": parquet.Optional(parquet.String()),
  1197  			},
  1198  		},
  1199  		{
  1200  			name: "nested optional groups",
  1201  			input: obj{
  1202  				"A": obj{
  1203  					"B": obj{
  1204  						"C": "here we are",
  1205  					},
  1206  					"D": nil,
  1207  				},
  1208  			},
  1209  			schema: parquet.Group{
  1210  				"A": parquet.Group{
  1211  					"B": parquet.Optional(parquet.Group{
  1212  						"C": parquet.String(),
  1213  					}),
  1214  					"D": parquet.Optional(parquet.Group{
  1215  						"E": parquet.String(),
  1216  					}),
  1217  				},
  1218  			},
  1219  		},
  1220  	} {
  1221  		test := test
  1222  		t.Run(test.name, func(t *testing.T) {
  1223  			var buf bytes.Buffer
  1224  			pWtr := parquet.NewGenericWriter[any](&buf, parquet.NewSchema("", test.schema))
  1225  			if _, err := pWtr.Write([]any{test.input}); err != nil {
  1226  				t.Fatal(err)
  1227  			}
  1228  			if err := pWtr.Close(); err != nil {
  1229  				t.Fatal(err)
  1230  			}
  1231  
  1232  			pRdr := parquet.NewGenericReader[any](bytes.NewReader(buf.Bytes()))
  1233  			outRows := make([]any, 1)
  1234  			if _, err := pRdr.Read(outRows); err != nil {
  1235  				t.Fatal(err)
  1236  			}
  1237  			if err := pRdr.Close(); err != nil {
  1238  				t.Fatal(err)
  1239  			}
  1240  
  1241  			expected := test.input
  1242  			if test.explicitOutput != nil {
  1243  				expected = test.explicitOutput
  1244  			}
  1245  
  1246  			if value1, value2 := expected, outRows[0]; !reflect.DeepEqual(value1, value2) {
  1247  				t.Errorf("value mismatch: want=%+v got=%+v", value1, value2)
  1248  			}
  1249  		})
  1250  	}
  1251  }
  1252  
  1253  func TestReadMapAsAny(t *testing.T) {
  1254  	type rec struct {
  1255  		N int            `parquet:"n"`
  1256  		M map[string]int `parquet:"m"`
  1257  	}
  1258  
  1259  	typed := []rec{{3, map[string]int{"a": 1, "b": 2}}}
  1260  	type obj = map[string]any
  1261  	anyd := []any{obj{"n": int64(3), "m": obj{"a": int64(1), "b": int64(2)}}}
  1262  
  1263  	var buf bytes.Buffer
  1264  	if err := parquet.Write(&buf, typed); err != nil {
  1265  		t.Fatal(err)
  1266  	}
  1267  
  1268  	data, size := bytes.NewReader(buf.Bytes()), int64(buf.Len())
  1269  	recs, err := parquet.Read[rec](data, size)
  1270  	if err != nil {
  1271  		t.Fatal(err)
  1272  	}
  1273  	if !reflect.DeepEqual(recs, typed) {
  1274  		t.Errorf("value mismatch: want=%+v got=%+v", typed, recs)
  1275  	}
  1276  
  1277  	anys, err := parquet.Read[any](data, size)
  1278  	if err != nil {
  1279  		t.Fatal(err)
  1280  	}
  1281  	if !reflect.DeepEqual(anys, anyd) {
  1282  		t.Errorf("value mismatch: want=%+v got=%+v", anyd, anys)
  1283  	}
  1284  
  1285  	vals, err := parquet.Read[any](data, size, parquet.NewSchema("", parquet.Group{
  1286  		"n": parquet.Int(64),
  1287  		"m": parquet.Map(parquet.String(), parquet.Int(64)),
  1288  	}))
  1289  	if err != nil {
  1290  		t.Fatal(err)
  1291  	}
  1292  	if !reflect.DeepEqual(vals, anyd) {
  1293  		t.Errorf("value mismatch: want=%+v got=%+v", anyd, anys)
  1294  	}
  1295  }