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