github.com/apache/arrow/go/v10@v10.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/v10/arrow/bitutil"
    28  	"github.com/apache/arrow/go/v10/arrow/memory"
    29  	arrutils "github.com/apache/arrow/go/v10/internal/utils"
    30  	"github.com/apache/arrow/go/v10/parquet"
    31  	"github.com/apache/arrow/go/v10/parquet/compress"
    32  	"github.com/apache/arrow/go/v10/parquet/file"
    33  	"github.com/apache/arrow/go/v10/parquet/internal/encoding"
    34  	"github.com/apache/arrow/go/v10/parquet/internal/encryption"
    35  	format "github.com/apache/arrow/go/v10/parquet/internal/gen-go/parquet"
    36  	"github.com/apache/arrow/go/v10/parquet/internal/testutils"
    37  	"github.com/apache/arrow/go/v10/parquet/internal/utils"
    38  	"github.com/apache/arrow/go/v10/parquet/metadata"
    39  	"github.com/apache/arrow/go/v10/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, version parquet.Version) file.ColumnChunkWriter {
   260  	p.sink = encoding.NewBufferWriter(0, mem)
   261  	opts := make([]parquet.WriterProperty, 0)
   262  	opts = append(opts, parquet.WithVersion(version))
   263  	if columnProps.Encoding == parquet.Encodings.PlainDict || columnProps.Encoding == parquet.Encodings.RLEDict {
   264  		opts = append(opts, parquet.WithDictionaryDefault(true), parquet.WithDictionaryPageSizeLimit(DictionaryPageSize))
   265  	} else {
   266  		opts = append(opts, parquet.WithDictionaryDefault(false), parquet.WithEncoding(columnProps.Encoding))
   267  	}
   268  	opts = append(opts, parquet.WithMaxStatsSize(columnProps.MaxStatsSize), parquet.WithStats(columnProps.StatsEnabled))
   269  	p.props = parquet.NewWriterProperties(opts...)
   270  
   271  	p.metadata = metadata.NewColumnChunkMetaDataBuilder(p.props, p.descr)
   272  	pager, _ := file.NewPageWriter(p.sink, columnProps.Codec, compress.DefaultCompressionLevel, p.metadata, -1, -1, memory.DefaultAllocator, false, nil, nil)
   273  	return file.NewColumnChunkWriter(p.metadata, pager, p.props)
   274  }
   275  
   276  func (p *PrimitiveWriterTestSuite) readColumn(compression compress.Compression) int64 {
   277  	totalValues := int64(len(p.DefLevelsOut))
   278  	reader := p.buildReader(totalValues, compression)
   279  	return p.ReadBatch(reader, totalValues, 0, p.DefLevelsOut, p.RepLevelsOut)
   280  }
   281  
   282  func (p *PrimitiveWriterTestSuite) readColumnFully(compression compress.Compression) int64 {
   283  	totalValues := int64(len(p.DefLevelsOut))
   284  	reader := p.buildReader(totalValues, compression)
   285  	valuesRead := int64(0)
   286  	for valuesRead < totalValues {
   287  		read := p.ReadBatch(reader, totalValues-valuesRead, valuesRead, p.DefLevelsOut[valuesRead:], p.RepLevelsOut[valuesRead:])
   288  		valuesRead += read
   289  	}
   290  	return valuesRead
   291  }
   292  
   293  func (p *PrimitiveWriterTestSuite) readAndCompare(compression compress.Compression, nrows int64) {
   294  	p.SetupValuesOut(nrows)
   295  	p.readColumnFully(compression)
   296  	p.Equal(p.Values, p.ValuesOut)
   297  }
   298  
   299  func (p *PrimitiveWriterTestSuite) writeRequiredWithSettings(encoding parquet.Encoding, compression compress.Compression, dict, stats bool, compressLvl int, nrows int64) {
   300  	columnProperties := parquet.ColumnProperties{
   301  		Encoding:          encoding,
   302  		Codec:             compression,
   303  		DictionaryEnabled: dict,
   304  		StatsEnabled:      stats,
   305  		CompressionLevel:  compressLvl,
   306  	}
   307  	writer := p.buildWriter(nrows, columnProperties, parquet.V1_0)
   308  	p.WriteBatchValues(writer, nil, nil)
   309  	// behavior should be independant of the number of calls to Close
   310  	writer.Close()
   311  	writer.Close()
   312  }
   313  
   314  func (p *PrimitiveWriterTestSuite) writeRequiredWithSettingsSpaced(encoding parquet.Encoding, compression compress.Compression, dict, stats bool, nrows int64, compressionLvl int) {
   315  	validBits := make([]byte, int(bitutil.BytesForBits(int64(len(p.DefLevels))))+1)
   316  	memory.Set(validBits, 255)
   317  	columnProperties := parquet.ColumnProperties{
   318  		Encoding:          encoding,
   319  		Codec:             compression,
   320  		DictionaryEnabled: dict,
   321  		StatsEnabled:      stats,
   322  		CompressionLevel:  compressionLvl,
   323  	}
   324  	writer := p.buildWriter(nrows, columnProperties, parquet.V1_0)
   325  	p.WriteBatchValuesSpaced(writer, nil, nil, validBits, 0)
   326  	// behavior should be independant from the number of close calls
   327  	writer.Close()
   328  	writer.Close()
   329  }
   330  
   331  func (p *PrimitiveWriterTestSuite) testRequiredWithSettings(encoding parquet.Encoding, compression compress.Compression, dict, stats bool, nrows int64, compressLvl int) {
   332  	p.GenerateData(nrows)
   333  	p.writeRequiredWithSettings(encoding, compression, dict, stats, compressLvl, nrows)
   334  	p.NotPanics(func() { p.readAndCompare(compression, nrows) })
   335  	p.writeRequiredWithSettingsSpaced(encoding, compression, dict, stats, nrows, compressLvl)
   336  	p.NotPanics(func() { p.readAndCompare(compression, nrows) })
   337  }
   338  
   339  func (p *PrimitiveWriterTestSuite) testRequiredWithEncoding(encoding parquet.Encoding) {
   340  	p.testRequiredWithSettings(encoding, compress.Codecs.Uncompressed, false, false, SmallSize, compress.DefaultCompressionLevel)
   341  }
   342  
   343  func (p *PrimitiveWriterTestSuite) metadataNumValues() int64 {
   344  	// metadata accessor created lazily
   345  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, nil, 0, 0, nil)
   346  	return metadata.NumValues()
   347  }
   348  
   349  func (p *PrimitiveWriterTestSuite) metadataEncodings() []parquet.Encoding {
   350  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, nil, 0, 0, nil)
   351  	return metadata.Encodings()
   352  }
   353  
   354  func (p *PrimitiveWriterTestSuite) metadataEncodingStats() []metadata.PageEncodingStats {
   355  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, nil, 0, 0, nil)
   356  	return metadata.EncodingStats()
   357  }
   358  
   359  func (p *PrimitiveWriterTestSuite) metadataStatsHasMinMax() (hasMin, hasMax bool) {
   360  	appVersion := metadata.NewAppVersion(p.props.CreatedBy())
   361  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, appVersion, 0, 0, nil)
   362  	stats, _ := metadata.Statistics()
   363  	encoded, _ := stats.Encode()
   364  	return encoded.HasMin, encoded.HasMax
   365  }
   366  
   367  func (p *PrimitiveWriterTestSuite) metadataIsStatsSet() bool {
   368  	appVersion := metadata.NewAppVersion(p.props.CreatedBy())
   369  	metadata, _ := metadata.NewColumnChunkMetaData(p.metadata.Contents(), p.descr, appVersion, 0, 0, nil)
   370  	set, _ := metadata.StatsSet()
   371  	return set
   372  }
   373  
   374  func (p *PrimitiveWriterTestSuite) testDictionaryFallbackEncoding(version parquet.Version) {
   375  	p.GenerateData(VeryLargeSize)
   376  	props := parquet.DefaultColumnProperties()
   377  	props.DictionaryEnabled = true
   378  
   379  	if version == parquet.V1_0 {
   380  		props.Encoding = parquet.Encodings.PlainDict
   381  	} else {
   382  		props.Encoding = parquet.Encodings.RLEDict
   383  	}
   384  
   385  	writer := p.buildWriter(VeryLargeSize, props, version)
   386  	p.WriteBatchValues(writer, nil, nil)
   387  	writer.Close()
   388  
   389  	// Read all the rows so that we are sure that also the non-dictionary pages are read correctly
   390  	p.SetupValuesOut(VeryLargeSize)
   391  	valuesRead := p.readColumnFully(compress.Codecs.Uncompressed)
   392  	p.EqualValues(VeryLargeSize, valuesRead)
   393  	p.Equal(p.Values, p.ValuesOut)
   394  
   395  	encodings := p.metadataEncodings()
   396  	if p.Typ.Kind() == reflect.Bool || p.Typ == reflect.TypeOf(parquet.Int96{}) {
   397  		// dictionary encoding is not allowed for booleans
   398  		// there are 2 encodings (PLAIN, RLE) in a non dictionary encoding case
   399  		p.Equal([]parquet.Encoding{parquet.Encodings.Plain, parquet.Encodings.RLE}, encodings)
   400  	} else if version == parquet.V1_0 {
   401  		// There are 4 encodings (PLAIN_DICTIONARY, PLAIN, RLE, PLAIN) in a fallback case
   402  		// for version 1.0
   403  		p.Equal([]parquet.Encoding{parquet.Encodings.PlainDict, parquet.Encodings.Plain, parquet.Encodings.RLE, parquet.Encodings.Plain}, encodings)
   404  	} else {
   405  		// There are 4 encodings (RLE_DICTIONARY, PLAIN, RLE, PLAIN) in a fallback case for
   406  		// version 2.0
   407  		p.Equal([]parquet.Encoding{parquet.Encodings.RLEDict, parquet.Encodings.Plain, parquet.Encodings.RLE, parquet.Encodings.Plain}, encodings)
   408  	}
   409  
   410  	encodingStats := p.metadataEncodingStats()
   411  	if p.Typ.Kind() == reflect.Bool || p.Typ == reflect.TypeOf(parquet.Int96{}) {
   412  		p.Equal(parquet.Encodings.Plain, encodingStats[0].Encoding)
   413  		p.Equal(format.PageType_DATA_PAGE, encodingStats[0].PageType)
   414  	} else if version == parquet.V1_0 {
   415  		expected := []metadata.PageEncodingStats{
   416  			{Encoding: parquet.Encodings.PlainDict, PageType: format.PageType_DICTIONARY_PAGE},
   417  			{Encoding: parquet.Encodings.Plain, PageType: format.PageType_DATA_PAGE},
   418  			{Encoding: parquet.Encodings.PlainDict, PageType: format.PageType_DATA_PAGE}}
   419  		p.Equal(expected[0], encodingStats[0])
   420  		p.ElementsMatch(expected[1:], encodingStats[1:])
   421  	} else {
   422  		expected := []metadata.PageEncodingStats{
   423  			{Encoding: parquet.Encodings.Plain, PageType: format.PageType_DICTIONARY_PAGE},
   424  			{Encoding: parquet.Encodings.Plain, PageType: format.PageType_DATA_PAGE},
   425  			{Encoding: parquet.Encodings.RLEDict, PageType: format.PageType_DATA_PAGE}}
   426  		p.Equal(expected[0], encodingStats[0])
   427  		p.ElementsMatch(expected[1:], encodingStats[1:])
   428  	}
   429  }
   430  
   431  func (p *PrimitiveWriterTestSuite) TestRequiredPlain() {
   432  	p.testRequiredWithEncoding(parquet.Encodings.Plain)
   433  }
   434  
   435  func (p *PrimitiveWriterTestSuite) TestRequiredDictionary() {
   436  	p.testRequiredWithEncoding(parquet.Encodings.PlainDict)
   437  }
   438  
   439  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStats() {
   440  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Uncompressed, false, true, LargeSize, compress.DefaultCompressionLevel)
   441  }
   442  
   443  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithSnappy() {
   444  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Snappy, false, false, LargeSize, compress.DefaultCompressionLevel)
   445  }
   446  
   447  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStatsAndSnappy() {
   448  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Snappy, false, true, LargeSize, compress.DefaultCompressionLevel)
   449  }
   450  
   451  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithBrotli() {
   452  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Brotli, false, false, LargeSize, compress.DefaultCompressionLevel)
   453  }
   454  
   455  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithBrotliAndLevel() {
   456  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Brotli, false, false, LargeSize, 10)
   457  }
   458  
   459  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStatsAndBrotli() {
   460  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Brotli, false, true, LargeSize, compress.DefaultCompressionLevel)
   461  }
   462  
   463  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithGzip() {
   464  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Gzip, false, false, LargeSize, compress.DefaultCompressionLevel)
   465  }
   466  
   467  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithGzipAndLevel() {
   468  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Gzip, false, false, LargeSize, 10)
   469  }
   470  
   471  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStatsAndGzip() {
   472  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Gzip, false, true, LargeSize, compress.DefaultCompressionLevel)
   473  }
   474  
   475  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithZstd() {
   476  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Zstd, false, false, LargeSize, compress.DefaultCompressionLevel)
   477  }
   478  
   479  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithZstdAndLevel() {
   480  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Zstd, false, false, LargeSize, 6)
   481  }
   482  
   483  func (p *PrimitiveWriterTestSuite) TestRequiredPlainWithStatsAndZstd() {
   484  	p.testRequiredWithSettings(parquet.Encodings.Plain, compress.Codecs.Zstd, false, true, LargeSize, compress.DefaultCompressionLevel)
   485  }
   486  
   487  func (p *PrimitiveWriterTestSuite) TestOptionalNonRepeated() {
   488  	p.SetupSchema(parquet.Repetitions.Optional, 1)
   489  	p.descr = p.Schema.Column(0)
   490  
   491  	p.GenerateData(SmallSize)
   492  	p.DefLevels[1] = 0
   493  
   494  	writer := p.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.V1_0)
   495  	p.WriteBatchValues(writer, p.DefLevels, nil)
   496  	writer.Close()
   497  
   498  	p.Equal(int64(100), p.metadataNumValues())
   499  
   500  	values := p.readColumn(compress.Codecs.Uncompressed)
   501  	p.EqualValues(99, values)
   502  	p.Equal(reflect.ValueOf(p.Values).Slice(0, 99).Interface(), reflect.ValueOf(p.ValuesOut).Slice(0, 99).Interface())
   503  }
   504  
   505  func (p *PrimitiveWriterTestSuite) TestOptionalSpaced() {
   506  	p.SetupSchema(parquet.Repetitions.Optional, 1)
   507  	p.descr = p.Schema.Column(0)
   508  
   509  	p.GenerateData(SmallSize)
   510  	validBits := make([]byte, int(bitutil.BytesForBits(SmallSize)))
   511  	memory.Set(validBits, 255)
   512  	p.DefLevels[SmallSize-1] = 0
   513  	bitutil.ClearBit(validBits, SmallSize-1)
   514  	p.DefLevels[1] = 0
   515  	bitutil.ClearBit(validBits, 1)
   516  
   517  	writer := p.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.V1_0)
   518  	p.WriteBatchValuesSpaced(writer, p.DefLevels, nil, validBits, 0)
   519  	writer.Close()
   520  
   521  	p.Equal(int64(100), p.metadataNumValues())
   522  
   523  	values := p.readColumn(compress.Codecs.Uncompressed)
   524  	p.EqualValues(98, values)
   525  
   526  	orig := reflect.ValueOf(p.Values)
   527  	orig = orig.Slice(0, 99)
   528  	reflect.Copy(orig.Slice(1, orig.Len()), orig.Slice(2, orig.Len()))
   529  	orig = orig.Slice(0, 98)
   530  	out := reflect.ValueOf(p.ValuesOut)
   531  	out = out.Slice(0, 98)
   532  
   533  	p.Equal(orig.Interface(), out.Interface())
   534  }
   535  
   536  func (p *PrimitiveWriterTestSuite) TestWriteRepeated() {
   537  	// optional and repeated so def and repetition levels
   538  	p.SetupSchema(parquet.Repetitions.Repeated, 1)
   539  	p.descr = p.Schema.Column(0)
   540  	p.GenerateData(SmallSize)
   541  	p.DefLevels[1] = 0
   542  	p.RepLevels = make([]int16, SmallSize)
   543  	for idx := range p.RepLevels {
   544  		p.RepLevels[idx] = 0
   545  	}
   546  
   547  	writer := p.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.V1_0)
   548  	p.WriteBatchValues(writer, p.DefLevels, p.RepLevels)
   549  	writer.Close()
   550  
   551  	values := p.readColumn(compress.Codecs.Uncompressed)
   552  	p.EqualValues(SmallSize-1, values)
   553  	out := reflect.ValueOf(p.ValuesOut).Slice(0, SmallSize-1).Interface()
   554  	vals := reflect.ValueOf(p.Values).Slice(0, SmallSize-1).Interface()
   555  	p.Equal(vals, out)
   556  }
   557  
   558  func (p *PrimitiveWriterTestSuite) TestRequiredLargeChunk() {
   559  	p.GenerateData(LargeSize)
   560  
   561  	// Test 1: required and non-repeated, so no def or rep levels
   562  	writer := p.buildWriter(LargeSize, parquet.DefaultColumnProperties(), parquet.V1_0)
   563  	p.WriteBatchValues(writer, nil, nil)
   564  	writer.Close()
   565  
   566  	// just read the first SmallSize rows to ensure we could read it back in
   567  	values := p.readColumn(compress.Codecs.Uncompressed)
   568  	p.EqualValues(SmallSize, values)
   569  	p.Equal(reflect.ValueOf(p.Values).Slice(0, SmallSize).Interface(), p.ValuesOut)
   570  }
   571  
   572  func (p *PrimitiveWriterTestSuite) TestDictionaryFallbackEncodingV1() {
   573  	p.testDictionaryFallbackEncoding(parquet.V1_0)
   574  }
   575  
   576  func (p *PrimitiveWriterTestSuite) TestDictionaryFallbackEncodingV2() {
   577  	p.testDictionaryFallbackEncoding(parquet.V2_LATEST)
   578  }
   579  
   580  func (p *PrimitiveWriterTestSuite) TestOptionalNullValueChunk() {
   581  	// test case for NULL values
   582  	p.SetupSchema(parquet.Repetitions.Optional, 1)
   583  	p.descr = p.Schema.Column(0)
   584  	p.GenerateData(LargeSize)
   585  	p.RepLevels = make([]int16, LargeSize)
   586  	for idx := range p.DefLevels {
   587  		p.DefLevels[idx] = 0
   588  		p.RepLevels[idx] = 0
   589  	}
   590  
   591  	writer := p.buildWriter(LargeSize, parquet.DefaultColumnProperties(), parquet.V1_0)
   592  	p.WriteBatchValues(writer, p.DefLevels, p.RepLevels)
   593  	writer.Close()
   594  
   595  	valuesRead := p.readColumn(compress.Codecs.Uncompressed)
   596  	p.Zero(valuesRead)
   597  }
   598  
   599  func createWriterTestSuite(typ reflect.Type) suite.TestingSuite {
   600  	switch typ {
   601  	case reflect.TypeOf(true):
   602  		return &BooleanValueWriterSuite{PrimitiveWriterTestSuite{PrimitiveTypedTest: testutils.NewPrimitiveTypedTest(typ)}}
   603  	case reflect.TypeOf(parquet.ByteArray{}):
   604  		return &ByteArrayWriterSuite{PrimitiveWriterTestSuite{PrimitiveTypedTest: testutils.NewPrimitiveTypedTest(typ)}}
   605  	}
   606  	return &PrimitiveWriterTestSuite{PrimitiveTypedTest: testutils.NewPrimitiveTypedTest(typ)}
   607  }
   608  
   609  func TestColumnWriter(t *testing.T) {
   610  	t.Parallel()
   611  	types := []struct {
   612  		typ reflect.Type
   613  	}{
   614  		{reflect.TypeOf(true)},
   615  		{reflect.TypeOf(int32(0))},
   616  		{reflect.TypeOf(int64(0))},
   617  		{reflect.TypeOf(float32(0))},
   618  		{reflect.TypeOf(float64(0))},
   619  		{reflect.TypeOf(parquet.Int96{})},
   620  		{reflect.TypeOf(parquet.ByteArray{})},
   621  		{reflect.TypeOf(parquet.FixedLenByteArray{})},
   622  	}
   623  	for _, tt := range types {
   624  		tt := tt
   625  		t.Run(tt.typ.String(), func(t *testing.T) {
   626  			t.Parallel()
   627  			suite.Run(t, createWriterTestSuite(tt.typ))
   628  		})
   629  	}
   630  }
   631  
   632  type ByteArrayWriterSuite struct {
   633  	PrimitiveWriterTestSuite
   634  }
   635  
   636  func (b *ByteArrayWriterSuite) TestOmitStats() {
   637  	// prevent writing large MIN,MAX stats
   638  	minLen := 1024 * 4
   639  	maxLen := 1024 * 8
   640  	b.SetupSchema(parquet.Repetitions.Required, 1)
   641  	b.Values = make([]parquet.ByteArray, SmallSize)
   642  	writer := b.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.V1_0)
   643  	testutils.RandomByteArray(0, b.Values.([]parquet.ByteArray), b.Buffer, minLen, maxLen)
   644  	writer.(*file.ByteArrayColumnChunkWriter).WriteBatch(b.Values.([]parquet.ByteArray), nil, nil)
   645  	writer.Close()
   646  
   647  	hasMin, hasMax := b.metadataStatsHasMinMax()
   648  	b.False(hasMin)
   649  	b.False(hasMax)
   650  }
   651  
   652  func (b *ByteArrayWriterSuite) TestOmitDataPageStats() {
   653  	// prevent writing large stats in DataPageHeader
   654  	minLen := math.Pow10(7)
   655  	maxLen := math.Pow10(7)
   656  	b.SetupSchema(parquet.Repetitions.Required, 1)
   657  	colprops := parquet.DefaultColumnProperties()
   658  	colprops.StatsEnabled = false
   659  
   660  	writer := b.buildWriter(SmallSize, colprops, parquet.V1_0)
   661  	b.Values = make([]parquet.ByteArray, 1)
   662  	testutils.RandomByteArray(0, b.Values.([]parquet.ByteArray), b.Buffer, int(minLen), int(maxLen))
   663  	writer.(*file.ByteArrayColumnChunkWriter).WriteBatch(b.Values.([]parquet.ByteArray), nil, nil)
   664  	writer.Close()
   665  
   666  	b.NotPanics(func() { b.readColumn(compress.Codecs.Uncompressed) })
   667  }
   668  
   669  func (b *ByteArrayWriterSuite) TestLimitStats() {
   670  	minLen := 1024 * 4
   671  	maxLen := 1024 * 8
   672  	b.SetupSchema(parquet.Repetitions.Required, 1)
   673  	colprops := parquet.DefaultColumnProperties()
   674  	colprops.MaxStatsSize = int64(maxLen)
   675  
   676  	writer := b.buildWriter(SmallSize, colprops, parquet.V1_0).(*file.ByteArrayColumnChunkWriter)
   677  	b.Values = make([]parquet.ByteArray, SmallSize)
   678  	testutils.RandomByteArray(0, b.Values.([]parquet.ByteArray), b.Buffer, minLen, maxLen)
   679  	writer.WriteBatch(b.Values.([]parquet.ByteArray), nil, nil)
   680  	writer.Close()
   681  
   682  	b.True(b.metadataIsStatsSet())
   683  }
   684  
   685  func (b *ByteArrayWriterSuite) TestCheckDefaultStats() {
   686  	b.SetupSchema(parquet.Repetitions.Required, 1)
   687  	writer := b.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.V1_0)
   688  	b.GenerateData(SmallSize)
   689  	b.WriteBatchValues(writer, nil, nil)
   690  	writer.Close()
   691  
   692  	b.True(b.metadataIsStatsSet())
   693  }
   694  
   695  type BooleanValueWriterSuite struct {
   696  	PrimitiveWriterTestSuite
   697  }
   698  
   699  func (b *BooleanValueWriterSuite) TestAlternateBooleanValues() {
   700  	b.SetupSchema(parquet.Repetitions.Required, 1)
   701  	writer := b.buildWriter(SmallSize, parquet.DefaultColumnProperties(), parquet.V1_0).(*file.BooleanColumnChunkWriter)
   702  	for i := 0; i < SmallSize; i++ {
   703  		val := i%2 == 0
   704  		writer.WriteBatch([]bool{val}, nil, nil)
   705  	}
   706  	writer.Close()
   707  	b.readColumn(compress.Codecs.Uncompressed)
   708  	for i := 0; i < SmallSize; i++ {
   709  		b.Equal(i%2 == 0, b.ValuesOut.([]bool)[i])
   710  	}
   711  }