github.com/apache/arrow/go/v14@v14.0.1/parquet/file/column_writer_test.go (about)

     1  // Licensed to the Apache Software Foundation (ASF) under one
     2  // or more contributor license agreements.  See the NOTICE file
     3  // distributed with this work for additional information
     4  // regarding copyright ownership.  The ASF licenses this file
     5  // to you under the Apache License, Version 2.0 (the
     6  // "License"); you may not use this file except in compliance
     7  // with the License.  You may obtain a copy of the License at
     8  //
     9  // http://www.apache.org/licenses/LICENSE-2.0
    10  //
    11  // Unless required by applicable law or agreed to in writing, software
    12  // distributed under the License is distributed on an "AS IS" BASIS,
    13  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    14  // See the License for the specific language governing permissions and
    15  // limitations under the License.
    16  
    17  package file_test
    18  
    19  import (
    20  	"bytes"
    21  	"math"
    22  	"reflect"
    23  	"runtime"
    24  	"sync"
    25  	"testing"
    26  
    27  	"github.com/apache/arrow/go/v14/arrow/bitutil"
    28  	"github.com/apache/arrow/go/v14/arrow/memory"
    29  	arrutils "github.com/apache/arrow/go/v14/internal/utils"
    30  	"github.com/apache/arrow/go/v14/parquet"
    31  	"github.com/apache/arrow/go/v14/parquet/compress"
    32  	"github.com/apache/arrow/go/v14/parquet/file"
    33  	"github.com/apache/arrow/go/v14/parquet/internal/encoding"
    34  	"github.com/apache/arrow/go/v14/parquet/internal/encryption"
    35  	format "github.com/apache/arrow/go/v14/parquet/internal/gen-go/parquet"
    36  	"github.com/apache/arrow/go/v14/parquet/internal/testutils"
    37  	"github.com/apache/arrow/go/v14/parquet/internal/utils"
    38  	"github.com/apache/arrow/go/v14/parquet/metadata"
    39  	"github.com/apache/arrow/go/v14/parquet/schema"
    40  	"github.com/stretchr/testify/assert"
    41  	"github.com/stretchr/testify/mock"
    42  	"github.com/stretchr/testify/suite"
    43  )
    44  
    45  const (
    46  	SmallSize = 100
    47  	// larger to test some corner cases, only in some specific cases
    48  	LargeSize = 100000
    49  	// very large to test dictionary fallback
    50  	VeryLargeSize = 400000
    51  	// dictionary page size for testing fallback
    52  	DictionaryPageSize = 1024 * 1024
    53  )
    54  
    55  type mockpagewriter struct {
    56  	mock.Mock
    57  }
    58  
    59  func (m *mockpagewriter) Close(hasDict, fallBack bool) error {
    60  	return m.Called(hasDict, fallBack).Error(0)
    61  }
    62  func (m *mockpagewriter) WriteDataPage(page file.DataPage) (int64, error) {
    63  	args := m.Called(page)
    64  	return int64(args.Int(0)), args.Error(1)
    65  }
    66  func (m *mockpagewriter) WriteDictionaryPage(page *file.DictionaryPage) (int64, error) {
    67  	args := m.Called(page)
    68  	return int64(args.Int(0)), args.Error(1)
    69  }
    70  func (m *mockpagewriter) HasCompressor() bool {
    71  	return m.Called().Bool(0)
    72  }
    73  func (m *mockpagewriter) Compress(buf *bytes.Buffer, src []byte) []byte {
    74  	return m.Called(buf, src).Get(0).([]byte)
    75  }
    76  func (m *mockpagewriter) Reset(sink utils.WriterTell, codec compress.Compression, compressionLevel int, metadata *metadata.ColumnChunkMetaDataBuilder, rgOrdinal, columnOrdinal int16, metaEncryptor, dataEncryptor encryption.Encryptor) error {
    77  	return m.Called().Error(0)
    78  }
    79  
    80  func TestWriteDataPageV1NumValues(t *testing.T) {
    81  	sc := schema.NewSchema(schema.MustGroup(schema.NewGroupNode("schema", parquet.Repetitions.Required, schema.FieldList{
    82  		schema.Must(schema.ListOf(
    83  			schema.Must(schema.NewPrimitiveNode("column", parquet.Repetitions.Optional, parquet.Types.Int32, -1, -1)),
    84  			parquet.Repetitions.Optional, -1)),
    85  	}, -1)))
    86  	descr := sc.Column(0)
    87  	props := parquet.NewWriterProperties(
    88  		parquet.WithStats(true),
    89  		parquet.WithVersion(parquet.V1_0),
    90  		parquet.WithDataPageVersion(parquet.DataPageV1),
    91  		parquet.WithDictionaryDefault(false))
    92  
    93  	metadata := metadata.NewColumnChunkMetaDataBuilder(props, descr)
    94  	pager := new(mockpagewriter)
    95  	defer pager.AssertExpectations(t)
    96  	pager.On("HasCompressor").Return(false)
    97  	wr := file.NewColumnChunkWriter(metadata, pager, props).(*file.Int32ColumnChunkWriter)
    98  
    99  	// write a list "[[0, 1], null, [2, null, 3]]"
   100  	// should be 6 values, 2 nulls and 3 rows
   101  	wr.WriteBatch([]int32{0, 1, 2, 3},
   102  		[]int16{3, 3, 0, 3, 2, 3},
   103  		[]int16{0, 1, 0, 0, 1, 1})
   104  
   105  	pager.On("WriteDataPage", mock.MatchedBy(func(page file.DataPage) bool {
   106  		pagev1, ok := page.(*file.DataPageV1)
   107  		if !ok {
   108  			return false
   109  		}
   110  
   111  		encodedStats := pagev1.Statistics()
   112  		// only match if the page being written has 2 nulls, 6 values and 3 rows
   113  		return pagev1.NumValues() == 6 &&
   114  			encodedStats.HasNullCount &&
   115  			encodedStats.NullCount == 2
   116  	})).Return(10, nil)
   117  
   118  	wr.FlushBufferedDataPages()
   119  	assert.EqualValues(t, 3, wr.RowsWritten())
   120  }
   121  
   122  func TestWriteDataPageV2NumRows(t *testing.T) {
   123  	// test issue from PARQUET-2066
   124  	sc := schema.NewSchema(schema.MustGroup(schema.NewGroupNode("schema", parquet.Repetitions.Required, schema.FieldList{
   125  		schema.Must(schema.ListOf(
   126  			schema.Must(schema.NewPrimitiveNode("column", parquet.Repetitions.Optional, parquet.Types.Int32, -1, -1)),
   127  			parquet.Repetitions.Optional, -1)),
   128  	}, -1)))
   129  	descr := sc.Column(0)
   130  	props := parquet.NewWriterProperties(
   131  		parquet.WithStats(true),
   132  		parquet.WithVersion(parquet.V2_LATEST),
   133  		parquet.WithDataPageVersion(parquet.DataPageV2),
   134  		parquet.WithDictionaryDefault(false))
   135  
   136  	metadata := metadata.NewColumnChunkMetaDataBuilder(props, descr)
   137  	pager := new(mockpagewriter)
   138  	defer pager.AssertExpectations(t)
   139  	pager.On("HasCompressor").Return(false)
   140  	wr := file.NewColumnChunkWriter(metadata, pager, props).(*file.Int32ColumnChunkWriter)
   141  
   142  	// write a list "[[0, 1], null, [2, null, 3]]"
   143  	// should be 6 values, 2 nulls and 3 rows
   144  	wr.WriteBatch([]int32{0, 1, 2, 3},
   145  		[]int16{3, 3, 0, 3, 2, 3},
   146  		[]int16{0, 1, 0, 0, 1, 1})
   147  
   148  	pager.On("WriteDataPage", mock.MatchedBy(func(page file.DataPage) bool {
   149  		pagev2, ok := page.(*file.DataPageV2)
   150  		if !ok {
   151  			return false
   152  		}
   153  
   154  		encodedStats := pagev2.Statistics()
   155  		// only match if the page being written has 2 nulls, 6 values and 3 rows
   156  		return !pagev2.IsCompressed() &&
   157  			pagev2.NumNulls() == 2 && encodedStats.NullCount == 2 &&
   158  			pagev2.NumValues() == 6 &&
   159  			pagev2.NumRows() == 3
   160  	})).Return(10, nil)
   161  
   162  	wr.FlushBufferedDataPages()
   163  	assert.EqualValues(t, 3, wr.RowsWritten())
   164  }
   165  
   166  func TestDataPageV2RowBoundaries(t *testing.T) {
   167  	sc := schema.NewSchema(schema.MustGroup(schema.NewGroupNode("schema", parquet.Repetitions.Required, schema.FieldList{
   168  		schema.Must(schema.ListOf(
   169  			schema.Must(schema.NewPrimitiveNode("column", parquet.Repetitions.Optional, parquet.Types.Int32, -1, -1)),
   170  			parquet.Repetitions.Optional, -1)),
   171  	}, -1)))
   172  	descr := sc.Column(0)
   173  	props := parquet.NewWriterProperties(
   174  		parquet.WithBatchSize(128),
   175  		parquet.WithDataPageSize(1024),
   176  		parquet.WithVersion(parquet.V2_LATEST),
   177  		parquet.WithDataPageVersion(parquet.DataPageV2),
   178  		parquet.WithDictionaryDefault(false))
   179  
   180  	metadata := metadata.NewColumnChunkMetaDataBuilder(props, descr)
   181  	pager := new(mockpagewriter)
   182  	defer pager.AssertExpectations(t)
   183  	pager.On("HasCompressor").Return(false)
   184  	wr := file.NewColumnChunkWriter(metadata, pager, props).(*file.Int32ColumnChunkWriter)
   185  
   186  	pager.On("WriteDataPage", mock.MatchedBy(func(page file.DataPage) bool {
   187  		pagev2, ok := page.(*file.DataPageV2)
   188  		if !ok {
   189  			return false
   190  		}
   191  
   192  		// only match if the page being written has 2 nulls, 6 values and 3 rows
   193  		return !pagev2.IsCompressed() &&
   194  			pagev2.NumNulls() == 0 &&
   195  			pagev2.NumValues() == 378 &&
   196  			pagev2.NumRows() == 126
   197  	})).Return(10, nil)
   198  
   199  	// create rows of lists of 3 values each
   200  	values := make([]int32, 1024)
   201  	defLevels := make([]int16, 1024)
   202  	repLevels := make([]int16, 1024)
   203  	for i := range values {
   204  		values[i] = int32(i)
   205  		defLevels[i] = 3
   206  
   207  		switch i % 3 {
   208  		case 0:
   209  			repLevels[i] = 0
   210  		case 1, 2:
   211  			repLevels[i] = 1
   212  		}
   213  	}
   214  
   215  	wr.WriteBatch(values, defLevels, repLevels)
   216  }
   217  
   218  type PrimitiveWriterTestSuite struct {
   219  	testutils.PrimitiveTypedTest
   220  	suite.Suite
   221  
   222  	props *parquet.WriterProperties
   223  	descr *schema.Column
   224  
   225  	metadata   *metadata.ColumnChunkMetaDataBuilder
   226  	sink       *encoding.BufferWriter
   227  	readbuffer *memory.Buffer
   228  
   229  	bufferPool sync.Pool
   230  }
   231  
   232  func (p *PrimitiveWriterTestSuite) SetupTest() {
   233  	p.SetupValuesOut(SmallSize)
   234  	p.props = parquet.NewWriterProperties()
   235  	p.SetupSchema(parquet.Repetitions.Required, 1)
   236  	p.descr = p.Schema.Column(0)
   237  
   238  	p.bufferPool = sync.Pool{
   239  		New: func() interface{} {
   240  			buf := memory.NewResizableBuffer(mem)
   241  			runtime.SetFinalizer(buf, func(obj *memory.Buffer) {
   242  				obj.Release()
   243  			})
   244  			return buf
   245  		},
   246  	}
   247  }
   248  
   249  func (p *PrimitiveWriterTestSuite) TearDownTest() {
   250  	p.bufferPool = sync.Pool{}
   251  }
   252  
   253  func (p *PrimitiveWriterTestSuite) buildReader(nrows int64, compression compress.Compression) file.ColumnChunkReader {
   254  	p.readbuffer = p.sink.Finish()
   255  	pagereader, _ := file.NewPageReader(arrutils.NewBufferedReader(bytes.NewReader(p.readbuffer.Bytes()), p.readbuffer.Len()), nrows, compression, mem, nil)
   256  	return file.NewColumnReader(p.descr, pagereader, mem, &p.bufferPool)
   257  }
   258  
   259  func (p *PrimitiveWriterTestSuite) buildWriter(_ int64, columnProps parquet.ColumnProperties, opts ...parquet.WriterProperty) file.ColumnChunkWriter {
   260  	p.sink = encoding.NewBufferWriter(0, mem)
   261  	if columnProps.Encoding == parquet.Encodings.PlainDict || columnProps.Encoding == parquet.Encodings.RLEDict {
   262  		opts = append(opts, parquet.WithDictionaryDefault(true), parquet.WithDictionaryPageSizeLimit(DictionaryPageSize))
   263  	} else {
   264  		opts = append(opts, parquet.WithDictionaryDefault(false), parquet.WithEncoding(columnProps.Encoding))
   265  	}
   266  	opts = append(opts, parquet.WithMaxStatsSize(columnProps.MaxStatsSize), parquet.WithStats(columnProps.StatsEnabled))
   267  	p.props = parquet.NewWriterProperties(opts...)
   268  
   269  	p.metadata = metadata.NewColumnChunkMetaDataBuilder(p.props, p.descr)
   270  	pager, _ := file.NewPageWriter(p.sink, columnProps.Codec, compress.DefaultCompressionLevel, p.metadata, -1, -1, memory.DefaultAllocator, false, nil, nil)
   271  	return file.NewColumnChunkWriter(p.metadata, pager, p.props)
   272  }
   273  
   274  func (p *PrimitiveWriterTestSuite) readColumn(compression compress.Compression) int64 {
   275  	totalValues := int64(len(p.DefLevelsOut))
   276  	reader := p.buildReader(totalValues, compression)
   277  	return p.ReadBatch(reader, totalValues, 0, p.DefLevelsOut, p.RepLevelsOut)
   278  }
   279  
   280  func (p *PrimitiveWriterTestSuite) readColumnFully(compression compress.Compression) int64 {
   281  	totalValues := int64(len(p.DefLevelsOut))
   282  	reader := p.buildReader(totalValues, compression)
   283  	valuesRead := int64(0)
   284  	for valuesRead < totalValues {
   285  		read := p.ReadBatch(reader, totalValues-valuesRead, valuesRead, p.DefLevelsOut[valuesRead:], p.RepLevelsOut[valuesRead:])
   286  		valuesRead += read
   287  	}
   288  	return valuesRead
   289  }
   290  
   291  func (p *PrimitiveWriterTestSuite) readAndCompare(compression compress.Compression, nrows int64) {
   292  	p.SetupValuesOut(nrows)
   293  	p.readColumnFully(compression)
   294  	p.Equal(p.Values, p.ValuesOut)
   295  }
   296  
   297  func (p *PrimitiveWriterTestSuite) writeRequiredWithSettings(encoding parquet.Encoding, compression compress.Compression, dict, stats bool, compressLvl int, nrows int64) {
   298  	columnProperties := parquet.ColumnProperties{
   299  		Encoding:          encoding,
   300  		Codec:             compression,
   301  		DictionaryEnabled: dict,
   302  		StatsEnabled:      stats,
   303  		CompressionLevel:  compressLvl,
   304  	}
   305  	writer := p.buildWriter(nrows, columnProperties, parquet.WithVersion(parquet.V1_0))
   306  	p.WriteBatchValues(writer, nil, nil)
   307  	// behavior should be independant of the number of calls to Close
   308  	writer.Close()
   309  	writer.Close()
   310  }
   311  
   312  func (p *PrimitiveWriterTestSuite) writeRequiredWithSettingsSpaced(encoding parquet.Encoding, compression compress.Compression, dict, stats bool, nrows int64, compressionLvl int) {
   313  	validBits := make([]byte, int(bitutil.BytesForBits(int64(len(p.DefLevels))))+1)
   314  	memory.Set(validBits, 255)
   315  	columnProperties := parquet.ColumnProperties{
   316  		Encoding:          encoding,
   317  		Codec:             compression,
   318  		DictionaryEnabled: dict,
   319  		StatsEnabled:      stats,
   320  		CompressionLevel:  compressionLvl,
   321  	}
   322  	writer := p.buildWriter(nrows, columnProperties, parquet.WithVersion(parquet.V1_0))
   323  	p.WriteBatchValuesSpaced(writer, nil, nil, validBits, 0)
   324  	// behavior should be independant from the number of close calls
   325  	writer.Close()
   326  	writer.Close()
   327  }
   328  
   329  func (p *PrimitiveWriterTestSuite) testRequiredWithSettings(encoding parquet.Encoding, compression compress.Compression, dict, stats bool, nrows int64, compressLvl int) {
   330  	p.GenerateData(nrows)
   331  	p.writeRequiredWithSettings(encoding, compression, dict, stats, compressLvl, nrows)
   332  	p.NotPanics(func() { p.readAndCompare(compression, nrows) })
   333  	p.writeRequiredWithSettingsSpaced(encoding, compression, dict, stats, nrows, compressLvl)
   334  	p.NotPanics(func() { p.readAndCompare(compression, nrows) })
   335  }
   336  
   337  func (p *PrimitiveWriterTestSuite) testRequiredWithEncoding(encoding parquet.Encoding) {
   338  	p.testRequiredWithSettings(encoding, compress.Codecs.Uncompressed, false, false, SmallSize, compress.DefaultCompressionLevel)
   339  }
   340  
   341  func (p *PrimitiveWriterTestSuite) metadataNumValues() int64 {
   342  	// metadata accessor created lazily
   343  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, nil, 0, 0, nil)
   344  	return metadata.NumValues()
   345  }
   346  
   347  func (p *PrimitiveWriterTestSuite) metadataEncodings() []parquet.Encoding {
   348  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, nil, 0, 0, nil)
   349  	return metadata.Encodings()
   350  }
   351  
   352  func (p *PrimitiveWriterTestSuite) metadataEncodingStats() []metadata.PageEncodingStats {
   353  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, nil, 0, 0, nil)
   354  	return metadata.EncodingStats()
   355  }
   356  
   357  func (p *PrimitiveWriterTestSuite) metadataStatsHasMinMax() (hasMin, hasMax bool) {
   358  	appVersion := metadata.NewAppVersion(p.props.CreatedBy())
   359  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, appVersion, 0, 0, nil)
   360  	stats, _ := metadata.Statistics()
   361  	encoded, _ := stats.Encode()
   362  	return encoded.HasMin, encoded.HasMax
   363  }
   364  
   365  func (p *PrimitiveWriterTestSuite) metadataIsStatsSet() bool {
   366  	appVersion := metadata.NewAppVersion(p.props.CreatedBy())
   367  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, appVersion, 0, 0, nil)
   368  	set, _ := metadata.StatsSet()
   369  	return set
   370  }
   371  
   372  func (p *PrimitiveWriterTestSuite) testDictionaryFallbackEncoding(version parquet.Version) {
   373  	p.GenerateData(VeryLargeSize)
   374  	props := parquet.DefaultColumnProperties()
   375  	props.DictionaryEnabled = true
   376  
   377  	if version == parquet.V1_0 {
   378  		props.Encoding = parquet.Encodings.PlainDict
   379  	} else {
   380  		props.Encoding = parquet.Encodings.RLEDict
   381  	}
   382  
   383  	writer := p.buildWriter(VeryLargeSize, props, parquet.WithVersion(version))
   384  	p.WriteBatchValues(writer, nil, nil)
   385  	writer.Close()
   386  
   387  	// Read all the rows so that we are sure that also the non-dictionary pages are read correctly
   388  	p.SetupValuesOut(VeryLargeSize)
   389  	valuesRead := p.readColumnFully(compress.Codecs.Uncompressed)
   390  	p.EqualValues(VeryLargeSize, valuesRead)
   391  	p.Equal(p.Values, p.ValuesOut)
   392  
   393  	encodings := p.metadataEncodings()
   394  	if p.Typ.Kind() == reflect.Bool || p.Typ == reflect.TypeOf(parquet.Int96{}) {
   395  		// dictionary encoding is not allowed for booleans
   396  		// there are 2 encodings (PLAIN, RLE) in a non dictionary encoding case
   397  		p.Equal([]parquet.Encoding{parquet.Encodings.Plain, parquet.Encodings.RLE}, encodings)
   398  	} else if version == parquet.V1_0 {
   399  		// There are 4 encodings (PLAIN_DICTIONARY, PLAIN, RLE, PLAIN) in a fallback case
   400  		// for version 1.0
   401  		p.Equal([]parquet.Encoding{parquet.Encodings.PlainDict, parquet.Encodings.Plain, parquet.Encodings.RLE, parquet.Encodings.Plain}, encodings)
   402  	} else {
   403  		// There are 4 encodings (RLE_DICTIONARY, PLAIN, RLE, PLAIN) in a fallback case for
   404  		// version 2.0
   405  		p.Equal([]parquet.Encoding{parquet.Encodings.RLEDict, parquet.Encodings.Plain, parquet.Encodings.RLE, parquet.Encodings.Plain}, encodings)
   406  	}
   407  
   408  	encodingStats := p.metadataEncodingStats()
   409  	if p.Typ.Kind() == reflect.Bool || p.Typ == reflect.TypeOf(parquet.Int96{}) {
   410  		p.Equal(parquet.Encodings.Plain, encodingStats[0].Encoding)
   411  		p.Equal(format.PageType_DATA_PAGE, encodingStats[0].PageType)
   412  	} else if version == parquet.V1_0 {
   413  		expected := []metadata.PageEncodingStats{
   414  			{Encoding: parquet.Encodings.PlainDict, PageType: format.PageType_DICTIONARY_PAGE},
   415  			{Encoding: parquet.Encodings.Plain, PageType: format.PageType_DATA_PAGE},
   416  			{Encoding: parquet.Encodings.PlainDict, PageType: format.PageType_DATA_PAGE}}
   417  		p.Equal(expected[0], encodingStats[0])
   418  		p.ElementsMatch(expected[1:], encodingStats[1:])
   419  	} else {
   420  		expected := []metadata.PageEncodingStats{
   421  			{Encoding: parquet.Encodings.Plain, PageType: format.PageType_DICTIONARY_PAGE},
   422  			{Encoding: parquet.Encodings.Plain, PageType: format.PageType_DATA_PAGE},
   423  			{Encoding: parquet.Encodings.RLEDict, PageType: format.PageType_DATA_PAGE}}
   424  		p.Equal(expected[0], encodingStats[0])
   425  		p.ElementsMatch(expected[1:], encodingStats[1:])
   426  	}
   427  }
   428  
   429  func (p *PrimitiveWriterTestSuite) TestRequiredPlain() {
   430  	p.testRequiredWithEncoding(parquet.Encodings.Plain)
   431  }
   432  
   433  func (p *PrimitiveWriterTestSuite) TestRequiredDictionary() {
   434  	p.testRequiredWithEncoding(parquet.Encodings.PlainDict)
   435  }
   436  
   437  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStats() {
   438  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Uncompressed, false, true, LargeSize, compress.DefaultCompressionLevel)
   439  }
   440  
   441  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithSnappy() {
   442  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Snappy, false, false, LargeSize, compress.DefaultCompressionLevel)
   443  }
   444  
   445  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStatsAndSnappy() {
   446  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Snappy, false, true, LargeSize, compress.DefaultCompressionLevel)
   447  }
   448  
   449  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithBrotli() {
   450  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Brotli, false, false, LargeSize, compress.DefaultCompressionLevel)
   451  }
   452  
   453  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithBrotliAndLevel() {
   454  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Brotli, false, false, LargeSize, 10)
   455  }
   456  
   457  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStatsAndBrotli() {
   458  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Brotli, false, true, LargeSize, compress.DefaultCompressionLevel)
   459  }
   460  
   461  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithGzip() {
   462  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Gzip, false, false, LargeSize, compress.DefaultCompressionLevel)
   463  }
   464  
   465  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithGzipAndLevel() {
   466  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Gzip, false, false, LargeSize, 10)
   467  }
   468  
   469  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStatsAndGzip() {
   470  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Gzip, false, true, LargeSize, compress.DefaultCompressionLevel)
   471  }
   472  
   473  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithZstd() {
   474  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Zstd, false, false, LargeSize, compress.DefaultCompressionLevel)
   475  }
   476  
   477  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithZstdAndLevel() {
   478  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Zstd, false, false, LargeSize, 6)
   479  }
   480  
   481  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStatsAndZstd() {
   482  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Zstd, false, true, LargeSize, compress.DefaultCompressionLevel)
   483  }
   484  
   485  func (p *PrimitiveWriterTestSuite) TestOptionalNonRepeated() {
   486  	p.SetupSchema(parquet.Repetitions.Optional, 1)
   487  	p.descr = p.Schema.Column(0)
   488  
   489  	p.GenerateData(SmallSize)
   490  	p.DefLevels[1] = 0
   491  
   492  	writer := p.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.WithVersion(parquet.V1_0))
   493  	p.WriteBatchValues(writer, p.DefLevels, nil)
   494  	writer.Close()
   495  
   496  	p.Equal(int64(100), p.metadataNumValues())
   497  
   498  	values := p.readColumn(compress.Codecs.Uncompressed)
   499  	p.EqualValues(99, values)
   500  	p.Equal(reflect.ValueOf(p.Values).Slice(0, 99).Interface(), reflect.ValueOf(p.ValuesOut).Slice(0, 99).Interface())
   501  }
   502  
   503  func (p *PrimitiveWriterTestSuite) TestOptionalSpaced() {
   504  	p.SetupSchema(parquet.Repetitions.Optional, 1)
   505  	p.descr = p.Schema.Column(0)
   506  
   507  	p.GenerateData(SmallSize)
   508  	validBits := make([]byte, int(bitutil.BytesForBits(SmallSize)))
   509  	memory.Set(validBits, 255)
   510  	p.DefLevels[SmallSize-1] = 0
   511  	bitutil.ClearBit(validBits, SmallSize-1)
   512  	p.DefLevels[1] = 0
   513  	bitutil.ClearBit(validBits, 1)
   514  
   515  	writer := p.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.WithVersion(parquet.V1_0))
   516  	p.WriteBatchValuesSpaced(writer, p.DefLevels, nil, validBits, 0)
   517  	writer.Close()
   518  
   519  	p.Equal(int64(100), p.metadataNumValues())
   520  
   521  	values := p.readColumn(compress.Codecs.Uncompressed)
   522  	p.EqualValues(98, values)
   523  
   524  	orig := reflect.ValueOf(p.Values)
   525  	orig = orig.Slice(0, 99)
   526  	reflect.Copy(orig.Slice(1, orig.Len()), orig.Slice(2, orig.Len()))
   527  	orig = orig.Slice(0, 98)
   528  	out := reflect.ValueOf(p.ValuesOut)
   529  	out = out.Slice(0, 98)
   530  
   531  	p.Equal(orig.Interface(), out.Interface())
   532  }
   533  
   534  func (p *PrimitiveWriterTestSuite) TestWriteRepeated() {
   535  	// optional and repeated so def and repetition levels
   536  	p.SetupSchema(parquet.Repetitions.Repeated, 1)
   537  	p.descr = p.Schema.Column(0)
   538  	p.GenerateData(SmallSize)
   539  	p.DefLevels[1] = 0
   540  	p.RepLevels = make([]int16, SmallSize)
   541  	for idx := range p.RepLevels {
   542  		p.RepLevels[idx] = 0
   543  	}
   544  
   545  	writer := p.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.WithVersion(parquet.V1_0))
   546  	p.WriteBatchValues(writer, p.DefLevels, p.RepLevels)
   547  	writer.Close()
   548  
   549  	values := p.readColumn(compress.Codecs.Uncompressed)
   550  	p.EqualValues(SmallSize-1, values)
   551  	out := reflect.ValueOf(p.ValuesOut).Slice(0, SmallSize-1).Interface()
   552  	vals := reflect.ValueOf(p.Values).Slice(0, SmallSize-1).Interface()
   553  	p.Equal(vals, out)
   554  }
   555  
   556  func (p *PrimitiveWriterTestSuite) TestRequiredLargeChunk() {
   557  	p.GenerateData(LargeSize)
   558  
   559  	// Test 1: required and non-repeated, so no def or rep levels
   560  	writer := p.buildWriter(LargeSize, parquet.DefaultColumnProperties(), parquet.WithVersion(parquet.V1_0))
   561  	p.WriteBatchValues(writer, nil, nil)
   562  	writer.Close()
   563  
   564  	// just read the first SmallSize rows to ensure we could read it back in
   565  	values := p.readColumn(compress.Codecs.Uncompressed)
   566  	p.EqualValues(SmallSize, values)
   567  	p.Equal(reflect.ValueOf(p.Values).Slice(0, SmallSize).Interface(), p.ValuesOut)
   568  }
   569  
   570  func (p *PrimitiveWriterTestSuite) TestDictionaryFallbackEncodingV1() {
   571  	p.testDictionaryFallbackEncoding(parquet.V1_0)
   572  }
   573  
   574  func (p *PrimitiveWriterTestSuite) TestDictionaryFallbackEncodingV2() {
   575  	p.testDictionaryFallbackEncoding(parquet.V2_LATEST)
   576  }
   577  
   578  func (p *PrimitiveWriterTestSuite) TestOptionalNullValueChunk() {
   579  	// test case for NULL values
   580  	p.SetupSchema(parquet.Repetitions.Optional, 1)
   581  	p.descr = p.Schema.Column(0)
   582  	p.GenerateData(LargeSize)
   583  	p.RepLevels = make([]int16, LargeSize)
   584  	for idx := range p.DefLevels {
   585  		p.DefLevels[idx] = 0
   586  		p.RepLevels[idx] = 0
   587  	}
   588  
   589  	writer := p.buildWriter(LargeSize, parquet.DefaultColumnProperties(), parquet.WithVersion(parquet.V1_0))
   590  	p.WriteBatchValues(writer, p.DefLevels, p.RepLevels)
   591  	writer.Close()
   592  
   593  	valuesRead := p.readColumn(compress.Codecs.Uncompressed)
   594  	p.Zero(valuesRead)
   595  }
   596  
   597  func createWriterTestSuite(typ reflect.Type) suite.TestingSuite {
   598  	switch typ {
   599  	case reflect.TypeOf(true):
   600  		return &BooleanValueWriterSuite{PrimitiveWriterTestSuite{PrimitiveTypedTest: testutils.NewPrimitiveTypedTest(typ)}}
   601  	case reflect.TypeOf(parquet.ByteArray{}):
   602  		return &ByteArrayWriterSuite{PrimitiveWriterTestSuite{PrimitiveTypedTest: testutils.NewPrimitiveTypedTest(typ)}}
   603  	}
   604  	return &PrimitiveWriterTestSuite{PrimitiveTypedTest: testutils.NewPrimitiveTypedTest(typ)}
   605  }
   606  
   607  func TestColumnWriter(t *testing.T) {
   608  	t.Parallel()
   609  	types := []struct {
   610  		typ reflect.Type
   611  	}{
   612  		{reflect.TypeOf(true)},
   613  		{reflect.TypeOf(int32(0))},
   614  		{reflect.TypeOf(int64(0))},
   615  		{reflect.TypeOf(float32(0))},
   616  		{reflect.TypeOf(float64(0))},
   617  		{reflect.TypeOf(parquet.Int96{})},
   618  		{reflect.TypeOf(parquet.ByteArray{})},
   619  		{reflect.TypeOf(parquet.FixedLenByteArray{})},
   620  	}
   621  	for _, tt := range types {
   622  		tt := tt
   623  		t.Run(tt.typ.String(), func(t *testing.T) {
   624  			t.Parallel()
   625  			suite.Run(t, createWriterTestSuite(tt.typ))
   626  		})
   627  	}
   628  }
   629  
   630  type ByteArrayWriterSuite struct {
   631  	PrimitiveWriterTestSuite
   632  }
   633  
   634  func (b *ByteArrayWriterSuite) TestOmitStats() {
   635  	// prevent writing large MIN,MAX stats
   636  	minLen := 1024 * 4
   637  	maxLen := 1024 * 8
   638  	b.SetupSchema(parquet.Repetitions.Required, 1)
   639  	b.Values = make([]parquet.ByteArray, SmallSize)
   640  	writer := b.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.WithVersion(parquet.V1_0))
   641  	testutils.RandomByteArray(0, b.Values.([]parquet.ByteArray), b.Buffer, minLen, maxLen)
   642  	writer.(*file.ByteArrayColumnChunkWriter).WriteBatch(b.Values.([]parquet.ByteArray), nil, nil)
   643  	writer.Close()
   644  
   645  	hasMin, hasMax := b.metadataStatsHasMinMax()
   646  	b.False(hasMin)
   647  	b.False(hasMax)
   648  }
   649  
   650  func (b *ByteArrayWriterSuite) TestOmitDataPageStats() {
   651  	// prevent writing large stats in DataPageHeader
   652  	minLen := math.Pow10(7)
   653  	maxLen := math.Pow10(7)
   654  	b.SetupSchema(parquet.Repetitions.Required, 1)
   655  	colprops := parquet.DefaultColumnProperties()
   656  	colprops.StatsEnabled = false
   657  
   658  	writer := b.buildWriter(SmallSize, colprops, parquet.WithVersion(parquet.V1_0))
   659  	b.Values = make([]parquet.ByteArray, 1)
   660  	testutils.RandomByteArray(0, b.Values.([]parquet.ByteArray), b.Buffer, int(minLen), int(maxLen))
   661  	writer.(*file.ByteArrayColumnChunkWriter).WriteBatch(b.Values.([]parquet.ByteArray), nil, nil)
   662  	writer.Close()
   663  
   664  	b.NotPanics(func() { b.readColumn(compress.Codecs.Uncompressed) })
   665  }
   666  
   667  func (b *ByteArrayWriterSuite) TestLimitStats() {
   668  	minLen := 1024 * 4
   669  	maxLen := 1024 * 8
   670  	b.SetupSchema(parquet.Repetitions.Required, 1)
   671  	colprops := parquet.DefaultColumnProperties()
   672  	colprops.MaxStatsSize = int64(maxLen)
   673  
   674  	writer := b.buildWriter(SmallSize, colprops, parquet.WithVersion(parquet.V1_0)).(*file.ByteArrayColumnChunkWriter)
   675  	b.Values = make([]parquet.ByteArray, SmallSize)
   676  	testutils.RandomByteArray(0, b.Values.([]parquet.ByteArray), b.Buffer, minLen, maxLen)
   677  	writer.WriteBatch(b.Values.([]parquet.ByteArray), nil, nil)
   678  	writer.Close()
   679  
   680  	b.True(b.metadataIsStatsSet())
   681  }
   682  
   683  func (b *ByteArrayWriterSuite) TestCheckDefaultStats() {
   684  	b.SetupSchema(parquet.Repetitions.Required, 1)
   685  	writer := b.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.WithVersion(parquet.V1_0))
   686  	b.GenerateData(SmallSize)
   687  	b.WriteBatchValues(writer, nil, nil)
   688  	writer.Close()
   689  
   690  	b.True(b.metadataIsStatsSet())
   691  }
   692  
   693  type BooleanValueWriterSuite struct {
   694  	PrimitiveWriterTestSuite
   695  }
   696  
   697  func (b *BooleanValueWriterSuite) TestAlternateBooleanValues() {
   698  	b.SetupSchema(parquet.Repetitions.Required, 1)
   699  	// We use an unusual data-page size to try to flush out Boolean encoder issues in usage of the BitMapWriter
   700  	writer := b.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.WithVersion(parquet.V1_0), parquet.WithDataPageSize(7)).(*file.BooleanColumnChunkWriter)
   701  	for i := 0; i < SmallSize; i++ {
   702  		val := i%2 == 0
   703  		writer.WriteBatch([]bool{val}, nil, nil)
   704  	}
   705  	writer.Close()
   706  	b.readColumn(compress.Codecs.Uncompressed)
   707  	for i := 0; i < SmallSize; i++ {
   708  		b.Equal(i%2 == 0, b.ValuesOut.([]bool)[i])
   709  	}
   710  }