github.com/parquet-go/parquet-go@v0.21.1-0.20240501160520-b3c3a0c3ed6f/writer_test.go (about)

     1  package parquet_test
     2  
     3  import (
     4  	"bytes"
     5  	"fmt"
     6  	"io"
     7  	"math/rand"
     8  	"os"
     9  	"os/exec"
    10  	"reflect"
    11  	"strconv"
    12  	"strings"
    13  	"testing"
    14  
    15  	"github.com/google/uuid"
    16  	"github.com/hexops/gotextdiff"
    17  	"github.com/hexops/gotextdiff/myers"
    18  	"github.com/hexops/gotextdiff/span"
    19  
    20  	"github.com/parquet-go/parquet-go"
    21  	"github.com/parquet-go/parquet-go/compress"
    22  )
    23  
    24  const (
    25  	v1 = 1
    26  	v2 = 2
    27  )
    28  
    29  func BenchmarkGenericWriter(b *testing.B) {
    30  	benchmarkGenericWriter[benchmarkRowType](b)
    31  	benchmarkGenericWriter[booleanColumn](b)
    32  	benchmarkGenericWriter[int32Column](b)
    33  	benchmarkGenericWriter[int64Column](b)
    34  	benchmarkGenericWriter[floatColumn](b)
    35  	benchmarkGenericWriter[doubleColumn](b)
    36  	benchmarkGenericWriter[byteArrayColumn](b)
    37  	benchmarkGenericWriter[fixedLenByteArrayColumn](b)
    38  	benchmarkGenericWriter[stringColumn](b)
    39  	benchmarkGenericWriter[indexedStringColumn](b)
    40  	benchmarkGenericWriter[uuidColumn](b)
    41  	benchmarkGenericWriter[timeColumn](b)
    42  	benchmarkGenericWriter[timeInMillisColumn](b)
    43  	benchmarkGenericWriter[mapColumn](b)
    44  	benchmarkGenericWriter[decimalColumn](b)
    45  	benchmarkGenericWriter[contact](b)
    46  	benchmarkGenericWriter[paddedBooleanColumn](b)
    47  	benchmarkGenericWriter[optionalInt32Column](b)
    48  	benchmarkGenericWriter[repeatedInt32Column](b)
    49  }
    50  
    51  func benchmarkGenericWriter[Row generator[Row]](b *testing.B) {
    52  	var model Row
    53  	b.Run(reflect.TypeOf(model).Name(), func(b *testing.B) {
    54  		prng := rand.New(rand.NewSource(0))
    55  		rows := make([]Row, benchmarkNumRows)
    56  		for i := range rows {
    57  			rows[i] = rows[i].generate(prng)
    58  		}
    59  
    60  		b.Run("go1.17", func(b *testing.B) {
    61  			writer := parquet.NewWriter(io.Discard, parquet.SchemaOf(rows[0]))
    62  			i := 0
    63  			benchmarkRowsPerSecond(b, func() int {
    64  				for j := 0; j < benchmarkRowsPerStep; j++ {
    65  					if err := writer.Write(&rows[i]); err != nil {
    66  						b.Fatal(err)
    67  					}
    68  				}
    69  
    70  				i += benchmarkRowsPerStep
    71  				i %= benchmarkNumRows
    72  
    73  				if i == 0 {
    74  					writer.Close()
    75  					writer.Reset(io.Discard)
    76  				}
    77  				return benchmarkRowsPerStep
    78  			})
    79  		})
    80  
    81  		b.Run("go1.18", func(b *testing.B) {
    82  			writer := parquet.NewGenericWriter[Row](io.Discard)
    83  			i := 0
    84  			benchmarkRowsPerSecond(b, func() int {
    85  				n, err := writer.Write(rows[i : i+benchmarkRowsPerStep])
    86  				if err != nil {
    87  					b.Fatal(err)
    88  				}
    89  
    90  				i += benchmarkRowsPerStep
    91  				i %= benchmarkNumRows
    92  
    93  				if i == 0 {
    94  					writer.Close()
    95  					writer.Reset(io.Discard)
    96  				}
    97  				return n
    98  			})
    99  		})
   100  	})
   101  }
   102  
   103  func TestIssue272(t *testing.T) {
   104  	type T2 struct {
   105  		X string `parquet:",dict,optional"`
   106  	}
   107  
   108  	type T1 struct {
   109  		TA *T2
   110  		TB *T2
   111  	}
   112  
   113  	type T struct {
   114  		T1 *T1
   115  	}
   116  
   117  	const nRows = 1
   118  
   119  	row := T{
   120  		T1: &T1{
   121  			TA: &T2{
   122  				X: "abc",
   123  			},
   124  		},
   125  	}
   126  
   127  	rows := make([]T, nRows)
   128  	for i := range rows {
   129  		rows[i] = row
   130  	}
   131  
   132  	b := new(bytes.Buffer)
   133  	w := parquet.NewGenericWriter[T](b)
   134  
   135  	if _, err := w.Write(rows); err != nil {
   136  		t.Fatal(err)
   137  	}
   138  	if err := w.Close(); err != nil {
   139  		t.Fatal(err)
   140  	}
   141  
   142  	f := bytes.NewReader(b.Bytes())
   143  	r := parquet.NewGenericReader[T](f)
   144  
   145  	parquetRows := make([]parquet.Row, nRows)
   146  	n, err := r.ReadRows(parquetRows)
   147  	if err != nil && err != io.EOF {
   148  		t.Fatal(err)
   149  	}
   150  	if n != nRows {
   151  		t.Fatalf("wrong number of rows read: want=%d got=%d", nRows, n)
   152  	}
   153  	for _, r := range parquetRows {
   154  		if d := r[0].DefinitionLevel(); d != 3 {
   155  			t.Errorf("wrong definition level for column 0: %d", d)
   156  		}
   157  		if d := r[1].DefinitionLevel(); d != 1 {
   158  			t.Errorf("wrong definition level for column 1: %d", d)
   159  		}
   160  	}
   161  }
   162  
   163  func TestIssue279(t *testing.T) {
   164  	type T2 struct {
   165  		Id   int    `parquet:",plain,optional"`
   166  		Name string `parquet:",plain,optional"`
   167  	}
   168  
   169  	type T1 struct {
   170  		TA []*T2
   171  	}
   172  
   173  	type T struct {
   174  		T1 *T1
   175  	}
   176  
   177  	const nRows = 1
   178  
   179  	row := T{
   180  		T1: &T1{
   181  			TA: []*T2{
   182  				{
   183  					Id:   43,
   184  					Name: "john",
   185  				},
   186  			},
   187  		},
   188  	}
   189  
   190  	rows := make([]T, nRows)
   191  	for i := range rows {
   192  		rows[i] = row
   193  	}
   194  
   195  	b := new(bytes.Buffer)
   196  	w := parquet.NewGenericWriter[T](b)
   197  
   198  	if _, err := w.Write(rows); err != nil {
   199  		t.Fatal(err)
   200  	}
   201  	if err := w.Close(); err != nil {
   202  		t.Fatal(err)
   203  	}
   204  
   205  	f := bytes.NewReader(b.Bytes())
   206  	r := parquet.NewGenericReader[T](f)
   207  
   208  	parquetRows := make([]parquet.Row, nRows)
   209  	n, err := r.ReadRows(parquetRows)
   210  	if err != nil && err != io.EOF {
   211  		t.Fatal(err)
   212  	}
   213  	if n != nRows {
   214  		t.Fatalf("wrong number of rows read: want=%d got=%d", nRows, n)
   215  	}
   216  	for _, r := range parquetRows {
   217  		if d := r[0].DefinitionLevel(); d != 3 {
   218  			t.Errorf("wrong definition level for column 0: %d", d)
   219  		}
   220  		if d := r[1].DefinitionLevel(); d != 3 {
   221  			t.Errorf("wrong definition level for column 1: %d", d)
   222  		}
   223  	}
   224  }
   225  
   226  func TestIssue302(t *testing.T) {
   227  	tests := []struct {
   228  		name string
   229  		fn   func(t *testing.T)
   230  	}{
   231  		{
   232  			name: "SimpleMap",
   233  			fn: func(t *testing.T) {
   234  				type M map[string]int
   235  
   236  				type T struct {
   237  					M M `parquet:","`
   238  				}
   239  
   240  				b := new(bytes.Buffer)
   241  				_ = parquet.NewGenericWriter[T](b)
   242  
   243  			},
   244  		},
   245  
   246  		{
   247  			name: "MapWithValueTag",
   248  			fn: func(t *testing.T) {
   249  				type M map[string]int
   250  
   251  				type T struct {
   252  					M M `parquet:"," parquet-value:",zstd"`
   253  				}
   254  
   255  				b := new(bytes.Buffer)
   256  				_ = parquet.NewGenericWriter[T](b)
   257  
   258  			},
   259  		},
   260  
   261  		{
   262  			name: "MapWithOptionalTag",
   263  			fn: func(t *testing.T) {
   264  				type M map[string]int
   265  
   266  				type T struct {
   267  					M M `parquet:",optional"`
   268  				}
   269  
   270  				b := new(bytes.Buffer)
   271  				w := parquet.NewGenericWriter[T](b)
   272  				expect := []T{
   273  					{
   274  						M: M{
   275  							"Holden": 1,
   276  							"Naomi":  2,
   277  						},
   278  					},
   279  					{
   280  						M: nil,
   281  					},
   282  					{
   283  						M: M{
   284  							"Naomi":  1,
   285  							"Holden": 2,
   286  						},
   287  					},
   288  				}
   289  				_, err := w.Write(expect)
   290  				if err != nil {
   291  					t.Fatal(err)
   292  				}
   293  				if err = w.Close(); err != nil {
   294  					t.Fatal(err)
   295  				}
   296  
   297  				bufReader := bytes.NewReader(b.Bytes())
   298  				r := parquet.NewGenericReader[T](bufReader)
   299  				values := make([]T, 3)
   300  				_, err = r.Read(values)
   301  				if !reflect.DeepEqual(expect, values) {
   302  					t.Fatalf("values do not match.\n\texpect: %v\n\tactual: %v", expect, values)
   303  				}
   304  			},
   305  		},
   306  	}
   307  
   308  	for _, test := range tests {
   309  		t.Run(test.name, test.fn)
   310  	}
   311  }
   312  
   313  func TestIssue347Writer(t *testing.T) {
   314  	type TestType struct {
   315  		Key int
   316  	}
   317  
   318  	b := new(bytes.Buffer)
   319  	// instantiating with concrete type shouldn't panic
   320  	_ = parquet.NewGenericWriter[TestType](b)
   321  
   322  	// instantiating with schema and interface type parameter shouldn't panic
   323  	schema := parquet.SchemaOf(TestType{})
   324  	_ = parquet.NewGenericWriter[any](b, schema)
   325  
   326  	defer func() {
   327  		if r := recover(); r == nil {
   328  			t.Errorf("instantiating generic buffer without schema and with interface " +
   329  				"type parameter should panic")
   330  		}
   331  	}()
   332  	_ = parquet.NewGenericWriter[any](b)
   333  }
   334  
   335  func TestIssue375(t *testing.T) {
   336  	type Row struct{ FirstName, LastName string }
   337  
   338  	output := new(bytes.Buffer)
   339  	writer := parquet.NewGenericWriter[Row](output, parquet.MaxRowsPerRowGroup(10))
   340  
   341  	rows := make([]Row, 100)
   342  	for i := range rows {
   343  		rows[i] = Row{
   344  			FirstName: "0123456789"[i%10 : i%10+1],
   345  			LastName:  "foo",
   346  		}
   347  	}
   348  
   349  	n, err := writer.Write(rows)
   350  	if err != nil {
   351  		t.Fatal(err)
   352  	}
   353  	if n != len(rows) {
   354  		t.Fatal("wrong number of rows written:", n)
   355  	}
   356  
   357  	if err := writer.Close(); err != nil {
   358  		t.Fatal(err)
   359  	}
   360  
   361  	f, err := parquet.OpenFile(bytes.NewReader(output.Bytes()), int64(output.Len()))
   362  	if err != nil {
   363  		t.Fatal(err)
   364  	}
   365  
   366  	rowGroups := f.RowGroups()
   367  	if len(rowGroups) != 10 {
   368  		t.Errorf("wrong number of row groups in parquet file: want=10 got=%d", len(rowGroups))
   369  	}
   370  }
   371  
   372  func TestGenericSetKeyValueMetadata(t *testing.T) {
   373  	testKey := "test-key"
   374  	testValue := "test-value"
   375  
   376  	type Row struct{ FirstName, LastName string }
   377  
   378  	output := new(bytes.Buffer)
   379  	writer := parquet.NewGenericWriter[Row](output, parquet.MaxRowsPerRowGroup(10))
   380  
   381  	rows := []Row{
   382  		{FirstName: "First", LastName: "Last"},
   383  	}
   384  
   385  	_, err := writer.Write(rows)
   386  	if err != nil {
   387  		t.Fatal(err)
   388  	}
   389  
   390  	writer.SetKeyValueMetadata(testKey, testValue)
   391  
   392  	err = writer.Close()
   393  	if err != nil {
   394  		t.Fatal(err)
   395  	}
   396  
   397  	f, err := parquet.OpenFile(bytes.NewReader(output.Bytes()), int64(output.Len()))
   398  	if err != nil {
   399  		t.Fatal(err)
   400  	}
   401  
   402  	value, ok := f.Lookup(testKey)
   403  	if !ok {
   404  		t.Fatalf("key/value metadata should have included %q", testKey)
   405  	}
   406  	if value != testValue {
   407  		t.Errorf("expected %q, got %q", testValue, value)
   408  	}
   409  }
   410  
   411  func scanParquetFile(f *os.File) error {
   412  	s, err := f.Stat()
   413  	if err != nil {
   414  		return err
   415  	}
   416  
   417  	p, err := parquet.OpenFile(f, s.Size())
   418  	if err != nil {
   419  		return err
   420  	}
   421  
   422  	return scanParquetValues(p.Root())
   423  }
   424  
   425  func scanParquetValues(col *parquet.Column) error {
   426  	return forEachColumnValue(col, func(leaf *parquet.Column, value parquet.Value) error {
   427  		fmt.Printf("%s > %+v\n", strings.Join(leaf.Path(), "."), value)
   428  		return nil
   429  	})
   430  }
   431  
   432  func generateParquetFile(rows rows, options ...parquet.WriterOption) (string, []byte, error) {
   433  	tmp, err := os.CreateTemp("/tmp", "*.parquet")
   434  	if err != nil {
   435  		return "", nil, err
   436  	}
   437  	defer tmp.Close()
   438  	path := tmp.Name()
   439  	defer os.Remove(path)
   440  
   441  	writerOptions := []parquet.WriterOption{
   442  		parquet.PageBufferSize(20),
   443  		parquet.DataPageStatistics(true),
   444  	}
   445  	writerOptions = append(writerOptions, options...)
   446  
   447  	if err := writeParquetFile(tmp, rows, writerOptions...); err != nil {
   448  		return "", nil, err
   449  	}
   450  
   451  	if err := scanParquetFile(tmp); err != nil {
   452  		return "", nil, err
   453  	}
   454  
   455  	var outputParts [][]byte
   456  	// Ideally, we could add the "cat" command here and validate each row in the parquet
   457  	// file using the parquet CLI tool. However, it seems to have a number of bugs related
   458  	// to reading repeated fields, so we cannot reliably do this validation for now.
   459  	// See https://issues.apache.org/jira/projects/PARQUET/issues/PARQUET-2181 and others.
   460  	for _, cmd := range []string{"meta", "pages"} {
   461  		out, err := parquetCLI(cmd, path)
   462  		if err != nil {
   463  			return "", nil, err
   464  		}
   465  		outputParts = append(outputParts, out)
   466  	}
   467  	return path, bytes.Join(outputParts, []byte("")), nil
   468  }
   469  
   470  type firstAndLastName struct {
   471  	FirstName string `parquet:"first_name,dict,zstd"`
   472  	LastName  string `parquet:"last_name,delta,zstd"`
   473  }
   474  
   475  type timeseries struct {
   476  	Name      string  `parquet:"name,dict"`
   477  	Timestamp int64   `parquet:"timestamp,delta"`
   478  	Value     float64 `parquet:"value"`
   479  }
   480  
   481  type event struct {
   482  	Name     string  `parquet:"name,dict"`
   483  	Type     string  `parquet:"-"`
   484  	Value    float64 `parquet:"value"`
   485  	Category string  `parquet:"-"`
   486  }
   487  
   488  var writerTests = []struct {
   489  	scenario string
   490  	version  int
   491  	codec    compress.Codec
   492  	rows     []interface{}
   493  	dump     string
   494  }{
   495  	{
   496  		scenario: "page v1 with dictionary encoding",
   497  		version:  v1,
   498  		rows: []interface{}{
   499  			&firstAndLastName{FirstName: "Han", LastName: "Solo"},
   500  			&firstAndLastName{FirstName: "Leia", LastName: "Skywalker"},
   501  			&firstAndLastName{FirstName: "Luke", LastName: "Skywalker"},
   502  		},
   503  		dump: `
   504  File path:  {file-path}
   505  Created by: github.com/parquet-go/parquet-go
   506  Properties: (none)
   507  Schema:
   508  message firstAndLastName {
   509    required binary first_name (STRING);
   510    required binary last_name (STRING);
   511  }
   512  
   513  
   514  Row group 0:  count: 3  107.67 B records  start: 4  total(compressed): 323 B total(uncompressed):299 B
   515  --------------------------------------------------------------------------------
   516              type      encodings count     avg size   nulls   min / max
   517  first_name  BINARY    Z _ R     3         38.00 B            "Han" / "Luke"
   518  last_name   BINARY    Z   D     3         69.67 B            "Skywalker" / "Solo"
   519  
   520  
   521  Column: first_name
   522  --------------------------------------------------------------------------------
   523    page   type  enc  count   avg size   size       rows     nulls   min / max
   524    0-D    dict  Z _  3       7.67 B     23 B
   525    0-1    data  Z R  3       2.33 B     7 B                         "Han" / "Luke"
   526  
   527  
   528  Column: last_name
   529  --------------------------------------------------------------------------------
   530    page   type  enc  count   avg size   size       rows     nulls   min / max
   531    0-0    data  Z D  2       28.00 B    56 B                        "Skywalker" / "Solo"
   532    0-1    data  Z D  1       19.00 B    19 B                        "Skywalker" / "Skywalker"
   533  
   534  `,
   535  	},
   536  
   537  	{ // same as the previous test but uses page v2 where data pages aren't compressed
   538  		scenario: "page v2 with dictionary encoding",
   539  		version:  v2,
   540  		rows: []interface{}{
   541  			&firstAndLastName{FirstName: "Han", LastName: "Solo"},
   542  			&firstAndLastName{FirstName: "Leia", LastName: "Skywalker"},
   543  			&firstAndLastName{FirstName: "Luke", LastName: "Skywalker"},
   544  		},
   545  		dump: `
   546  File path:  {file-path}
   547  Created by: github.com/parquet-go/parquet-go
   548  Properties: (none)
   549  Schema:
   550  message firstAndLastName {
   551    required binary first_name (STRING);
   552    required binary last_name (STRING);
   553  }
   554  
   555  
   556  Row group 0:  count: 3  109.67 B records  start: 4  total(compressed): 329 B total(uncompressed):314 B
   557  --------------------------------------------------------------------------------
   558              type      encodings count     avg size   nulls   min / max
   559  first_name  BINARY    Z _ R     3         36.67 B            "Han" / "Luke"
   560  last_name   BINARY    Z   D     3         73.00 B            "Skywalker" / "Solo"
   561  
   562  
   563  Column: first_name
   564  --------------------------------------------------------------------------------
   565    page   type  enc  count   avg size   size       rows     nulls   min / max
   566    0-D    dict  Z _  3       7.67 B     23 B
   567    0-1    data  _ R  3       2.33 B     7 B        3        0       "Han" / "Luke"
   568  
   569  
   570  Column: last_name
   571  --------------------------------------------------------------------------------
   572    page   type  enc  count   avg size   size       rows     nulls   min / max
   573    0-0    data  _ D  2       28.00 B    56 B       2        0       "Skywalker" / "Solo"
   574    0-1    data  _ D  1       19.00 B    19 B       1        0       "Skywalker" / "Skywalker"
   575  
   576  `,
   577  	},
   578  
   579  	{
   580  		scenario: "timeseries with delta encoding",
   581  		version:  v2,
   582  		codec:    &parquet.Gzip,
   583  		rows: []interface{}{
   584  			timeseries{Name: "http_request_total", Timestamp: 1639444033, Value: 100},
   585  			timeseries{Name: "http_request_total", Timestamp: 1639444058, Value: 0},
   586  			timeseries{Name: "http_request_total", Timestamp: 1639444085, Value: 42},
   587  			timeseries{Name: "http_request_total", Timestamp: 1639444093, Value: 1},
   588  			timeseries{Name: "http_request_total", Timestamp: 1639444101, Value: 2},
   589  			timeseries{Name: "http_request_total", Timestamp: 1639444108, Value: 5},
   590  			timeseries{Name: "http_request_total", Timestamp: 1639444133, Value: 4},
   591  			timeseries{Name: "http_request_total", Timestamp: 1639444137, Value: 5},
   592  			timeseries{Name: "http_request_total", Timestamp: 1639444141, Value: 6},
   593  			timeseries{Name: "http_request_total", Timestamp: 1639444144, Value: 10},
   594  		},
   595  		dump: `
   596  File path:  {file-path}
   597  Created by: github.com/parquet-go/parquet-go
   598  Properties: (none)
   599  Schema:
   600  message timeseries {
   601    required binary name (STRING);
   602    required int64 timestamp (INTEGER(64,true));
   603    required double value;
   604  }
   605  
   606  
   607  Row group 0:  count: 10  121.70 B records  start: 4  total(compressed): 1.188 kB total(uncompressed):1.312 kB
   608  --------------------------------------------------------------------------------
   609             type      encodings count     avg size   nulls   min / max
   610  name       BINARY    G _ R     10        29.00 B            "http_request_total" / "http_request_total"
   611  timestamp  INT64     G   D     10        46.70 B            "1639444033" / "1639444144"
   612  value      DOUBLE    G   _     10        46.00 B            "-0.0" / "100.0"
   613  
   614  
   615  Column: name
   616  --------------------------------------------------------------------------------
   617    page   type  enc  count   avg size   size       rows     nulls   min / max
   618    0-D    dict  G _  1       22.00 B    22 B
   619    0-1    data  _ R  5       0.40 B     2 B        5        0       "http_request_total" / "http_request_total"
   620    0-2    data  _ R  5       0.40 B     2 B        5        0       "http_request_total" / "http_request_total"
   621  
   622  
   623  Column: timestamp
   624  --------------------------------------------------------------------------------
   625    page   type  enc  count   avg size   size       rows     nulls   min / max
   626    0-0    data  _ D  3       47.33 B    142 B      3        0       "1639444033" / "1639444085"
   627    0-1    data  _ D  3       47.33 B    142 B      3        0       "1639444093" / "1639444108"
   628    0-2    data  _ D  3       47.33 B    142 B      3        0       "1639444133" / "1639444141"
   629    0-3    data  _ D  1       9.00 B     9 B        1        0       "1639444144" / "1639444144"
   630  
   631  
   632  Column: value
   633  --------------------------------------------------------------------------------
   634    page   type  enc  count   avg size   size       rows     nulls   min / max
   635    0-0    data  _ _  3       8.00 B     24 B       3        0       "-0.0" / "100.0"
   636    0-1    data  _ _  3       8.00 B     24 B       3        0       "1.0" / "5.0"
   637    0-2    data  _ _  3       8.00 B     24 B       3        0       "4.0" / "6.0"
   638    0-3    data  _ _  1       8.00 B     8 B        1        0       "10.0" / "10.0"
   639  
   640  `,
   641  	},
   642  
   643  	{
   644  		scenario: "example from the twitter blog (v1)",
   645  		version:  v1,
   646  		rows: []interface{}{
   647  			AddressBook{
   648  				Owner: "Julien Le Dem",
   649  				OwnerPhoneNumbers: []string{
   650  					"555 123 4567",
   651  					"555 666 1337",
   652  				},
   653  				Contacts: []Contact{
   654  					{
   655  						Name:        "Dmitriy Ryaboy",
   656  						PhoneNumber: "555 987 6543",
   657  					},
   658  					{
   659  						Name: "Chris Aniszczyk",
   660  					},
   661  				},
   662  			},
   663  			AddressBook{
   664  				Owner:             "A. Nonymous",
   665  				OwnerPhoneNumbers: nil,
   666  			},
   667  		},
   668  		dump: `
   669  File path:  {file-path}
   670  Created by: github.com/parquet-go/parquet-go
   671  Properties: (none)
   672  Schema:
   673  message AddressBook {
   674    required binary owner (STRING);
   675    repeated binary ownerPhoneNumbers (STRING);
   676    repeated group contacts {
   677      required binary name (STRING);
   678      optional binary phoneNumber (STRING);
   679    }
   680  }
   681  
   682  
   683  Row group 0:  count: 2  384.00 B records  start: 4  total(compressed): 768 B total(uncompressed):691 B
   684  --------------------------------------------------------------------------------
   685                        type      encodings count     avg size   nulls   min / max
   686  owner                 BINARY    Z         2         70.00 B            "A. Nonymous" / "Julien Le Dem"
   687  ownerPhoneNumbers     BINARY    G         3         80.33 B    1       "555 123 4567" / "555 666 1337"
   688  contacts.name         BINARY    _         3         70.00 B    1       "Chris Aniszczyk" / "Dmitriy Ryaboy"
   689  contacts.phoneNumber  BINARY    Z         3         59.00 B    2       "555 987 6543" / "555 987 6543"
   690  
   691  
   692  Column: owner
   693  --------------------------------------------------------------------------------
   694    page   type  enc  count   avg size   size       rows     nulls   min / max
   695    0-0    data  Z D  2       25.00 B    50 B                        "A. Nonymous" / "Julien Le Dem"
   696  
   697  
   698  Column: ownerPhoneNumbers
   699  --------------------------------------------------------------------------------
   700    page   type  enc  count   avg size   size       rows     nulls   min / max
   701    0-0    data  Z D  2       32.00 B    64 B                        "555 123 4567" / "555 666 1337"
   702    0-1    data  Z D  1       17.00 B    17 B                1
   703  
   704  
   705  Column: contacts.name
   706  --------------------------------------------------------------------------------
   707    page   type  enc  count   avg size   size       rows     nulls   min / max
   708    0-0    data  Z D  2       36.50 B    73 B                        "Chris Aniszczyk" / "Dmitriy Ryaboy"
   709    0-1    data  Z D  1       17.00 B    17 B                1
   710  
   711  
   712  Column: contacts.phoneNumber
   713  --------------------------------------------------------------------------------
   714    page   type  enc  count   avg size   size       rows     nulls   min / max
   715    0-0    data  Z D  2       16.50 B    33 B                1       "555 987 6543" / "555 987 6543"
   716    0-1    data  Z D  1       17.00 B    17 B                1
   717  
   718  `,
   719  	},
   720  
   721  	{
   722  		scenario: "example from the twitter blog (v2)",
   723  		version:  v2,
   724  		rows: []interface{}{
   725  			AddressBook{
   726  				Owner: "Julien Le Dem",
   727  				OwnerPhoneNumbers: []string{
   728  					"555 123 4567",
   729  					"555 666 1337",
   730  				},
   731  				Contacts: []Contact{
   732  					{
   733  						Name:        "Dmitriy Ryaboy",
   734  						PhoneNumber: "555 987 6543",
   735  					},
   736  					{
   737  						Name: "Chris Aniszczyk",
   738  					},
   739  				},
   740  			},
   741  			AddressBook{
   742  				Owner:             "A. Nonymous",
   743  				OwnerPhoneNumbers: nil,
   744  			},
   745  		},
   746  
   747  		dump: `
   748  File path:  {file-path}
   749  Created by: github.com/parquet-go/parquet-go
   750  Properties: (none)
   751  Schema:
   752  message AddressBook {
   753    required binary owner (STRING);
   754    repeated binary ownerPhoneNumbers (STRING);
   755    repeated group contacts {
   756      required binary name (STRING);
   757      optional binary phoneNumber (STRING);
   758    }
   759  }
   760  
   761  
   762  Row group 0:  count: 2  377.50 B records  start: 4  total(compressed): 755 B total(uncompressed):678 B
   763  --------------------------------------------------------------------------------
   764                        type      encodings count     avg size   nulls   min / max
   765  owner                 BINARY    Z         2         72.50 B            "A. Nonymous" / "Julien Le Dem"
   766  ownerPhoneNumbers     BINARY    G         3         78.00 B    1       "555 123 4567" / "555 666 1337"
   767  contacts.name         BINARY    _         3         68.00 B    1       "Chris Aniszczyk" / "Dmitriy Ryaboy"
   768  contacts.phoneNumber  BINARY    Z         3         57.33 B    2       "555 987 6543" / "555 987 6543"
   769  
   770  
   771  Column: owner
   772  --------------------------------------------------------------------------------
   773    page   type  enc  count   avg size   size       rows     nulls   min / max
   774    0-0    data  _ D  2       25.00 B    50 B       2        0       "A. Nonymous" / "Julien Le Dem"
   775  
   776  
   777  Column: ownerPhoneNumbers
   778  --------------------------------------------------------------------------------
   779    page   type  enc  count   avg size   size       rows     nulls   min / max
   780    0-0    data  _ D  2       28.00 B    56 B       1        0       "555 123 4567" / "555 666 1337"
   781    0-1    data  _ D  1       9.00 B     9 B        1        1
   782  
   783  
   784  Column: contacts.name
   785  --------------------------------------------------------------------------------
   786    page   type  enc  count   avg size   size       rows     nulls   min / max
   787    0-0    data  _ D  2       32.50 B    65 B       1        0       "Chris Aniszczyk" / "Dmitriy Ryaboy"
   788    0-1    data  _ D  1       9.00 B     9 B        1        1
   789  
   790  
   791  Column: contacts.phoneNumber
   792  --------------------------------------------------------------------------------
   793    page   type  enc  count   avg size   size       rows     nulls   min / max
   794    0-0    data  _ D  2       12.50 B    25 B       1        1       "555 987 6543" / "555 987 6543"
   795    0-1    data  _ D  1       9.00 B     9 B        1        1
   796  
   797  `,
   798  	},
   799  	{
   800  		scenario: "omit `-` fields",
   801  		version:  v1,
   802  		rows: []interface{}{
   803  			&event{Name: "customer1", Type: "request", Value: 42.0},
   804  			&event{Name: "customer2", Type: "access", Value: 1.0},
   805  		},
   806  		dump: `
   807  File path:  {file-path}
   808  Created by: github.com/parquet-go/parquet-go
   809  Properties: (none)
   810  Schema:
   811  message event {
   812    required binary name (STRING);
   813    required double value;
   814  }
   815  
   816  
   817  Row group 0:  count: 2  100.00 B records  start: 4  total(compressed): 200 B total(uncompressed):200 B
   818  --------------------------------------------------------------------------------
   819         type      encodings count     avg size   nulls   min / max
   820  name   BINARY    _ _ R     2         59.50 B            "customer1" / "customer2"
   821  value  DOUBLE    _   _     2         40.50 B            "1.0" / "42.0"
   822  
   823  
   824  Column: name
   825  --------------------------------------------------------------------------------
   826    page   type  enc  count   avg size   size       rows     nulls   min / max
   827    0-D    dict  _ _  2       13.00 B    26 B
   828    0-1    data  _ R  2       2.50 B     5 B                         "customer1" / "customer2"
   829  
   830  
   831  Column: value
   832  --------------------------------------------------------------------------------
   833    page   type  enc  count   avg size   size       rows     nulls   min / max
   834    0-0    data  _ _  2       8.00 B     16 B                        "1.0" / "42.0"
   835  
   836  `,
   837  	},
   838  }
   839  
   840  // TestWriter uses the Apache parquet-cli tool to validate generated parquet files.
   841  // On MacOS systems using brew, this can be installed with `brew install parquet-cli`.
   842  // For more information on installing and running this tool, see:
   843  // https://github.com/apache/parquet-mr/blob/ef9929c130f8f2e24fca1c7b42b0742a4d9d5e61/parquet-cli/README.md
   844  // This test expects the parquet-cli command to exist in the environment path as `parquet`
   845  // and to require no additional arguments before the primary command. If you need to run
   846  // it in some other way on your system, you can configure the environment variable
   847  // `PARQUET_GO_TEST_CLI`.
   848  func TestWriter(t *testing.T) {
   849  	if !hasParquetCli() {
   850  		t.Skip("Skipping TestWriter writerTests because parquet-cli is not installed in Github CI. FIXME.") // TODO
   851  	}
   852  
   853  	for _, test := range writerTests {
   854  		dataPageVersion := test.version
   855  		codec := test.codec
   856  		rows := test.rows
   857  		dump := test.dump
   858  
   859  		t.Run(test.scenario, func(t *testing.T) {
   860  			t.Parallel()
   861  
   862  			path, b, err := generateParquetFile(makeRows(rows),
   863  				parquet.DataPageVersion(dataPageVersion),
   864  				parquet.Compression(codec),
   865  			)
   866  			if err != nil {
   867  				t.Logf("\n%s", string(b))
   868  				t.Fatal(err)
   869  			}
   870  
   871  			// The CLI output includes the file-path of the parquet file. Because the test
   872  			// uses a temp file, this value is not consistent between test runs and cannot
   873  			// be hard-coded. Therefore, the expected value includes a placeholder value
   874  			// and we replace it here.
   875  			dump = strings.Replace(dump, "{file-path}", path, 1)
   876  			if string(b) != dump {
   877  				edits := myers.ComputeEdits(span.URIFromPath("want.txt"), dump, string(b))
   878  				diff := fmt.Sprint(gotextdiff.ToUnified("want.txt", "got.txt", dump, edits))
   879  				t.Errorf("\n%s", diff)
   880  			}
   881  		})
   882  	}
   883  }
   884  
   885  func hasParquetCli() bool {
   886  	// If PARQUET_GO_TEST_CLI is defined, always attempt to run the test. If it's defined
   887  	// but the command cannot be called, the test itself should fail.
   888  	if os.Getenv("PARQUET_GO_TEST_CLI") != "" {
   889  		return true
   890  	}
   891  	_, err := exec.LookPath("parquet")
   892  	return err == nil
   893  }
   894  
   895  func parquetCLI(cmd, path string) ([]byte, error) {
   896  	execPath := "parquet"
   897  	envCmd := os.Getenv("PARQUET_GO_TEST_CLI")
   898  	var cmdArgs []string
   899  	if envCmd != "" {
   900  		envSplit := strings.Split(envCmd, " ")
   901  		execPath = envSplit[0]
   902  		cmdArgs = envSplit[1:]
   903  	}
   904  	cmdArgs = append(cmdArgs, cmd, path)
   905  	p := exec.Command(execPath, cmdArgs...)
   906  
   907  	output, err := p.CombinedOutput()
   908  	if err != nil {
   909  		return output, err
   910  	}
   911  
   912  	// parquet-cli has trailing spaces on some lines.
   913  	lines := bytes.Split(output, []byte("\n"))
   914  
   915  	for i, line := range lines {
   916  		lines[i] = bytes.TrimRight(line, " ")
   917  	}
   918  
   919  	return bytes.Join(lines, []byte("\n")), nil
   920  }
   921  
   922  func TestWriterGenerateBloomFilters(t *testing.T) {
   923  	type Person struct {
   924  		FirstName utf8string `parquet:"first_name"`
   925  		LastName  utf8string `parquet:"last_name"`
   926  	}
   927  
   928  	err := quickCheck(func(rows []Person) bool {
   929  		if len(rows) == 0 { // TODO: support writing files with no rows
   930  			return true
   931  		}
   932  
   933  		buffer := new(bytes.Buffer)
   934  		writer := parquet.NewWriter(buffer,
   935  			parquet.BloomFilters(
   936  				parquet.SplitBlockFilter(10, "last_name"),
   937  			),
   938  		)
   939  		for i := range rows {
   940  			if err := writer.Write(&rows[i]); err != nil {
   941  				t.Error(err)
   942  				return false
   943  			}
   944  		}
   945  		if err := writer.Close(); err != nil {
   946  			t.Error(err)
   947  			return false
   948  		}
   949  
   950  		reader := bytes.NewReader(buffer.Bytes())
   951  		f, err := parquet.OpenFile(reader, reader.Size())
   952  		if err != nil {
   953  			t.Error(err)
   954  			return false
   955  		}
   956  		rowGroup := f.RowGroups()[0]
   957  		columns := rowGroup.ColumnChunks()
   958  		firstName := columns[0]
   959  		lastName := columns[1]
   960  
   961  		if firstName.BloomFilter() != nil {
   962  			t.Errorf(`"first_name" column has a bloom filter even though none were configured`)
   963  			return false
   964  		}
   965  
   966  		bloomFilter := lastName.BloomFilter()
   967  		if bloomFilter == nil {
   968  			t.Error(`"last_name" column has no bloom filter despite being configured to have one`)
   969  			return false
   970  		}
   971  
   972  		for i, row := range rows {
   973  			if ok, err := bloomFilter.Check(parquet.ValueOf(row.LastName)); err != nil {
   974  				t.Errorf("unexpected error checking bloom filter: %v", err)
   975  				return false
   976  			} else if !ok {
   977  				t.Errorf("bloom filter does not contain value %q of row %d", row.LastName, i)
   978  				return false
   979  			}
   980  		}
   981  
   982  		return true
   983  	})
   984  	if err != nil {
   985  		t.Error(err)
   986  	}
   987  }
   988  
   989  func TestBloomFilterForDict(t *testing.T) {
   990  	type testStruct struct {
   991  		A string `parquet:"a,dict"`
   992  	}
   993  
   994  	schema := parquet.SchemaOf(&testStruct{})
   995  
   996  	b := bytes.NewBuffer(nil)
   997  	w := parquet.NewWriter(
   998  		b,
   999  		schema,
  1000  		parquet.BloomFilters(parquet.SplitBlockFilter(10, "a")),
  1001  	)
  1002  
  1003  	err := w.Write(&testStruct{A: "test"})
  1004  	if err != nil {
  1005  		t.Fatal(err)
  1006  	}
  1007  
  1008  	err = w.Close()
  1009  	if err != nil {
  1010  		t.Fatal(err)
  1011  	}
  1012  
  1013  	f, err := parquet.OpenFile(bytes.NewReader(b.Bytes()), int64(b.Len()))
  1014  	if err != nil {
  1015  		t.Fatal(err)
  1016  	}
  1017  
  1018  	ok, err := f.RowGroups()[0].ColumnChunks()[0].BloomFilter().Check(parquet.ValueOf("test"))
  1019  	if err != nil {
  1020  		t.Fatal(err)
  1021  	}
  1022  	if !ok {
  1023  		t.Error("bloom filter should have contained 'test'")
  1024  	}
  1025  }
  1026  
  1027  func TestWriterRepeatedUUIDDict(t *testing.T) {
  1028  	inputID := uuid.MustParse("123456ab-0000-0000-0000-000000000000")
  1029  	records := []struct {
  1030  		List []uuid.UUID `parquet:"list,dict"`
  1031  	}{{
  1032  		[]uuid.UUID{inputID},
  1033  	}}
  1034  	schema := parquet.SchemaOf(&records[0])
  1035  	b := bytes.NewBuffer(nil)
  1036  	w := parquet.NewWriter(b, schema)
  1037  	if err := w.Write(records[0]); err != nil {
  1038  		t.Fatal(err)
  1039  	}
  1040  	if err := w.Close(); err != nil {
  1041  		t.Fatal(err)
  1042  	}
  1043  
  1044  	f, err := parquet.OpenFile(bytes.NewReader(b.Bytes()), int64(b.Len()))
  1045  	if err != nil {
  1046  		t.Fatal(err)
  1047  	}
  1048  
  1049  	rowbuf := make([]parquet.Row, 1)
  1050  	rows := f.RowGroups()[0].Rows()
  1051  	defer rows.Close()
  1052  	n, err := rows.ReadRows(rowbuf)
  1053  	if n == 0 {
  1054  		t.Fatalf("reading row from parquet file: %v", err)
  1055  	}
  1056  	if len(rowbuf[0]) != 1 {
  1057  		t.Errorf("expected 1 value in row, got %d", len(rowbuf[0]))
  1058  	}
  1059  	if !bytes.Equal(inputID[:], rowbuf[0][0].Bytes()) {
  1060  		t.Errorf("expected to get UUID %q back out, got %q", inputID, rowbuf[0][0].Bytes())
  1061  	}
  1062  }
  1063  
  1064  func TestWriterResetWithBloomFilters(t *testing.T) {
  1065  	type Test struct {
  1066  		Value string `parquet:"value,dict"`
  1067  	}
  1068  
  1069  	writer := parquet.NewWriter(new(bytes.Buffer),
  1070  		parquet.BloomFilters(
  1071  			parquet.SplitBlockFilter(10, "value"),
  1072  		),
  1073  	)
  1074  
  1075  	if err := writer.Write(&Test{Value: "foo"}); err != nil {
  1076  		t.Fatal(err)
  1077  	}
  1078  
  1079  	if err := writer.Close(); err != nil {
  1080  		t.Fatal(err)
  1081  	}
  1082  
  1083  	writer.Reset(new(bytes.Buffer))
  1084  
  1085  	if err := writer.Write(&Test{Value: "bar"}); err != nil {
  1086  		t.Fatal(err)
  1087  	}
  1088  
  1089  	if err := writer.Close(); err != nil {
  1090  		t.Fatal(err)
  1091  	}
  1092  }
  1093  
  1094  func TestWriterMaxRowsPerRowGroup(t *testing.T) {
  1095  	output := new(bytes.Buffer)
  1096  	writer := parquet.NewWriter(output, parquet.MaxRowsPerRowGroup(10))
  1097  
  1098  	for i := 0; i < 100; i++ {
  1099  		err := writer.Write(struct{ FirstName, LastName string }{
  1100  			FirstName: "0123456789"[i%10 : i%10+1],
  1101  			LastName:  "foo",
  1102  		})
  1103  		if err != nil {
  1104  			t.Fatal(err)
  1105  		}
  1106  	}
  1107  
  1108  	if err := writer.Close(); err != nil {
  1109  		t.Fatal(err)
  1110  	}
  1111  
  1112  	f, err := parquet.OpenFile(bytes.NewReader(output.Bytes()), int64(output.Len()))
  1113  	if err != nil {
  1114  		t.Fatal(err)
  1115  	}
  1116  
  1117  	rowGroups := f.RowGroups()
  1118  	if len(rowGroups) != 10 {
  1119  		t.Errorf("wrong number of row groups in parquet file: want=10 got=%d", len(rowGroups))
  1120  	}
  1121  }
  1122  
  1123  func TestSetKeyValueMetadata(t *testing.T) {
  1124  	testKey := "test-key"
  1125  	testValue := "test-value"
  1126  
  1127  	type testStruct struct {
  1128  		A string `parquet:"a,dict"`
  1129  	}
  1130  
  1131  	schema := parquet.SchemaOf(&testStruct{})
  1132  
  1133  	b := bytes.NewBuffer(nil)
  1134  	w := parquet.NewWriter(
  1135  		b,
  1136  		schema,
  1137  	)
  1138  
  1139  	err := w.Write(&testStruct{A: "test"})
  1140  	if err != nil {
  1141  		t.Fatal(err)
  1142  	}
  1143  
  1144  	w.SetKeyValueMetadata(testKey, testValue)
  1145  
  1146  	err = w.Close()
  1147  	if err != nil {
  1148  		t.Fatal(err)
  1149  	}
  1150  
  1151  	f, err := parquet.OpenFile(bytes.NewReader(b.Bytes()), int64(b.Len()))
  1152  	if err != nil {
  1153  		t.Fatal(err)
  1154  	}
  1155  
  1156  	value, ok := f.Lookup(testKey)
  1157  	if !ok {
  1158  		t.Fatalf("key/value metadata should have included %q", testKey)
  1159  	}
  1160  	if value != testValue {
  1161  		t.Errorf("expected %q, got %q", testValue, value)
  1162  	}
  1163  }
  1164  
  1165  func TestSetKeyValueMetadataOverwritesExisting(t *testing.T) {
  1166  	testKey := "test-key"
  1167  	testValue := "test-value"
  1168  
  1169  	type testStruct struct {
  1170  		A string `parquet:"a,dict"`
  1171  	}
  1172  
  1173  	schema := parquet.SchemaOf(&testStruct{})
  1174  
  1175  	b := bytes.NewBuffer(nil)
  1176  	w := parquet.NewWriter(
  1177  		b,
  1178  		schema,
  1179  		parquet.KeyValueMetadata(testKey, "original-value"),
  1180  	)
  1181  
  1182  	err := w.Write(&testStruct{A: "test"})
  1183  	if err != nil {
  1184  		t.Fatal(err)
  1185  	}
  1186  
  1187  	w.SetKeyValueMetadata(testKey, testValue)
  1188  
  1189  	err = w.Close()
  1190  	if err != nil {
  1191  		t.Fatal(err)
  1192  	}
  1193  
  1194  	f, err := parquet.OpenFile(bytes.NewReader(b.Bytes()), int64(b.Len()))
  1195  	if err != nil {
  1196  		t.Fatal(err)
  1197  	}
  1198  
  1199  	value, ok := f.Lookup(testKey)
  1200  	if !ok {
  1201  		t.Fatalf("key/value metadata should have included %q", testKey)
  1202  	}
  1203  	if value != testValue {
  1204  		t.Errorf("expected %q, got %q", testValue, value)
  1205  	}
  1206  }
  1207  
  1208  func TestColumnMaxValueAndMinValue(t *testing.T) {
  1209  	type testStruct struct {
  1210  		A string `parquet:"a,plain"`
  1211  	}
  1212  
  1213  	tests := make([]testStruct, 100)
  1214  	tests[0] = testStruct{A: ""}
  1215  	for i := 1; i < 100; i++ {
  1216  		tests[i] = testStruct{A: strconv.Itoa(i)}
  1217  	}
  1218  	schema := parquet.SchemaOf(&testStruct{})
  1219  	b := bytes.NewBuffer(nil)
  1220  	config := parquet.DefaultWriterConfig()
  1221  	config.PageBufferSize = 256
  1222  	w := parquet.NewGenericWriter[testStruct](b, schema, config)
  1223  	_, _ = w.Write(tests[0:50])
  1224  	_, _ = w.Write(tests[50:100])
  1225  	_ = w.Close()
  1226  
  1227  	f, err := parquet.OpenFile(bytes.NewReader(b.Bytes()), int64(b.Len()))
  1228  	if err != nil {
  1229  		t.Fatal(err)
  1230  	}
  1231  	if len(f.RowGroups()) != 1 {
  1232  		t.Fatalf("wrong number of row groups in parquet file: want=1 got=%d", f.RowGroups())
  1233  	}
  1234  	statistics := f.Metadata().RowGroups[0].Columns[0].MetaData.Statistics
  1235  	if string(statistics.MinValue) != "" {
  1236  		t.Fatalf("wrong min value of row groups in parquet file: want=' '() got=%s", string(statistics.MinValue))
  1237  	}
  1238  	if string(statistics.MaxValue) != "99" {
  1239  		t.Fatalf("wrong max value of row groups in parquet file: want=99 got=%s", string(statistics.MaxValue))
  1240  	}
  1241  }