From 79b479957e8b0ae1c4dd45210f6b6903ff14f26c Mon Sep 17 00:00:00 2001 From: Matthew Topol Date: Tue, 17 Aug 2021 11:22:32 -0400 Subject: [PATCH 1/7] adding the parquet metadata package --- go/parquet/metadata/app_version.go | 172 ++ go/parquet/metadata/column_chunk.go | 385 +++ go/parquet/metadata/file.go | 480 ++++ go/parquet/metadata/metadata_test.go | 325 +++ go/parquet/metadata/row_group.go | 168 ++ go/parquet/metadata/stat_compare_test.go | 223 ++ go/parquet/metadata/statistics.go | 517 ++++ go/parquet/metadata/statistics_test.go | 190 ++ go/parquet/metadata/statistics_types.gen.go | 2292 +++++++++++++++++ .../metadata/statistics_types.gen.go.tmpl | 418 +++ 10 files changed, 5170 insertions(+) create mode 100644 go/parquet/metadata/app_version.go create mode 100644 go/parquet/metadata/column_chunk.go create mode 100644 go/parquet/metadata/file.go create mode 100644 go/parquet/metadata/metadata_test.go create mode 100644 go/parquet/metadata/row_group.go create mode 100644 go/parquet/metadata/stat_compare_test.go create mode 100644 go/parquet/metadata/statistics.go create mode 100644 go/parquet/metadata/statistics_test.go create mode 100644 go/parquet/metadata/statistics_types.gen.go create mode 100644 go/parquet/metadata/statistics_types.gen.go.tmpl diff --git a/go/parquet/metadata/app_version.go b/go/parquet/metadata/app_version.go new file mode 100644 index 00000000000..2a2d076d67a --- /dev/null +++ b/go/parquet/metadata/app_version.go @@ -0,0 +1,172 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "regexp" + "strconv" + "strings" + + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/schema" +) + +var ( + // Regular expression for the version format + // major . minor . patch unknown - prerelease.x + build info + // Eg: 1.5.0ab-cdh5.5.0+cd + versionRx = regexp.MustCompile(`^(\d+)\.(\d+)\.(\d+)([^-+]*)?(?:-([^+]*))?(?:\+(.*))?$`) + // Regular expression for the application format + // application_name version VERSION_FORMAT (build build_name) + // Eg: parquet-cpp version 1.5.0ab-xyz5.5.0+cd (build abcd) + applicationRx = regexp.MustCompile(`^(.*?)\s*(?:(version\s*(?:([^(]*?)\s*(?:\(\s*build\s*([^)]*?)\s*\))?)?)?)$`) + + // Parquet816FixedVersion is the version used for fixing PARQUET-816 + // that changed the padding calculations for dictionary headers on row groups. + Parquet816FixedVersion = NewAppVersionExplicit("parquet-mr", 1, 2, 9) + parquet251FixedVersion = NewAppVersionExplicit("parquet-mr", 1, 8, 0) + parquetCPPFixedStatsVersion = NewAppVersionExplicit("parquet-cpp", 1, 3, 0) + parquetMRFixedStatsVersion = NewAppVersionExplicit("parquet-mr", 1, 10, 0) +) + +// AppVersion represents a specific application version either read from +// or written to a parquet file. +type AppVersion struct { + App string + Build string + Version struct { + Major int + Minor int + Patch int + Unknown string + PreRelease string + BuildInfo string + } +} + +// NewAppVersionExplicit is a convenience function to construct a specific +// application version from the given app string and version +func NewAppVersionExplicit(app string, major, minor, patch int) *AppVersion { + v := &AppVersion{App: app} + v.Version.Major = major + v.Version.Minor = minor + v.Version.Patch = patch + return v +} + +// NewAppVersion parses a "created by" string such as "parquet-go 1.0.0". +// +// It also supports handling pre-releases and build info such as +// parquet-cpp version 1.5.0ab-xyz5.5.0+cd (build abcd) +func NewAppVersion(createdby string) *AppVersion { + v := &AppVersion{} + + var ver []string + + m := applicationRx.FindStringSubmatch(strings.ToLower(createdby)) + if len(m) >= 4 { + v.App = m[1] + v.Build = m[4] + ver = versionRx.FindStringSubmatch(m[3]) + } else { + v.App = "unknown" + } + + if len(ver) >= 7 { + v.Version.Major, _ = strconv.Atoi(ver[1]) + v.Version.Minor, _ = strconv.Atoi(ver[2]) + v.Version.Patch, _ = strconv.Atoi(ver[3]) + v.Version.Unknown = ver[4] + v.Version.PreRelease = ver[5] + v.Version.BuildInfo = ver[6] + } + return v +} + +// LessThan compares the app versions and returns true if this version +// is "less than" the passed version. +// +// If the apps don't match, this always returns false. Otherwise it compares +// the major versions first, then the minor versions, and finally the patch +// versions. +// +// Pre-release and build info are not considered. +func (v AppVersion) LessThan(other *AppVersion) bool { + switch { + case v.App != other.App: + return false + case v.Version.Major < other.Version.Major: + return true + case v.Version.Major > other.Version.Major: + return false + case v.Version.Minor < other.Version.Minor: + return true + case v.Version.Minor > other.Version.Minor: + return false + } + + return v.Version.Patch < other.Version.Patch +} + +// Equal only compares the Application and major/minor/patch versions. +// +// Pre-release and build info are not considered. +func (v AppVersion) Equal(other *AppVersion) bool { + return v.App == other.App && + v.Version.Major == other.Version.Major && + v.Version.Minor == other.Version.Minor && + v.Version.Patch == other.Version.Patch +} + +// HasCorrectStatistics checks whether or not the statistics are valid to be used +// based on the primitive type and the version since previous versions had issues with +// properly computing stats. +// +// Reference: parquet-cpp/src/parquet/metadata.cc +// +// PARQUET-686 has more discussion on statistics +func (v AppVersion) HasCorrectStatistics(coltype parquet.Type, stats EncodedStatistics, sort schema.SortOrder) bool { + // parquet-cpp version 1.3.0 and parquet-mr 1.10.0 onwards stats are computed correctly for all types + if (v.App == "parquet-cpp" && v.LessThan(parquetCPPFixedStatsVersion)) || + (v.App == "parquet-mr" && v.LessThan(parquetMRFixedStatsVersion)) { + // only SIGNED are valid unless max and min are the same (in which case the sort order doesn't matter) + var maxEqualsMin bool + if stats.HasMin && stats.HasMax { + maxEqualsMin = string(stats.Min) == string(stats.Max) + } + if sort != schema.SortSIGNED && !maxEqualsMin { + return false + } + + if coltype != parquet.Types.FixedLenByteArray && coltype != parquet.Types.ByteArray { + return true + } + } + // created_by is not populated, which could have been caused by + // parquet-mr during the same time as PARQUET-251, see PARQUET-297 + if v.App == "unknown" { + return true + } + + // unknown sort order has incorrect stats + if sort == schema.SortUNKNOWN { + return false + } + + // PARQUET-251 + return !v.LessThan(parquet251FixedVersion) +} diff --git a/go/parquet/metadata/column_chunk.go b/go/parquet/metadata/column_chunk.go new file mode 100644 index 00000000000..4f55355baca --- /dev/null +++ b/go/parquet/metadata/column_chunk.go @@ -0,0 +1,385 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "bytes" + "context" + "io" + "reflect" + + "github.com/apache/arrow/go/arrow/memory" + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/compress" + "github.com/apache/arrow/go/parquet/internal/encryption" + format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet" + "github.com/apache/arrow/go/parquet/internal/thrift" + "github.com/apache/arrow/go/parquet/schema" + "golang.org/x/xerrors" +) + +// PageEncodingStats is used for counting the number of pages of specific +// types with the given internal encoding. +type PageEncodingStats struct { + Encoding parquet.Encoding + PageType format.PageType +} + +func makeColumnStats(metadata *format.ColumnMetaData, descr *schema.Column, mem memory.Allocator) TypedStatistics { + if descr.ColumnOrder() == parquet.ColumnOrders.TypeDefinedOrder { + return NewStatisticsFromEncoded(descr, mem, metadata.Statistics.MinValue, metadata.Statistics.MaxValue, + metadata.NumValues-metadata.Statistics.GetNullCount(), + metadata.Statistics.GetNullCount(), metadata.Statistics.GetDistinctCount(), + metadata.Statistics.IsSetMaxValue() || metadata.Statistics.IsSetMinValue(), + metadata.Statistics.IsSetNullCount(), metadata.Statistics.IsSetDistinctCount()) + } + return NewStatisticsFromEncoded(descr, mem, metadata.Statistics.Min, metadata.Statistics.Max, + metadata.NumValues-metadata.Statistics.GetNullCount(), + metadata.Statistics.GetNullCount(), metadata.Statistics.GetDistinctCount(), + metadata.Statistics.IsSetMax() || metadata.Statistics.IsSetMin(), + metadata.Statistics.IsSetNullCount(), metadata.Statistics.IsSetDistinctCount()) +} + +// ColumnChunkMetaData is a proxy around format.ColumnChunkMetaData +// containing all of the information and metadata for a given column chunk +// and it's associated Column +type ColumnChunkMetaData struct { + column *format.ColumnChunk + columnMeta *format.ColumnMetaData + decryptedMeta format.ColumnMetaData + descr *schema.Column + writerVersion *AppVersion + encodings []parquet.Encoding + encodingStats []format.PageEncodingStats + possibleStats TypedStatistics + mem memory.Allocator +} + +// NewColumnChunkMetaData creates an instance of the metadata from a column chunk and descriptor +// +// this is primarily used internally or between the subpackages. ColumnChunkMetaDataBuilder should +// be used by consumers instead of using this directly. +func NewColumnChunkMetaData(column *format.ColumnChunk, descr *schema.Column, writerVersion *AppVersion, rowGroupOrdinal, columnOrdinal int16, fileDecryptor encryption.FileDecryptor) (*ColumnChunkMetaData, error) { + c := &ColumnChunkMetaData{ + column: column, + columnMeta: column.GetMetaData(), + descr: descr, + writerVersion: writerVersion, + mem: memory.DefaultAllocator, + } + if column.IsSetCryptoMetadata() { + ccmd := column.CryptoMetadata + + if ccmd.IsSetENCRYPTION_WITH_COLUMN_KEY() { + if fileDecryptor != nil && fileDecryptor.Properties() != nil { + // should decrypt metadata + path := parquet.ColumnPath(ccmd.ENCRYPTION_WITH_COLUMN_KEY.GetPathInSchema()) + keyMetadata := ccmd.ENCRYPTION_WITH_COLUMN_KEY.GetKeyMetadata() + aadColumnMetadata := encryption.CreateModuleAad(fileDecryptor.FileAad(), encryption.ColumnMetaModule, rowGroupOrdinal, columnOrdinal, -1) + decryptor := fileDecryptor.GetColumnMetaDecryptor(path.String(), string(keyMetadata), aadColumnMetadata) + thrift.DeserializeThrift(&c.decryptedMeta, decryptor.Decrypt(column.GetEncryptedColumnMetadata())) + c.columnMeta = &c.decryptedMeta + } else { + return nil, xerrors.New("cannot decrypt column metadata. file decryption not setup correctly") + } + } + } + for _, enc := range c.columnMeta.Encodings { + c.encodings = append(c.encodings, parquet.Encoding(enc)) + } + for _, enc := range c.columnMeta.EncodingStats { + c.encodingStats = append(c.encodingStats, *enc) + } + return c, nil +} + +// CryptoMetadata returns the cryptographic metadata for how this column was +// encrypted and how to decrypt it. +func (c *ColumnChunkMetaData) CryptoMetadata() *format.ColumnCryptoMetaData { + return c.column.GetCryptoMetadata() +} + +// FileOffset is the location in the file where the column data begins +func (c *ColumnChunkMetaData) FileOffset() int64 { return c.column.FileOffset } + +// FilePath gives the name of the parquet file if provided in the metadata +func (c *ColumnChunkMetaData) FilePath() string { return c.column.GetFilePath() } + +// Type is the physical storage type used in the parquet file for this column chunk. +func (c *ColumnChunkMetaData) Type() parquet.Type { return parquet.Type(c.columnMeta.Type) } + +// NumValues is the number of values stored in just this chunk +func (c *ColumnChunkMetaData) NumValues() int64 { return c.columnMeta.NumValues } + +// PathInSchema is the full path to this column from the root of the schema including +// any nested columns +func (c *ColumnChunkMetaData) PathInSchema() parquet.ColumnPath { + return c.columnMeta.GetPathInSchema() +} + +// Compression provides the type of compression used for this particular chunk. +func (c *ColumnChunkMetaData) Compression() compress.Compression { + return compress.Compression(c.columnMeta.Codec) +} + +// Encodings returns the list of different encodings used in this chunk +func (c *ColumnChunkMetaData) Encodings() []parquet.Encoding { return c.encodings } + +// EncodingStats connects the order of encodings based on the list of pages and types +func (c *ColumnChunkMetaData) EncodingStats() []PageEncodingStats { + ret := make([]PageEncodingStats, len(c.encodingStats)) + for idx := range ret { + ret[idx].Encoding = parquet.Encoding(c.encodingStats[idx].Encoding) + ret[idx].PageType = c.encodingStats[idx].PageType + } + return ret +} + +// HasDictionaryPage returns true if there is a dictionary page offset set in +// this metadata. +func (c *ColumnChunkMetaData) HasDictionaryPage() bool { + return c.columnMeta.IsSetDictionaryPageOffset() +} + +// DictionaryPageOffset returns the location in the file where the dictionary page starts +func (c *ColumnChunkMetaData) DictionaryPageOffset() int64 { + return c.columnMeta.GetDictionaryPageOffset() +} + +// DataPageOffset returns the location in the file where the data pages begin for this column +func (c *ColumnChunkMetaData) DataPageOffset() int64 { return c.columnMeta.GetDataPageOffset() } + +// HasIndexPage returns true if the offset for the index page is set in the metadata +func (c *ColumnChunkMetaData) HasIndexPage() bool { return c.columnMeta.IsSetIndexPageOffset() } + +// IndexPageOffset is the location in the file where the index page starts. +func (c *ColumnChunkMetaData) IndexPageOffset() int64 { return c.columnMeta.GetIndexPageOffset() } + +// TotalCompressedSize will be equal to TotalUncompressedSize if the data is not compressed. +// Otherwise this will be the size of the actual data in the file. +func (c *ColumnChunkMetaData) TotalCompressedSize() int64 { + return c.columnMeta.GetTotalCompressedSize() +} + +// TotalUncompressedSize is the total size of the raw data after uncompressing the chunk +func (c *ColumnChunkMetaData) TotalUncompressedSize() int64 { + return c.columnMeta.GetTotalUncompressedSize() +} + +// StatsSet returns true only if there are statistics set in the metadata and the column +// descriptor has a sort order that is not SortUnknown +// +// It also checks the writer version to ensure that it was not written by a version +// of parquet which is known to have incorrect stat computations. +func (c *ColumnChunkMetaData) StatsSet() (bool, error) { + if !c.columnMeta.IsSetStatistics() || c.descr.SortOrder() == schema.SortUNKNOWN { + return false, nil + } + + if c.possibleStats == nil { + c.possibleStats = makeColumnStats(c.columnMeta, c.descr, c.mem) + } + + encoded, err := c.possibleStats.Encode() + if err != nil { + return false, err + } + return c.writerVersion.HasCorrectStatistics(c.Type(), encoded, c.descr.SortOrder()), nil +} + +func (c *ColumnChunkMetaData) Equals(other *ColumnChunkMetaData) bool { + return reflect.DeepEqual(c.columnMeta, other.columnMeta) +} + +// Statistics can return nil if there are no stats in this metadata +func (c *ColumnChunkMetaData) Statistics() (TypedStatistics, error) { + ok, err := c.StatsSet() + if err != nil { + return nil, err + } + + if ok { + return c.possibleStats, nil + } + return nil, nil +} + +// ColumnChunkMetaDataBuilder is used during writing to construct metadata +// for a given column chunk while writing, providing a proxy around constructing +// the actual thrift object. +type ColumnChunkMetaDataBuilder struct { + chunk *format.ColumnChunk + props *parquet.WriterProperties + column *schema.Column + + compressedSize int64 +} + +func NewColumnChunkMetaDataBuilder(props *parquet.WriterProperties, column *schema.Column) *ColumnChunkMetaDataBuilder { + return NewColumnChunkMetaDataBuilderWithContents(props, column, format.NewColumnChunk()) +} + +// NewColumnChunkMetaDataBuilderWithContents will construct a builder and start it with the provided +// column chunk information rather than with an empty column chunk. +func NewColumnChunkMetaDataBuilderWithContents(props *parquet.WriterProperties, column *schema.Column, chunk *format.ColumnChunk) *ColumnChunkMetaDataBuilder { + b := &ColumnChunkMetaDataBuilder{ + props: props, + column: column, + chunk: chunk, + } + b.init(chunk) + return b +} + +// Contents returns the underlying thrift ColumnChunk object so that it can be used +// for constructing or duplicating column metadata +func (c *ColumnChunkMetaDataBuilder) Contents() *format.ColumnChunk { return c.chunk } + +func (c *ColumnChunkMetaDataBuilder) init(chunk *format.ColumnChunk) { + c.chunk = chunk + if !c.chunk.IsSetMetaData() { + c.chunk.MetaData = format.NewColumnMetaData() + } + c.chunk.MetaData.Type = format.Type(c.column.PhysicalType()) + c.chunk.MetaData.PathInSchema = schema.ColumnPathFromNode(c.column.SchemaNode()) + c.chunk.MetaData.Codec = format.CompressionCodec(c.props.CompressionFor(c.column.Path())) +} + +func (c *ColumnChunkMetaDataBuilder) SetFilePath(val string) { + c.chunk.FilePath = &val +} + +// Descr returns the associated column descriptor for this column chunk +func (c *ColumnChunkMetaDataBuilder) Descr() *schema.Column { return c.column } + +func (c *ColumnChunkMetaDataBuilder) TotalCompressedSize() int64 { + // if this column is encrypted, after Finish is called, the MetaData + // field is set to nil and we store the compressed size so return that + if c.chunk.MetaData == nil { + return c.compressedSize + } + return c.chunk.MetaData.GetTotalCompressedSize() +} + +func (c *ColumnChunkMetaDataBuilder) SetStats(val EncodedStatistics) { + c.chunk.MetaData.Statistics = val.ToThrift() +} + +// Finish finalizes the metadata with the given offsets, +// flushes any compression that needs to be done, and performs +// any encryption if an encryptor is provided. +func (c *ColumnChunkMetaDataBuilder) Finish(nvalues, dictPageOffset, indexPageOffset, dataPageOffset, compressed, uncompressed int64, hasDict, dictFallback bool, dictEncodingStats, dataEncodingStats map[parquet.Encoding]int32, metaEncryptor encryption.Encryptor) error { + if dictPageOffset > 0 { + c.chunk.MetaData.DictionaryPageOffset = &dictPageOffset + c.chunk.FileOffset = dictPageOffset + compressed + } else { + c.chunk.FileOffset = dataPageOffset + compressed + } + + c.chunk.MetaData.NumValues = nvalues + if indexPageOffset >= 0 { + c.chunk.MetaData.IndexPageOffset = &indexPageOffset + } + + c.chunk.MetaData.DataPageOffset = dataPageOffset + c.chunk.MetaData.TotalUncompressedSize = uncompressed + c.chunk.MetaData.TotalCompressedSize = compressed + + thriftEncodings := make([]format.Encoding, 0) + if hasDict { + thriftEncodings = append(thriftEncodings, format.Encoding(c.props.DictionaryIndexEncoding())) + if c.props.Version() == parquet.V1 { + thriftEncodings = append(thriftEncodings, format.Encoding_PLAIN) + } else { + thriftEncodings = append(thriftEncodings, format.Encoding(c.props.DictionaryPageEncoding())) + } + } else { // no dictionary + thriftEncodings = append(thriftEncodings, format.Encoding(c.props.EncodingFor(c.column.Path()))) + } + + thriftEncodings = append(thriftEncodings, format.Encoding(parquet.Encodings.RLE)) + // Only PLAIN encoding is supported for fallback in V1 + // TODO(zeroshade): Use user specified encoding for V2 + if dictFallback { + thriftEncodings = append(thriftEncodings, format.Encoding_PLAIN) + } + c.chunk.MetaData.Encodings = thriftEncodings + + thriftEncodingStats := make([]*format.PageEncodingStats, 0, len(dictEncodingStats)+len(dataEncodingStats)) + for k, v := range dictEncodingStats { + thriftEncodingStats = append(thriftEncodingStats, &format.PageEncodingStats{ + PageType: format.PageType_DICTIONARY_PAGE, + Encoding: format.Encoding(k), + Count: v, + }) + } + for k, v := range dataEncodingStats { + thriftEncodingStats = append(thriftEncodingStats, &format.PageEncodingStats{ + PageType: format.PageType_DATA_PAGE, + Encoding: format.Encoding(k), + Count: v, + }) + } + c.chunk.MetaData.EncodingStats = thriftEncodingStats + + encryptProps := c.props.ColumnEncryptionProperties(c.column.Path()) + if encryptProps != nil && encryptProps.IsEncrypted() { + ccmd := format.NewColumnCryptoMetaData() + if encryptProps.IsEncryptedWithFooterKey() { + ccmd.ENCRYPTION_WITH_FOOTER_KEY = format.NewEncryptionWithFooterKey() + } else { + ccmd.ENCRYPTION_WITH_COLUMN_KEY = &format.EncryptionWithColumnKey{ + KeyMetadata: []byte(encryptProps.KeyMetadata()), + PathInSchema: c.column.ColumnPath(), + } + } + c.chunk.CryptoMetadata = ccmd + + encryptedFooter := c.props.FileEncryptionProperties().EncryptedFooter() + encryptMetadata := !encryptedFooter || !encryptProps.IsEncryptedWithFooterKey() + if encryptMetadata { + // Serialize and encrypt ColumnMetadata separately + // Thrift-serialize the ColumnMetaData structure, + // encrypt it with the column key, and write to encrypted_column_metadata + serializer := thrift.NewThriftSerializer() + data, err := serializer.Write(context.Background(), c.chunk.MetaData) + if err != nil { + return err + } + var buf bytes.Buffer + metaEncryptor.Encrypt(&buf, data) + c.chunk.EncryptedColumnMetadata = buf.Bytes() + + if encryptedFooter { + c.compressedSize = c.chunk.MetaData.GetTotalCompressedSize() + c.chunk.MetaData = nil + } else { + // Keep redacted metadata version for old readers + c.chunk.MetaData.Statistics = nil + c.chunk.MetaData.EncodingStats = nil + } + } + } + return nil +} + +// WriteTo will always return 0 as the int64 since the thrift writer library +// does not return the number of bytes written, we only use the signature +// of (int64, error) in order to match the standard WriteTo interfaces. +func (c *ColumnChunkMetaDataBuilder) WriteTo(w io.Writer) (int64, error) { + return 0, thrift.SerializeThriftStream(c.chunk, w) +} diff --git a/go/parquet/metadata/file.go b/go/parquet/metadata/file.go new file mode 100644 index 00000000000..8c57ed5f09a --- /dev/null +++ b/go/parquet/metadata/file.go @@ -0,0 +1,480 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "bytes" + "context" + "io" + "reflect" + + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/compress" + "github.com/apache/arrow/go/parquet/internal/encryption" + format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet" + "github.com/apache/arrow/go/parquet/internal/thrift" + "github.com/apache/arrow/go/parquet/schema" + "golang.org/x/xerrors" +) + +// DefaultCompressionType is used unless a different compression is specified +// in the properties +var DefaultCompressionType = compress.Codecs.Uncompressed + +// FileMetaDataBuilder is a proxy for more easily constructing file metadata +// particularly used when writing a file out. +type FileMetaDataBuilder struct { + metadata *format.FileMetaData + props *parquet.WriterProperties + schema *schema.Schema + rowGroups []*format.RowGroup + currentRgBldr *RowGroupMetaDataBuilder + kvmeta KeyValueMetadata + cryptoMetadata *format.FileCryptoMetaData +} + +// NewFileMetadataBuilder will use the default writer properties if nil is passed for +// the writer properties and nil is allowable for the key value metadata. +func NewFileMetadataBuilder(schema *schema.Schema, props *parquet.WriterProperties, kvmeta KeyValueMetadata) *FileMetaDataBuilder { + var crypto *format.FileCryptoMetaData + if props.FileEncryptionProperties() != nil && props.FileEncryptionProperties().EncryptedFooter() { + crypto = format.NewFileCryptoMetaData() + } + return &FileMetaDataBuilder{ + metadata: format.NewFileMetaData(), + props: props, + schema: schema, + kvmeta: kvmeta, + cryptoMetadata: crypto, + } +} + +// GetFileCryptoMetaData returns the cryptographic information for encrypting/ +// decrypting the file. +func (f *FileMetaDataBuilder) GetFileCryptoMetaData() *FileCryptoMetadata { + if f.cryptoMetadata == nil { + return nil + } + + props := f.props.FileEncryptionProperties() + f.cryptoMetadata.EncryptionAlgorithm = props.Algorithm().ToThrift() + keyMetadata := props.FooterKeyMetadata() + if keyMetadata != "" { + f.cryptoMetadata.KeyMetadata = []byte(keyMetadata) + } + + return &FileCryptoMetadata{f.cryptoMetadata, 0} +} + +// AppendRowGroup adds a rowgroup to the list and returns a builder +// for that row group +func (f *FileMetaDataBuilder) AppendRowGroup() *RowGroupMetaDataBuilder { + if f.rowGroups == nil { + f.rowGroups = make([]*format.RowGroup, 0, 1) + } + + rg := format.NewRowGroup() + f.rowGroups = append(f.rowGroups, rg) + f.currentRgBldr = NewRowGroupMetaDataBuilder(f.props, f.schema, rg) + return f.currentRgBldr +} + +// Finish will finalize the metadata of the number of rows, row groups, +// version etc. This will clear out this filemetadatabuilder so it can +// be re-used +func (f *FileMetaDataBuilder) Finish() (*FileMetaData, error) { + totalRows := int64(0) + for _, rg := range f.rowGroups { + totalRows += rg.NumRows + } + f.metadata.NumRows = totalRows + f.metadata.RowGroups = f.rowGroups + switch f.props.Version() { + case parquet.V1: + f.metadata.Version = 1 + case parquet.V2: + f.metadata.Version = 2 + default: + f.metadata.Version = 0 + } + createdBy := f.props.CreatedBy() + f.metadata.CreatedBy = &createdBy + + // Users cannot set the `ColumnOrder` since we donot not have user defined sort order + // in the spec yet. + // We always default to `TYPE_DEFINED_ORDER`. We can expose it in + // the API once we have user defined sort orders in the Parquet format. + // TypeDefinedOrder implies choose SortOrder based on ConvertedType/PhysicalType + typeDefined := format.NewTypeDefinedOrder() + colOrder := &format.ColumnOrder{TYPE_ORDER: typeDefined} + f.metadata.ColumnOrders = make([]*format.ColumnOrder, f.schema.NumColumns()) + for idx := range f.metadata.ColumnOrders { + f.metadata.ColumnOrders[idx] = colOrder + } + + fileEncProps := f.props.FileEncryptionProperties() + if fileEncProps != nil && !fileEncProps.EncryptedFooter() { + var signingAlgo parquet.Algorithm + algo := fileEncProps.Algorithm() + signingAlgo.Aad.AadFileUnique = algo.Aad.AadFileUnique + signingAlgo.Aad.SupplyAadPrefix = algo.Aad.SupplyAadPrefix + if !algo.Aad.SupplyAadPrefix { + signingAlgo.Aad.AadPrefix = algo.Aad.AadPrefix + } + signingAlgo.Algo = parquet.AesGcm + f.metadata.EncryptionAlgorithm = signingAlgo.ToThrift() + footerSigningMetadata := f.props.FileEncryptionProperties().FooterKeyMetadata() + if footerSigningMetadata != "" { + f.metadata.FooterSigningKeyMetadata = []byte(footerSigningMetadata) + } + } + + f.metadata.Schema = schema.ToThrift(f.schema.Root()) + f.metadata.KeyValueMetadata = f.kvmeta + + out := &FileMetaData{ + FileMetaData: f.metadata, + version: NewAppVersion(f.metadata.GetCreatedBy()), + } + if err := out.initSchema(); err != nil { + return nil, err + } + out.initColumnOrders() + + f.metadata = format.NewFileMetaData() + f.rowGroups = nil + return out, nil +} + +// KeyValueMetadata is an alias for a slice of thrift keyvalue pairs +type KeyValueMetadata []*format.KeyValue + +// NewKeyValueMetadata is equivalent to make(KeyValueMetadata, 0) +func NewKeyValueMetadata() KeyValueMetadata { + return make(KeyValueMetadata, 0) +} + +func (k *KeyValueMetadata) Append(key, value string) { + *k = append(*k, &format.KeyValue{Key: key, Value: &value}) +} + +func (k KeyValueMetadata) Len() int { return len(k) } + +// Equals compares all of the metadata keys and values to check they are equal +func (k KeyValueMetadata) Equals(other KeyValueMetadata) bool { + return reflect.DeepEqual(k, other) +} + +func (k KeyValueMetadata) Keys() (ret []string) { + ret = make([]string, len(k)) + for idx, v := range k { + ret[idx] = v.GetKey() + } + return +} + +func (k KeyValueMetadata) Values() (ret []string) { + ret = make([]string, len(k)) + for idx, v := range k { + ret[idx] = v.GetValue() + } + return +} + +func (k KeyValueMetadata) FindValue(key string) *string { + for _, v := range k { + if v.Key == key { + return v.Value + } + } + return nil +} + +// FileMetaData is a proxy around the underlying thrift FileMetaData object +// to make it easier to use and interact with. +type FileMetaData struct { + *format.FileMetaData + version *AppVersion + Schema *schema.Schema + FileDecryptor encryption.FileDecryptor + metadataLen int +} + +// NewFileMetaData takes in the raw bytes of the serialized metadata to deserialize +// and will attempt to decrypt the footer if a decryptor is provided. +func NewFileMetaData(data []byte, fileDecryptor encryption.FileDecryptor) (*FileMetaData, error) { + meta := format.NewFileMetaData() + if fileDecryptor != nil { + footerDecryptor := fileDecryptor.GetFooterDecryptor() + data = footerDecryptor.Decrypt(data) + } + + remain, err := thrift.DeserializeThrift(meta, data) + if err != nil { + return nil, err + } + + f := &FileMetaData{ + FileMetaData: meta, + version: NewAppVersion(meta.GetCreatedBy()), + metadataLen: len(data) - int(remain), + FileDecryptor: fileDecryptor, + } + + f.initSchema() + f.initColumnOrders() + + // init keyvalue metadata + + return f, nil +} + +// Size is the length of the raw serialized metadata bytes in the footer +func (f *FileMetaData) Size() int { return f.metadataLen } + +// NumSchemaElements is the length of the flattened schema list in the thrift +func (f *FileMetaData) NumSchemaElements() int { + return len(f.FileMetaData.Schema) +} + +// RowGroup provides the metadata for the (0-based) index of the row group +func (f *FileMetaData) RowGroup(i int) *RowGroupMetaData { + return &RowGroupMetaData{ + f.RowGroups[i], f.Schema, f.version, f.FileDecryptor, + } +} + +func (f *FileMetaData) Serialize(ctx context.Context) ([]byte, error) { + return thrift.NewThriftSerializer().Write(ctx, f.FileMetaData) +} + +func (f *FileMetaData) SerializeString(ctx context.Context) (string, error) { + return thrift.NewThriftSerializer().WriteString(ctx, f.FileMetaData) +} + +// EncryptionAlgorithm constructs the algorithm object from the thrift +// information or returns an empty instance if it was not set. +func (f *FileMetaData) EncryptionAlgorithm() parquet.Algorithm { + if f.IsSetEncryptionAlgorithm() { + return parquet.AlgorithmFromThrift(f.GetEncryptionAlgorithm()) + } + return parquet.Algorithm{} +} + +func (f *FileMetaData) initSchema() error { + root, err := schema.FromParquet(f.FileMetaData.Schema) + if err != nil { + return err + } + f.Schema = schema.NewSchema(root.(*schema.GroupNode)) + return nil +} + +func (f *FileMetaData) initColumnOrders() { + orders := make([]parquet.ColumnOrder, 0, f.Schema.NumColumns()) + if f.IsSetColumnOrders() { + for _, o := range f.GetColumnOrders() { + if o.IsSetTYPE_ORDER() { + orders = append(orders, parquet.ColumnOrders.TypeDefinedOrder) + } else { + orders = append(orders, parquet.ColumnOrders.Undefined) + } + } + } else { + orders = orders[:f.Schema.NumColumns()] + orders[0] = parquet.ColumnOrders.Undefined + for i := 1; i < len(orders); i *= 2 { + copy(orders[i:], orders[:i]) + } + } + f.Schema.UpdateColumnOrders(orders) +} + +// WriterVersion returns the constructed application version from the +// created by string +func (f *FileMetaData) WriterVersion() *AppVersion { + if f.version == nil { + f.version = NewAppVersion(f.GetCreatedBy()) + } + return f.version +} + +// SetFilePath will set the file path into all of the columns in each row group. +func (f *FileMetaData) SetFilePath(path string) { + for _, rg := range f.RowGroups { + for _, chunk := range rg.Columns { + chunk.FilePath = &path + } + } +} + +// AppendRowGroups will add all of the rowgroup metadata from other to the +// current file metadata +func (f *FileMetaData) AppendRowGroups(other *FileMetaData) error { + if !f.Schema.Equals(other.Schema) { + return xerrors.New("parquet/FileMetaData: AppendRowGroups requires equal schemas") + } + + f.RowGroups = append(f.RowGroups, other.GetRowGroups()...) + for _, rg := range other.GetRowGroups() { + f.NumRows += rg.NumRows + } + return nil +} + +// Subset will construct a new FileMetaData object containing only the requested +// row groups by index +func (f *FileMetaData) Subset(rowGroups []int) (*FileMetaData, error) { + for _, i := range rowGroups { + if i < len(f.RowGroups) { + continue + } + return nil, xerrors.Errorf("parquet: this file only has %d row groups, but requested a subset including row group: %d", len(f.RowGroups), i) + } + + out := &FileMetaData{ + &format.FileMetaData{ + Schema: f.FileMetaData.Schema, + CreatedBy: f.CreatedBy, + ColumnOrders: f.GetColumnOrders(), + EncryptionAlgorithm: f.FileMetaData.EncryptionAlgorithm, + FooterSigningKeyMetadata: f.FooterSigningKeyMetadata, + Version: f.Version, + KeyValueMetadata: f.KeyValueMetadata(), + }, + f.version, + f.Schema, + f.FileDecryptor, + 0, + } + + out.RowGroups = make([]*format.RowGroup, 0, len(rowGroups)) + for _, selected := range rowGroups { + out.RowGroups = append(out.RowGroups, f.RowGroups[selected]) + out.NumRows += f.RowGroups[selected].GetNumRows() + } + + return out, nil +} + +func (f *FileMetaData) Equals(other *FileMetaData) bool { + return reflect.DeepEqual(f.FileMetaData, other.FileMetaData) +} + +func (f *FileMetaData) KeyValueMetadata() KeyValueMetadata { + return f.GetKeyValueMetadata() +} + +// VerifySignature constructs a cryptographic signature using the FileDecryptor +// of the footer and then verifies it's integrity. +// +// Panics if f.FileDecryptor is nil +func (f *FileMetaData) VerifySignature(signature []byte) bool { + if f.FileDecryptor == nil { + panic("decryption not set propertly, cannot verify signature") + } + + serializer := thrift.NewThriftSerializer() + data, _ := serializer.Write(context.Background(), f.FileMetaData) + nonce := signature[:encryption.NonceLength] + tag := signature[encryption.NonceLength : encryption.NonceLength+encryption.GcmTagLength] + + key := f.FileDecryptor.GetFooterKey() + aad := encryption.CreateFooterAad(f.FileDecryptor.FileAad()) + + enc := encryption.NewAesEncryptor(f.FileDecryptor.Algorithm(), true) + var buf bytes.Buffer + buf.Grow(enc.CiphertextSizeDelta() + len(data)) + encryptedLen := enc.SignedFooterEncrypt(&buf, data, []byte(key), []byte(aad), nonce) + return bytes.Equal(buf.Bytes()[encryptedLen-encryption.GcmTagLength:], tag) +} + +// WriteTo will serialize and write out this file metadata, encrypting it if +// appropriate. +// +// If it is an encrypted file with a plaintext footer, then we will write the +// signature with the unencrypted footer. +func (f *FileMetaData) WriteTo(w io.Writer, encryptor encryption.Encryptor) (int64, error) { + serializer := thrift.NewThriftSerializer() + // only in encrypted files with plaintext footers, the encryption algorithm is set in the footer + if f.IsSetEncryptionAlgorithm() { + data, err := serializer.Write(context.Background(), f.FileMetaData) + if err != nil { + return 0, err + } + + // encrypt the footer key + var buf bytes.Buffer + buf.Grow(encryptor.CiphertextSizeDelta() + len(data)) + encryptedLen := encryptor.Encrypt(&buf, data) + + wrote := 0 + n := 0 + // write unencrypted footer + if n, err = w.Write(data); err != nil { + return int64(n), err + } + wrote += n + // write signature (nonce and tag) + buf.Next(4) + if n, err = w.Write(buf.Next(encryption.NonceLength)); err != nil { + return int64(wrote + n), err + } + wrote += n + buf.Next(encryptedLen - 4 - encryption.NonceLength - encryption.GcmTagLength) + n, err = w.Write(buf.Next(encryption.GcmTagLength)) + return int64(wrote + n), err + } + n, err := serializer.Serialize(f.FileMetaData, w, encryptor) + return int64(n), err +} + +// FileCryptoMetadata is a proxy for the thrift fileCryptoMetadata object +type FileCryptoMetadata struct { + metadata *format.FileCryptoMetaData + cryptoMetadataLen uint32 +} + +// NewFileCryptoMetaData takes in the raw serialized bytes to deserialize +// storing the number of bytes that were actually deserialized. +func NewFileCryptoMetaData(metadata []byte) (ret FileCryptoMetadata, err error) { + ret.metadata = format.NewFileCryptoMetaData() + var remain uint64 + remain, err = thrift.DeserializeThrift(ret.metadata, metadata) + ret.cryptoMetadataLen = uint32(uint64(len(metadata)) - remain) + return +} + +// WriteTo writes out the serialized crypto metadata to w +func (fc FileCryptoMetadata) WriteTo(w io.Writer) (int64, error) { + serializer := thrift.NewThriftSerializer() + n, err := serializer.Serialize(fc.metadata, w, nil) + return int64(n), err +} + +// Len is the number of bytes that were deserialized to create this object +func (fc FileCryptoMetadata) Len() int { return int(fc.cryptoMetadataLen) } + +func (fc FileCryptoMetadata) KeyMetadata() []byte { + return fc.metadata.KeyMetadata +} + +// EncryptionAlgorithm constructs the object from the thrift instance of +// the encryption algorithm +func (fc FileCryptoMetadata) EncryptionAlgorithm() parquet.Algorithm { + return parquet.AlgorithmFromThrift(fc.metadata.GetEncryptionAlgorithm()) +} diff --git a/go/parquet/metadata/metadata_test.go b/go/parquet/metadata/metadata_test.go new file mode 100644 index 00000000000..8e1625b590a --- /dev/null +++ b/go/parquet/metadata/metadata_test.go @@ -0,0 +1,325 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata_test + +import ( + "context" + "testing" + "unsafe" + + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/metadata" + "github.com/apache/arrow/go/parquet/schema" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func generateTableMetaData(schema *schema.Schema, props *parquet.WriterProperties, nrows int64, statsInt, statsFloat metadata.EncodedStatistics) (*metadata.FileMetaData, error) { + fbuilder := metadata.NewFileMetadataBuilder(schema, props, nil) + rg1Builder := fbuilder.AppendRowGroup() + // metadata + // row group 1 + col1Builder := rg1Builder.NextColumnChunk() + col2Builder := rg1Builder.NextColumnChunk() + // column metadata + dictEncodingStats := map[parquet.Encoding]int32{parquet.Encodings.RLEDict: 1} + dataEncodingStats := map[parquet.Encoding]int32{parquet.Encodings.Plain: 1, parquet.Encodings.RLE: 1} + statsInt.Signed = true + col1Builder.SetStats(statsInt) + statsFloat.Signed = true + col2Builder.SetStats(statsFloat) + + col1Builder.Finish(nrows/2, 4, 0, 10, 512, 600, true, false, dictEncodingStats, dataEncodingStats, nil) + col2Builder.Finish(nrows/2, 24, 0, 30, 512, 600, true, false, dictEncodingStats, dataEncodingStats, nil) + + rg1Builder.SetNumRows(nrows / 2) + rg1Builder.Finish(1024, -1) + + // rowgroup2 metadata + rg2Builder := fbuilder.AppendRowGroup() + col1Builder = rg2Builder.NextColumnChunk() + col2Builder = rg2Builder.NextColumnChunk() + // column metadata + col1Builder.SetStats(statsInt) + col2Builder.SetStats(statsFloat) + col1Builder.Finish(nrows/2, 6, 0, 10, 512, 600, true, false, dictEncodingStats, dataEncodingStats, nil) + col2Builder.Finish(nrows/2, 16, 0, 26, 512, 600, true, false, dictEncodingStats, dataEncodingStats, nil) + + rg2Builder.SetNumRows(nrows / 2) + rg2Builder.Finish(1024, -1) + + return fbuilder.Finish() +} + +func assertStatsSet(t *testing.T, m *metadata.ColumnChunkMetaData) { + ok, err := m.StatsSet() + assert.NoError(t, err) + assert.True(t, ok) +} + +func assertStats(t *testing.T, m *metadata.ColumnChunkMetaData) metadata.TypedStatistics { + s, err := m.Statistics() + assert.NoError(t, err) + assert.NotNil(t, s) + return s +} + +func TestBuildAccess(t *testing.T) { + props := parquet.NewWriterProperties(parquet.WithVersion(parquet.V2)) + + fields := schema.FieldList{ + schema.NewInt32Node("int_col", parquet.Repetitions.Required, -1), + schema.NewFloat32Node("float_col", parquet.Repetitions.Required, -1), + } + root, err := schema.NewGroupNode("schema", parquet.Repetitions.Repeated, fields, -1) + require.NoError(t, err) + schema := schema.NewSchema(root) + + var ( + nrows int64 = 1000 + intMin int32 = 100 + intMax int32 = 200 + floatMin float32 = 100.100 + floatMax float32 = 200.200 + statsInt metadata.EncodedStatistics + statsFloat metadata.EncodedStatistics + ) + + statsInt.SetNullCount(0). + SetDistinctCount(nrows). + SetMin((*(*[4]byte)(unsafe.Pointer(&intMin)))[:]). + SetMax((*(*[4]byte)(unsafe.Pointer(&intMax)))[:]) + + statsFloat.SetNullCount(0). + SetDistinctCount(nrows). + SetMin((*(*[4]byte)(unsafe.Pointer(&floatMin)))[:]). + SetMax((*(*[4]byte)(unsafe.Pointer(&floatMax)))[:]) + + faccessor, err := generateTableMetaData(schema, props, nrows, statsInt, statsFloat) + require.NoError(t, err) + serialized, err := faccessor.SerializeString(context.Background()) + assert.NoError(t, err) + faccessorCopy, err := metadata.NewFileMetaData([]byte(serialized), nil) + assert.NoError(t, err) + + for _, accessor := range []*metadata.FileMetaData{faccessor, faccessorCopy} { + assert.Equal(t, nrows, accessor.NumRows) + assert.Len(t, accessor.RowGroups, 2) + assert.EqualValues(t, parquet.V2, accessor.Version) + assert.Equal(t, parquet.DefaultCreatedBy, accessor.GetCreatedBy()) + assert.Equal(t, 3, accessor.NumSchemaElements()) + + rg1Access := accessor.RowGroup(0) + assert.Equal(t, 2, rg1Access.NumColumns()) + assert.Equal(t, nrows/2, rg1Access.NumRows()) + assert.Equal(t, int64(1024), rg1Access.TotalByteSize()) + + rg1Col1, err := rg1Access.ColumnChunk(0) + assert.NoError(t, err) + rg1Col2, err := rg1Access.ColumnChunk(1) + assert.NoError(t, err) + assertStatsSet(t, rg1Col1) + assertStatsSet(t, rg1Col2) + assert.Equal(t, statsInt.Min, assertStats(t, rg1Col1).EncodeMin()) + assert.Equal(t, statsInt.Max, assertStats(t, rg1Col1).EncodeMax()) + assert.Equal(t, statsFloat.Min, assertStats(t, rg1Col2).EncodeMin()) + assert.Equal(t, statsFloat.Max, assertStats(t, rg1Col2).EncodeMax()) + assert.Zero(t, assertStats(t, rg1Col1).NullCount()) + assert.Zero(t, assertStats(t, rg1Col2).NullCount()) + assert.Equal(t, nrows, assertStats(t, rg1Col1).DistinctCount()) + assert.Equal(t, nrows, assertStats(t, rg1Col2).DistinctCount()) + assert.Equal(t, metadata.DefaultCompressionType, rg1Col1.Compression()) + assert.Equal(t, metadata.DefaultCompressionType, rg1Col2.Compression()) + assert.Equal(t, nrows/2, rg1Col1.NumValues()) + assert.Equal(t, nrows/2, rg1Col2.NumValues()) + assert.Len(t, rg1Col1.Encodings(), 3) + assert.Len(t, rg1Col2.Encodings(), 3) + assert.EqualValues(t, 512, rg1Col1.TotalCompressedSize()) + assert.EqualValues(t, 512, rg1Col2.TotalCompressedSize()) + assert.EqualValues(t, 600, rg1Col1.TotalUncompressedSize()) + assert.EqualValues(t, 600, rg1Col2.TotalUncompressedSize()) + assert.EqualValues(t, 4, rg1Col1.DictionaryPageOffset()) + assert.EqualValues(t, 24, rg1Col2.DictionaryPageOffset()) + assert.EqualValues(t, 10, rg1Col1.DataPageOffset()) + assert.EqualValues(t, 30, rg1Col2.DataPageOffset()) + assert.Len(t, rg1Col1.EncodingStats(), 3) + assert.Len(t, rg1Col2.EncodingStats(), 3) + + rg2Access := accessor.RowGroup(1) + assert.Equal(t, 2, rg2Access.NumColumns()) + assert.Equal(t, nrows/2, rg2Access.NumRows()) + assert.EqualValues(t, 1024, rg2Access.TotalByteSize()) + + rg2Col1, err := rg2Access.ColumnChunk(0) + assert.NoError(t, err) + rg2Col2, err := rg2Access.ColumnChunk(1) + assert.NoError(t, err) + assertStatsSet(t, rg1Col1) + assertStatsSet(t, rg1Col2) + assert.Equal(t, statsInt.Min, assertStats(t, rg1Col1).EncodeMin()) + assert.Equal(t, statsInt.Max, assertStats(t, rg1Col1).EncodeMax()) + assert.Equal(t, statsFloat.Min, assertStats(t, rg1Col2).EncodeMin()) + assert.Equal(t, statsFloat.Max, assertStats(t, rg1Col2).EncodeMax()) + assert.Zero(t, assertStats(t, rg1Col1).NullCount()) + assert.Zero(t, assertStats(t, rg1Col2).NullCount()) + assert.Equal(t, nrows, assertStats(t, rg1Col1).DistinctCount()) + assert.Equal(t, nrows, assertStats(t, rg1Col2).DistinctCount()) + assert.Equal(t, metadata.DefaultCompressionType, rg2Col1.Compression()) + assert.Equal(t, metadata.DefaultCompressionType, rg2Col2.Compression()) + assert.Equal(t, nrows/2, rg2Col1.NumValues()) + assert.Equal(t, nrows/2, rg2Col2.NumValues()) + assert.Len(t, rg2Col1.Encodings(), 3) + assert.Len(t, rg2Col2.Encodings(), 3) + assert.EqualValues(t, 512, rg2Col1.TotalCompressedSize()) + assert.EqualValues(t, 512, rg2Col2.TotalCompressedSize()) + assert.EqualValues(t, 600, rg2Col1.TotalUncompressedSize()) + assert.EqualValues(t, 600, rg2Col2.TotalUncompressedSize()) + assert.EqualValues(t, 6, rg2Col1.DictionaryPageOffset()) + assert.EqualValues(t, 16, rg2Col2.DictionaryPageOffset()) + assert.EqualValues(t, 10, rg2Col1.DataPageOffset()) + assert.EqualValues(t, 26, rg2Col2.DataPageOffset()) + assert.Len(t, rg2Col1.EncodingStats(), 3) + assert.Len(t, rg2Col2.EncodingStats(), 3) + + assert.Empty(t, rg2Col1.FilePath()) + accessor.SetFilePath("/foo/bar/bar.parquet") + assert.Equal(t, "/foo/bar/bar.parquet", rg2Col1.FilePath()) + } + + faccessor2, err := generateTableMetaData(schema, props, nrows, statsInt, statsFloat) + require.NoError(t, err) + faccessor.AppendRowGroups(faccessor2) + assert.Len(t, faccessor.RowGroups, 4) + assert.Equal(t, nrows*2, faccessor.NumRows) + assert.EqualValues(t, parquet.V2, faccessor.Version) + assert.Equal(t, parquet.DefaultCreatedBy, faccessor.GetCreatedBy()) + assert.Equal(t, 3, faccessor.NumSchemaElements()) + + faccessor1, err := faccessor.Subset([]int{2, 3}) + require.NoError(t, err) + assert.True(t, faccessor1.Equals(faccessor2)) + + faccessor1, err = faccessor2.Subset([]int{0}) + require.NoError(t, err) + + next, err := faccessor.Subset([]int{0}) + require.NoError(t, err) + faccessor1.AppendRowGroups(next) + + sub, err := faccessor.Subset([]int{2, 0}) + require.NoError(t, err) + assert.True(t, faccessor1.Equals(sub)) +} + +func TestV1VersionMetadata(t *testing.T) { + props := parquet.NewWriterProperties(parquet.WithVersion(parquet.V1)) + + fields := schema.FieldList{ + schema.NewInt32Node("int_col", parquet.Repetitions.Required, -1), + schema.NewFloat32Node("float_col", parquet.Repetitions.Required, -1), + } + root, err := schema.NewGroupNode("schema", parquet.Repetitions.Repeated, fields, -1) + require.NoError(t, err) + schema := schema.NewSchema(root) + + fbuilder := metadata.NewFileMetadataBuilder(schema, props, nil) + faccessor, err := fbuilder.Finish() + require.NoError(t, err) + assert.EqualValues(t, parquet.V1, faccessor.Version) +} + +func TestKeyValueMetadata(t *testing.T) { + props := parquet.NewWriterProperties(parquet.WithVersion(parquet.V1)) + + fields := schema.FieldList{ + schema.NewInt32Node("int_col", parquet.Repetitions.Required, -1), + schema.NewFloat32Node("float_col", parquet.Repetitions.Required, -1), + } + root, err := schema.NewGroupNode("schema", parquet.Repetitions.Repeated, fields, -1) + require.NoError(t, err) + schema := schema.NewSchema(root) + kvmeta := metadata.NewKeyValueMetadata() + kvmeta.Append("test_key", "test_value") + + fbuilder := metadata.NewFileMetadataBuilder(schema, props, kvmeta) + faccessor, err := fbuilder.Finish() + require.NoError(t, err) + + assert.True(t, faccessor.KeyValueMetadata().Equals(kvmeta)) +} + +func TestApplicationVersion(t *testing.T) { + version := metadata.NewAppVersion("parquet-mr version 1.7.9") + version1 := metadata.NewAppVersion("parquet-mr version 1.8.0") + version2 := metadata.NewAppVersion("parquet-cpp version 1.0.0") + version3 := metadata.NewAppVersion("") + version4 := metadata.NewAppVersion("parquet-mr version 1.5.0ab-cdh5.5.0+cd (build abcd)") + version5 := metadata.NewAppVersion("parquet-mr") + + assert.Equal(t, "parquet-mr", version.App) + assert.Equal(t, 1, version.Version.Major) + assert.Equal(t, 7, version.Version.Minor) + assert.Equal(t, 9, version.Version.Patch) + + assert.Equal(t, "parquet-cpp", version2.App) + assert.Equal(t, 1, version2.Version.Major) + assert.Equal(t, 0, version2.Version.Minor) + assert.Equal(t, 0, version2.Version.Patch) + + assert.Equal(t, "parquet-mr", version4.App) + assert.Equal(t, "abcd", version4.Build) + assert.Equal(t, 1, version4.Version.Major) + assert.Equal(t, 5, version4.Version.Minor) + assert.Equal(t, 0, version4.Version.Patch) + assert.Equal(t, "ab", version4.Version.Unknown) + assert.Equal(t, "cdh5.5.0", version4.Version.PreRelease) + assert.Equal(t, "cd", version4.Version.BuildInfo) + + assert.Equal(t, "parquet-mr", version5.App) + assert.Equal(t, 0, version5.Version.Major) + assert.Equal(t, 0, version5.Version.Minor) + assert.Equal(t, 0, version5.Version.Patch) + + assert.True(t, version.LessThan(version1)) + + var stats metadata.EncodedStatistics + assert.False(t, version1.HasCorrectStatistics(parquet.Types.Int96, stats, schema.SortUNKNOWN)) + assert.True(t, version.HasCorrectStatistics(parquet.Types.Int32, stats, schema.SortSIGNED)) + assert.False(t, version.HasCorrectStatistics(parquet.Types.ByteArray, stats, schema.SortSIGNED)) + assert.True(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, stats, schema.SortSIGNED)) + assert.False(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, stats, schema.SortUNSIGNED)) + assert.True(t, version3.HasCorrectStatistics(parquet.Types.FixedLenByteArray, stats, schema.SortSIGNED)) + + // check that the old stats are correct if min and max are the same regardless of sort order + var statsStr metadata.EncodedStatistics + statsStr.SetMin([]byte("a")).SetMax([]byte("b")) + assert.False(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, statsStr, schema.SortUNSIGNED)) + statsStr.SetMax([]byte("a")) + assert.True(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, statsStr, schema.SortUNSIGNED)) + + // check that the same holds true for ints + var ( + intMin int32 = 100 + intMax int32 = 200 + ) + var statsInt metadata.EncodedStatistics + statsInt.SetMin((*(*[4]byte)(unsafe.Pointer(&intMin)))[:]) + statsInt.SetMax((*(*[4]byte)(unsafe.Pointer(&intMax)))[:]) + assert.False(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, statsInt, schema.SortUNSIGNED)) + statsInt.SetMax((*(*[4]byte)(unsafe.Pointer(&intMin)))[:]) + assert.True(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, statsInt, schema.SortUNSIGNED)) +} diff --git a/go/parquet/metadata/row_group.go b/go/parquet/metadata/row_group.go new file mode 100644 index 00000000000..a252143ba22 --- /dev/null +++ b/go/parquet/metadata/row_group.go @@ -0,0 +1,168 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "reflect" + + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/internal/encryption" + format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet" + "github.com/apache/arrow/go/parquet/schema" + "golang.org/x/xerrors" +) + +// RowGroupMetaData is a proxy around the thrift RowGroup meta data object +type RowGroupMetaData struct { + rowGroup *format.RowGroup + Schema *schema.Schema + version *AppVersion + fileDecryptor encryption.FileDecryptor +} + +// NewRowGroupMetaData constructs an object from the underlying thrift objects and schema, +// decrypting if provided and necessary. This is primarily used internally and consumers +// should use the RowGroupMetaDataBuilder rather than this directly. +func NewRowGroupMetaData(rg *format.RowGroup, sc *schema.Schema, version *AppVersion, decryptor encryption.FileDecryptor) *RowGroupMetaData { + return &RowGroupMetaData{ + rowGroup: rg, + Schema: sc, + version: version, + fileDecryptor: decryptor, + } +} + +// NumColumns returns the number of column metadata objects in this row group +func (r *RowGroupMetaData) NumColumns() int { + return len(r.rowGroup.GetColumns()) +} + +func (r *RowGroupMetaData) Equals(other *RowGroupMetaData) bool { + return reflect.DeepEqual(r.rowGroup, other.rowGroup) +} + +// NumRows is just the number of rows in this row group +func (r *RowGroupMetaData) NumRows() int64 { return r.rowGroup.NumRows } + +// TotalByteSize is the total size of this rowgroup on disk +func (r *RowGroupMetaData) TotalByteSize() int64 { return r.rowGroup.GetTotalByteSize() } + +// FileOffset is the location in the file where the data for this rowgroup begins +func (r *RowGroupMetaData) FileOffset() int64 { return r.rowGroup.GetFileOffset() } + +func (r *RowGroupMetaData) TotalCompressedSize() int64 { return r.rowGroup.GetTotalCompressedSize() } + +// Ordinal is the row group number in order for the given file. +func (r *RowGroupMetaData) Ordinal() int16 { return r.rowGroup.GetOrdinal() } + +// ColumnChunk returns the metadata for the requested (0-based) chunk index +func (r *RowGroupMetaData) ColumnChunk(i int) (*ColumnChunkMetaData, error) { + if i >= r.NumColumns() { + panic(xerrors.Errorf("parquet: the file only has %d columns, requested metadata for column: %d", r.NumColumns(), i)) + } + + return NewColumnChunkMetaData(r.rowGroup.Columns[i], r.Schema.Column(i), r.version, r.rowGroup.GetOrdinal(), int16(i), r.fileDecryptor) +} + +// RowGroupMetaDataBuilder is a convenience object for constructing row group +// metadata information. Primarily used in conjunction with writing new files. +type RowGroupMetaDataBuilder struct { + rg *format.RowGroup + props *parquet.WriterProperties + schema *schema.Schema + colBuilders []*ColumnChunkMetaDataBuilder + nextCol int +} + +// NewRowGroupMetaDataBuilder returns a builder using the given properties and underlying thrift object. +// +// This is primarily used internally, consumers should use the file metadatabuilder and call +// AppendRowGroup on it to get instances of RowGroupMetaDataBuilder +func NewRowGroupMetaDataBuilder(props *parquet.WriterProperties, schema *schema.Schema, rg *format.RowGroup) *RowGroupMetaDataBuilder { + r := &RowGroupMetaDataBuilder{ + rg: rg, + props: props, + schema: schema, + colBuilders: make([]*ColumnChunkMetaDataBuilder, 0), + } + r.rg.Columns = make([]*format.ColumnChunk, schema.NumColumns()) + return r +} + +// NumColumns returns the current number of columns in this metadata +func (r *RowGroupMetaDataBuilder) NumColumns() int { + return int(len(r.rg.GetColumns())) +} + +func (r *RowGroupMetaDataBuilder) NumRows() int64 { + return r.rg.GetNumRows() +} + +func (r *RowGroupMetaDataBuilder) SetNumRows(nrows int64) { + r.rg.NumRows = nrows +} + +// CurrentColumn returns the current column chunk (0-based) index that is being built. +// +// Returns -1 until the first time NextColumnChunk is called. +func (r *RowGroupMetaDataBuilder) CurrentColumn() int { return r.nextCol - 1 } + +// NextColumnChunk appends a new column chunk, updates the column index, +// and returns a builder for that column chunk's metadata +func (r *RowGroupMetaDataBuilder) NextColumnChunk() *ColumnChunkMetaDataBuilder { + if r.nextCol >= r.NumColumns() { + panic(xerrors.Errorf("parquet: the schema only has %d columns, requested metadata for col: %d", r.NumColumns(), r.nextCol)) + } + + col := r.schema.Column(r.nextCol) + if r.rg.Columns[r.nextCol] == nil { + r.rg.Columns[r.nextCol] = &format.ColumnChunk{MetaData: format.NewColumnMetaData()} + } + colBldr := NewColumnChunkMetaDataBuilderWithContents(r.props, col, r.rg.Columns[r.nextCol]) + r.nextCol++ + r.colBuilders = append(r.colBuilders, colBldr) + return colBldr +} + +func (r *RowGroupMetaDataBuilder) Finish(totalBytesWritten int64, ordinal int16) error { + if r.nextCol != r.NumColumns() { + return xerrors.Errorf("parquet: only %d out of %d columns are initialized", r.nextCol-1, r.schema.NumColumns()) + } + + var ( + fileOffset int64 = 0 + totalCompressed int64 = 0 + ) + + for idx, col := range r.rg.Columns { + if col.FileOffset < 0 { + return xerrors.Errorf("parquet: Column %d is not complete", idx) + } + if idx == 0 { + fileOffset = col.FileOffset + } + // sometimes column metadata is encrypted and not available to read + // so we must get total compressed size from column builder + totalCompressed += r.colBuilders[idx].TotalCompressedSize() + } + + r.rg.FileOffset = &fileOffset + r.rg.TotalCompressedSize = &totalCompressed + r.rg.TotalByteSize = totalBytesWritten + r.rg.Ordinal = &ordinal + return nil +} diff --git a/go/parquet/metadata/stat_compare_test.go b/go/parquet/metadata/stat_compare_test.go new file mode 100644 index 00000000000..373657de867 --- /dev/null +++ b/go/parquet/metadata/stat_compare_test.go @@ -0,0 +1,223 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "encoding/binary" + "testing" + + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/schema" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestSignedByteArrayCompare(t *testing.T) { + s := ByteArrayStatistics{ + statistics: statistics{ + order: schema.SortSIGNED, + }, + } + + s1ba := parquet.ByteArray("12345") + s2ba := parquet.ByteArray("12345678") + assert.True(t, s.less(s1ba, s2ba)) + + // This is case where signed comparison UTF-8 (PARQUET-686) is incorrect + // This example is to only check signed comparison and not UTF-8. + s1ba = parquet.ByteArray("bügeln") + s2ba = parquet.ByteArray("braten") + assert.True(t, s.less(s1ba, s2ba)) +} + +func TestUnsignedByteArrayCompare(t *testing.T) { + s := ByteArrayStatistics{ + statistics: statistics{ + order: schema.SortUNSIGNED, + }, + } + + s1ba := parquet.ByteArray("arrange") + s2ba := parquet.ByteArray("arrangement") + assert.True(t, s.less(s1ba, s2ba)) + + // multi-byte utf-8 characters + s1ba = parquet.ByteArray("braten") + s2ba = parquet.ByteArray("bügeln") + assert.True(t, s.less(s1ba, s2ba)) + + s1ba = parquet.ByteArray("ünk123456") // ü = 252 + s2ba = parquet.ByteArray("ănk123456") // ă = 259 + assert.True(t, s.less(s1ba, s2ba)) +} + +func TestSignedCompareFLBA(t *testing.T) { + s := FixedLenByteArrayStatistics{ + statistics: statistics{order: schema.SortSIGNED}, + } + + s1flba := parquet.FixedLenByteArray("Anti123456") + s2flba := parquet.FixedLenByteArray("Bunkd123456") + assert.True(t, s.less(s1flba, s2flba)) + + s1flba = parquet.FixedLenByteArray("Bünk123456") + s2flba = parquet.FixedLenByteArray("Bunk123456") + assert.True(t, s.less(s1flba, s2flba)) +} + +func TestUnsignedCompareFLBA(t *testing.T) { + s := FixedLenByteArrayStatistics{ + statistics: statistics{order: schema.SortUNSIGNED}, + } + + s1flba := parquet.FixedLenByteArray("Anti123456") + s2flba := parquet.FixedLenByteArray("Bunkd123456") + assert.True(t, s.less(s1flba, s2flba)) + + s1flba = parquet.FixedLenByteArray("Bunk123456") + s2flba = parquet.FixedLenByteArray("Bünk123456") + assert.True(t, s.less(s1flba, s2flba)) +} + +func TestSignedCompareInt96(t *testing.T) { + s := Int96Statistics{ + statistics: statistics{order: schema.SortSIGNED}, + } + + val := -14 + + var ( + a = parquet.NewInt96([3]uint32{1, 41, 14}) + b = parquet.NewInt96([3]uint32{1, 41, 42}) + aa = parquet.NewInt96([3]uint32{1, 41, 14}) + bb = parquet.NewInt96([3]uint32{1, 41, 14}) + aaa = parquet.NewInt96([3]uint32{1, 41, uint32(val)}) + bbb = parquet.NewInt96([3]uint32{1, 41, 42}) + ) + + assert.True(t, s.less(a, b)) + assert.True(t, !s.less(aa, bb) && !s.less(bb, aa)) + assert.True(t, s.less(aaa, bbb)) +} + +func TestUnsignedCompareInt96(t *testing.T) { + s := Int96Statistics{ + statistics: statistics{order: schema.SortUNSIGNED}, + } + + valb := -41 + valbb := -14 + + var ( + a = parquet.NewInt96([3]uint32{1, 41, 14}) + b = parquet.NewInt96([3]uint32{1, uint32(valb), 42}) + aa = parquet.NewInt96([3]uint32{1, 41, 14}) + bb = parquet.NewInt96([3]uint32{1, 41, uint32(valbb)}) + aaa parquet.Int96 + bbb parquet.Int96 + ) + + assert.True(t, s.less(a, b)) + assert.True(t, s.less(aa, bb)) + + binary.LittleEndian.PutUint32(aaa[8:], 2451545) // 2000-01-01 + binary.LittleEndian.PutUint32(bbb[8:], 2451546) // 2000-01-02 + // 12 hours + 34 minutes + 56 seconds + aaa.SetNanoSeconds(45296000000000) + // 12 hours + 34 minutes + 50 seconds + bbb.SetNanoSeconds(45290000000000) + assert.True(t, s.less(aaa, bbb)) + + binary.LittleEndian.PutUint32(aaa[8:], 2451545) // 2000-01-01 + binary.LittleEndian.PutUint32(bbb[8:], 2451545) // 2000-01-01 + // 11 hours + 34 minutes + 56 seconds + aaa.SetNanoSeconds(41696000000000) + // 12 hours + 34 minutes + 50 seconds + bbb.SetNanoSeconds(45290000000000) + assert.True(t, s.less(aaa, bbb)) + + binary.LittleEndian.PutUint32(aaa[8:], 2451545) // 2000-01-01 + binary.LittleEndian.PutUint32(bbb[8:], 2451545) // 2000-01-01 + // 12 hours + 34 minutes + 55 seconds + aaa.SetNanoSeconds(45295000000000) + // 12 hours + 34 minutes + 56 seconds + bbb.SetNanoSeconds(45296000000000) + assert.True(t, s.less(aaa, bbb)) +} + +func TestCompareSignedInt64(t *testing.T) { + var ( + a int64 = 1 + b int64 = 4 + aa int64 = 1 + bb int64 = 1 + aaa int64 = -1 + bbb int64 = 1 + ) + + n := schema.NewInt64Node("signedint64", parquet.Repetitions.Required, -1) + descr := schema.NewColumn(n, 0, 0) + s := NewStatistics(descr, nil).(*Int64Statistics) + + assert.True(t, s.less(a, b)) + assert.True(t, !s.less(aa, bb) && !s.less(bb, aa)) + assert.True(t, s.less(aaa, bbb)) +} + +func TestCompareUnsignedInt64(t *testing.T) { + var ( + a int64 = 1 + b int64 = 4 + aa int64 = 1 + bb int64 = 1 + aaa int64 = 1 + bbb int64 = -1 + ) + + n, err := schema.NewPrimitiveNodeConverted("unsigned int64", parquet.Repetitions.Required, parquet.Types.Int64, schema.ConvertedTypes.Uint64, 0, 0, 0, 0) + require.NoError(t, err) + descr := schema.NewColumn(n, 0, 0) + + assert.Equal(t, schema.SortUNSIGNED, descr.SortOrder()) + s := NewStatistics(descr, nil).(*Int64Statistics) + + assert.True(t, s.less(a, b)) + assert.True(t, !s.less(aa, bb) && !s.less(bb, aa)) + assert.True(t, s.less(aaa, bbb)) +} + +func TestCompareUnsignedInt32(t *testing.T) { + var ( + a int32 = 1 + b int32 = 4 + aa int32 = 1 + bb int32 = 1 + aaa int32 = 1 + bbb int32 = -1 + ) + + n, err := schema.NewPrimitiveNodeConverted("unsigned int32", parquet.Repetitions.Required, parquet.Types.Int32, schema.ConvertedTypes.Uint32, 0, 0, 0, 0) + require.NoError(t, err) + descr := schema.NewColumn(n, 0, 0) + + assert.Equal(t, schema.SortUNSIGNED, descr.SortOrder()) + s := NewStatistics(descr, nil).(*Int32Statistics) + + assert.True(t, s.less(a, b)) + assert.True(t, !s.less(aa, bb) && !s.less(bb, aa)) + assert.True(t, s.less(aaa, bbb)) +} diff --git a/go/parquet/metadata/statistics.go b/go/parquet/metadata/statistics.go new file mode 100644 index 00000000000..dd982e04f74 --- /dev/null +++ b/go/parquet/metadata/statistics.go @@ -0,0 +1,517 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "bytes" + "encoding/binary" + "math" + "unsafe" + + "github.com/apache/arrow/go/arrow" + "github.com/apache/arrow/go/arrow/array" + "github.com/apache/arrow/go/arrow/memory" + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/internal/encoding" + format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet" + "github.com/apache/arrow/go/parquet/internal/utils" + "github.com/apache/arrow/go/parquet/schema" +) + +//go:generate go run ../../arrow/_tools/tmpl/main.go -i -data=../internal/encoding/physical_types.tmpldata statistics_types.gen.go.tmpl + +// EncodedStatistics are raw statistics with encoded values that will be written +// to the parquet file, or was read from the parquet file. +type EncodedStatistics struct { + HasMax bool + Max []byte + HasMin bool + Min []byte + Signed bool + HasNullCount bool + NullCount int64 + HasDistinctCount bool + DistinctCount int64 +} + +// ApplyStatSizeLimits sets the maximum size of the min/max values. +// +// from parquet-mr +// we don't write stats larger than the max size rather than truncating. +// the rationale is that some engines may use the minimum value in the page +// as the true minimum for aggregations and there is no way to mark that +// a value has been truncated and is a lower bound and not in the page +func (e *EncodedStatistics) ApplyStatSizeLimits(length int) { + if len(e.Max) > length { + e.HasMax = false + } + if len(e.Min) > length { + e.HasMin = false + } +} + +// IsSet returns true iff one of the Has* values is true. +func (e *EncodedStatistics) IsSet() bool { + return e.HasMin || e.HasMax || e.HasNullCount || e.HasDistinctCount +} + +// SetMax sets the encoded Max value to val and sets HasMax to true +func (e *EncodedStatistics) SetMax(val []byte) *EncodedStatistics { + e.Max = val[:] + e.HasMax = true + return e +} + +// SetMin sets the encoded Min value to val, and sets HasMin to true +func (e *EncodedStatistics) SetMin(val []byte) *EncodedStatistics { + e.Min = val[:] + e.HasMin = true + return e +} + +// SetNullCount sets the NullCount to val and sets HasNullCount to true +func (e *EncodedStatistics) SetNullCount(val int64) *EncodedStatistics { + e.NullCount = val + e.HasNullCount = true + return e +} + +// SetDistinctCount sets the DistinctCount to val and sets HasDistinctCount to true +func (e *EncodedStatistics) SetDistinctCount(val int64) *EncodedStatistics { + e.DistinctCount = val + e.HasDistinctCount = true + return e +} + +func (e *EncodedStatistics) ToThrift() (stats *format.Statistics) { + stats = format.NewStatistics() + if e.HasMin { + stats.MinValue = e.Min + // if sort order is SIGNED then the old min value must be set too for backwards compatibility + if e.Signed { + stats.Min = e.Min + } + } + if e.HasMax { + stats.MaxValue = e.Max + // if sort order is SIGNED then old max value must be set to + if e.Signed { + stats.Max = e.Max + } + } + if e.HasNullCount { + stats.NullCount = &e.NullCount + } + if e.HasDistinctCount { + stats.DistinctCount = &e.DistinctCount + } + return +} + +// TypedStatistics is the base interface for dealing with stats as +// they are being populated +type TypedStatistics interface { + // Type is the underlying physical type for this stat block + Type() parquet.Type + // Returns true if there is a min and max value set for this stat object + HasMinMax() bool + // Returns true if a nullcount has been set + HasNullCount() bool + // returns true only if a distinct count has been set + // current implementation does of the writer does not automatically populate + // the distinct count right now. + HasDistinctCount() bool + NullCount() int64 + DistinctCount() int64 + NumValues() int64 + // return the column descriptor that this stat object was initialized with + Descr() *schema.Column + + // Encode the current min value and return the bytes. ByteArray does not + // include the len in the encoded bytes, otherwise this is identical to + // plain encoding + EncodeMin() []byte + // Encode the current max value and return the bytes. ByteArray does not + // include the len in the encoded bytes, otherwise this is identical to + // plain encoding + EncodeMax() []byte + // Populate an EncodedStatistics object from the current stats + Encode() (EncodedStatistics, error) + // Resets all values to 0 to enable reusing this stat object for multiple + // columns, by calling Encode to get the finished values and then calling + // reset + Reset() + // Merge the min/max/nullcounts and distinct count from the passed stat object + // into this one. + Merge(TypedStatistics) +} + +type statistics struct { + descr *schema.Column + hasMinMax bool + hasNullCount bool + hasDistinctCount bool + mem memory.Allocator + nvalues int64 + stats EncodedStatistics + order schema.SortOrder + + encoder encoding.TypedEncoder +} + +func (s *statistics) incNulls(n int64) { + s.stats.NullCount += n + s.hasNullCount = true +} +func (s *statistics) incDistinct(n int64) { + s.stats.DistinctCount += n + s.hasDistinctCount = true +} + +func (s *statistics) Descr() *schema.Column { return s.descr } +func (s *statistics) Type() parquet.Type { return s.descr.PhysicalType() } +func (s *statistics) HasDistinctCount() bool { return s.hasDistinctCount } +func (s *statistics) HasMinMax() bool { return s.hasMinMax } +func (s *statistics) HasNullCount() bool { return s.hasNullCount } +func (s *statistics) NullCount() int64 { return s.stats.NullCount } +func (s *statistics) DistinctCount() int64 { return s.stats.DistinctCount } +func (s *statistics) NumValues() int64 { return s.nvalues } + +func (s *statistics) Reset() { + s.stats.NullCount = 0 + s.stats.DistinctCount = 0 + s.nvalues = 0 + s.hasMinMax = false + s.hasDistinctCount = false + s.hasNullCount = false +} + +func (s *statistics) merge(other TypedStatistics) { + s.nvalues += other.NumValues() + if other.HasNullCount() { + s.stats.NullCount += other.NullCount() + } + if other.HasDistinctCount() { + s.stats.DistinctCount += other.DistinctCount() + } +} + +func coalesce(val, fallback interface{}) interface{} { + switch v := val.(type) { + case float32: + if math.IsNaN(float64(v)) { + return fallback + } + case float64: + if math.IsNaN(v) { + return fallback + } + } + return val +} + +func (BooleanStatistics) defaultMin() bool { return true } +func (BooleanStatistics) defaultMax() bool { return false } +func (s *Int32Statistics) defaultMin() int32 { + if s.order == schema.SortUNSIGNED { + val := math.MaxUint32 + return int32(val) + } + return math.MaxInt32 +} + +func (s *Int32Statistics) defaultMax() int32 { + if s.order == schema.SortUNSIGNED { + return int32(0) + } + return math.MinInt32 +} + +func (s *Int64Statistics) defaultMin() int64 { + if s.order == schema.SortUNSIGNED { + val := uint64(math.MaxUint64) + return int64(val) + } + return math.MaxInt64 +} + +func (s *Int64Statistics) defaultMax() int64 { + if s.order == schema.SortUNSIGNED { + return int64(0) + } + return math.MinInt64 +} + +var ( + defaultMinInt96 parquet.Int96 + defaultMinUInt96 parquet.Int96 + defaultMaxInt96 parquet.Int96 + defaultMaxUInt96 parquet.Int96 +) + +func init() { + i96 := arrow.Uint32Traits.CastFromBytes(defaultMinInt96[:]) + i96[0] = math.MaxUint32 + i96[1] = math.MaxUint32 + i96[2] = math.MaxInt32 + + i96 = arrow.Uint32Traits.CastFromBytes(defaultMinUInt96[:]) + i96[0] = math.MaxUint32 + i96[1] = math.MaxUint32 + i96[2] = math.MaxUint32 + + // golang will initialize the bytes to 0 + i96 = arrow.Uint32Traits.CastFromBytes(defaultMaxInt96[:]) + i96[2] = math.MaxInt32 + 1 + + // defaultMaxUInt96 will be initialized to 0 as desired +} + +func (s *Int96Statistics) defaultMin() parquet.Int96 { + if s.order == schema.SortUNSIGNED { + return defaultMinUInt96 + } + return defaultMinInt96 +} + +func (s *Int96Statistics) defaultMax() parquet.Int96 { + if s.order == schema.SortUNSIGNED { + return defaultMaxUInt96 + } + return defaultMaxInt96 +} + +func (Float32Statistics) defaultMin() float32 { return math.MaxFloat32 } +func (Float32Statistics) defaultMax() float32 { return -math.MaxFloat32 } +func (Float64Statistics) defaultMin() float64 { return math.MaxFloat64 } +func (Float64Statistics) defaultMax() float64 { return -math.MaxFloat64 } +func (ByteArrayStatistics) defaultMin() parquet.ByteArray { return nil } +func (ByteArrayStatistics) defaultMax() parquet.ByteArray { return nil } +func (FixedLenByteArrayStatistics) defaultMin() parquet.FixedLenByteArray { return nil } +func (FixedLenByteArrayStatistics) defaultMax() parquet.FixedLenByteArray { return nil } + +func (BooleanStatistics) equal(a, b bool) bool { return a == b } +func (Int32Statistics) equal(a, b int32) bool { return a == b } +func (Int64Statistics) equal(a, b int64) bool { return a == b } +func (Float32Statistics) equal(a, b float32) bool { return a == b } +func (Float64Statistics) equal(a, b float64) bool { return a == b } +func (Int96Statistics) equal(a, b parquet.Int96) bool { return bytes.Equal(a[:], b[:]) } +func (ByteArrayStatistics) equal(a, b parquet.ByteArray) bool { return bytes.Equal(a, b) } +func (FixedLenByteArrayStatistics) equal(a, b parquet.FixedLenByteArray) bool { + return bytes.Equal(a, b) +} + +func (BooleanStatistics) less(a, b bool) bool { + return !a && b +} + +func (s *Int32Statistics) less(a, b int32) bool { + if s.order == schema.SortUNSIGNED { + return uint32(a) < uint32(b) + } + return a < b +} + +func (s *Int64Statistics) less(a, b int64) bool { + if s.order == schema.SortUNSIGNED { + return uint64(a) < uint64(b) + } + return a < b +} +func (Float32Statistics) less(a, b float32) bool { return a < b } +func (Float64Statistics) less(a, b float64) bool { return a < b } +func (s *Int96Statistics) less(a, b parquet.Int96) bool { + i96a := arrow.Uint32Traits.CastFromBytes(a[:]) + i96b := arrow.Uint32Traits.CastFromBytes(b[:]) + if i96a[2] != i96b[2] { + // only the msb bit is by signed comparison + if s.order == schema.SortSIGNED { + return int32(i96a[2]) < int32(i96b[2]) + } + return i96a[2] < i96b[2] + } else if i96a[1] != i96b[1] { + return i96a[1] < i96b[1] + } + return i96a[0] < i96b[0] +} + +func (s *ByteArrayStatistics) less(a, b parquet.ByteArray) bool { + if s.order == schema.SortUNSIGNED { + return bytes.Compare(a, b) == -1 + } + + sa := *(*[]int8)(unsafe.Pointer(&a)) + sb := *(*[]int8)(unsafe.Pointer(&b)) + i := 0 + for ; i < len(sa) && i < len(sb); i++ { + if sa[i] < sb[i] { + return true + } + if sb[i] < sa[i] { + return false + } + } + return i == len(sa) && i != len(sb) +} + +func (s *FixedLenByteArrayStatistics) less(a, b parquet.FixedLenByteArray) bool { + if s.order == schema.SortUNSIGNED { + return bytes.Compare(a, b) == -1 + } + + sa := *(*[]int8)(unsafe.Pointer(&a)) + sb := *(*[]int8)(unsafe.Pointer(&b)) + i := 0 + for ; i < len(sa) && i < len(sb); i++ { + if sa[i] < sb[i] { + return true + } + if sb[i] < sa[i] { + return false + } + } + return i == len(sa) && i != len(sb) +} + +func (BooleanStatistics) cleanStat(minMax minmaxPairBoolean) *minmaxPairBoolean { return &minMax } +func (Int32Statistics) cleanStat(minMax minmaxPairInt32) *minmaxPairInt32 { return &minMax } +func (Int64Statistics) cleanStat(minMax minmaxPairInt64) *minmaxPairInt64 { return &minMax } +func (Int96Statistics) cleanStat(minMax minmaxPairInt96) *minmaxPairInt96 { return &minMax } + +// in the case of floating point types, the following rules are applied as per parquet-mr: +// - if any of min/max is NaN, return nothing +// - if min is 0.0f replace with -0.0f +// - if max is -0.0f replace with 0.0f +func (Float32Statistics) cleanStat(minMax minmaxPairFloat32) *minmaxPairFloat32 { + if math.IsNaN(float64(minMax[0])) || math.IsNaN(float64(minMax[1])) { + return nil + } + + if minMax[0] == math.MaxFloat32 && minMax[1] == -math.MaxFloat32 { + return nil + } + + var zero float32 = 0 + if minMax[0] == zero && !math.Signbit(float64(minMax[0])) { + minMax[0] = -minMax[0] + } + + if minMax[1] == zero && math.Signbit(float64(minMax[1])) { + minMax[1] = -minMax[1] + } + + return &minMax +} + +func (Float64Statistics) cleanStat(minMax minmaxPairFloat64) *minmaxPairFloat64 { + if math.IsNaN(minMax[0]) || math.IsNaN(minMax[1]) { + return nil + } + + if minMax[0] == math.MaxFloat64 && minMax[1] == -math.MaxFloat64 { + return nil + } + + var zero float64 = 0 + if minMax[0] == zero && !math.Signbit(minMax[0]) { + minMax[0] = -minMax[0] + } + + if minMax[1] == zero && math.Signbit(minMax[1]) { + minMax[1] = -minMax[1] + } + + return &minMax +} + +func (ByteArrayStatistics) cleanStat(minMax minmaxPairByteArray) *minmaxPairByteArray { + if minMax[0] == nil || minMax[1] == nil { + return nil + } + return &minMax +} + +func (FixedLenByteArrayStatistics) cleanStat(minMax minmaxPairFixedLenByteArray) *minmaxPairFixedLenByteArray { + if minMax[0] == nil || minMax[1] == nil { + return nil + } + return &minMax +} + +func GetStatValue(typ parquet.Type, val []byte) interface{} { + switch typ { + case parquet.Types.Boolean: + return val[0] != 0 + case parquet.Types.Int32: + return int32(binary.LittleEndian.Uint32(val)) + case parquet.Types.Int64: + return int64(binary.LittleEndian.Uint64(val)) + case parquet.Types.Int96: + p := parquet.Int96{} + copy(p[:], val) + return p + case parquet.Types.Float: + return math.Float32frombits(binary.LittleEndian.Uint32(val)) + case parquet.Types.Double: + return math.Float64frombits(binary.LittleEndian.Uint64(val)) + case parquet.Types.ByteArray: + fallthrough + case parquet.Types.FixedLenByteArray: + return val + } + return nil +} + +func (s *ByteArrayStatistics) UpdateWithArrow(values array.Interface) { + if _, ok := values.DataType().(arrow.BinaryDataType); !ok { + panic("can't update bytearray stats with non binary arrow data") + } + + s.incNulls(int64(values.NullN())) + s.nvalues += int64(values.Len()) - int64(values.NullN()) + if values.NullN() == values.Len() { + return + } + + var ( + min = s.defaultMin() + max = s.defaultMax() + + offsets []int32 + valueBuf []byte + ) + + switch arr := values.(type) { + case *array.Binary: + offsets = arr.ValueOffsets() + case *array.String: + offsets = arrow.Int32Traits.CastFromBytes(arr.Data().Buffers()[1].Bytes())[arr.Offset() : arr.Offset()+arr.Len()+1] + } + valueBuf = values.Data().Buffers()[2].Bytes() + + utils.VisitBitBlocks(values.NullBitmapBytes(), int64(values.Data().Offset()), int64(values.Len()), + func(_ int64) { + val := valueBuf[offsets[0]:offsets[1]] + min = s.minval(min, val) + max = s.maxval(max, val) + offsets = offsets[1:] + }, func() { + offsets = offsets[1:] + }) + + s.SetMinMax(min, max) +} diff --git a/go/parquet/metadata/statistics_test.go b/go/parquet/metadata/statistics_test.go new file mode 100644 index 00000000000..b052b37f778 --- /dev/null +++ b/go/parquet/metadata/statistics_test.go @@ -0,0 +1,190 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata_test + +import ( + "math" + "reflect" + "testing" + + "github.com/apache/arrow/go/arrow/bitutil" + "github.com/apache/arrow/go/arrow/memory" + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/metadata" + "github.com/apache/arrow/go/parquet/schema" + "github.com/stretchr/testify/assert" +) + +// NOTE(zeroshade): tests will be added and updated after merging the "file" package +// since the tests that I wrote relied on the file writer/reader for ease of use. + +func TestCheckNaNs(t *testing.T) { + const ( + numvals = 8 + min = -4.0 + max = 3.0 + ) + nan := math.NaN() + + allNans := []float64{nan, nan, nan, nan, nan, nan, nan, nan} + allNansf32 := make([]float32, numvals) + for idx, v := range allNans { + allNansf32[idx] = float32(v) + } + + someNans := []float64{nan, max, -3.0, -1.0, nan, 2.0, min, nan} + someNansf32 := make([]float32, numvals) + for idx, v := range someNans { + someNansf32[idx] = float32(v) + } + + validBitmap := []byte{0x7F} // 0b01111111 + validBitmapNoNaNs := []byte{0x6E} // 0b01101110 + + assertUnsetMinMax := func(stats metadata.TypedStatistics, values interface{}, bitmap []byte) { + if bitmap == nil { + switch s := stats.(type) { + case *metadata.Float32Statistics: + s.Update(values.([]float32), 0) + case *metadata.Float64Statistics: + s.Update(values.([]float64), 0) + } + assert.False(t, stats.HasMinMax()) + } else { + nvalues := reflect.ValueOf(values).Len() + nullCount := bitutil.CountSetBits(bitmap, 0, nvalues) + switch s := stats.(type) { + case *metadata.Float32Statistics: + s.UpdateSpaced(values.([]float32), bitmap, 0, int64(nullCount)) + case *metadata.Float64Statistics: + s.UpdateSpaced(values.([]float64), bitmap, 0, int64(nullCount)) + } + assert.False(t, stats.HasMinMax()) + } + } + + assertMinMaxAre := func(stats metadata.TypedStatistics, values interface{}, expectedMin, expectedMax interface{}) { + switch s := stats.(type) { + case *metadata.Float32Statistics: + s.Update(values.([]float32), 0) + assert.True(t, stats.HasMinMax()) + assert.Equal(t, expectedMin, s.Min()) + assert.Equal(t, expectedMax, s.Max()) + case *metadata.Float64Statistics: + s.Update(values.([]float64), 0) + assert.True(t, stats.HasMinMax()) + assert.Equal(t, expectedMin, s.Min()) + assert.Equal(t, expectedMax, s.Max()) + } + } + + assertMinMaxAreSpaced := func(stats metadata.TypedStatistics, values interface{}, bitmap []byte, expectedMin, expectedMax interface{}) { + nvalues := reflect.ValueOf(values).Len() + nullCount := bitutil.CountSetBits(bitmap, 0, nvalues) + switch s := stats.(type) { + case *metadata.Float32Statistics: + s.UpdateSpaced(values.([]float32), bitmap, 0, int64(nullCount)) + assert.True(t, s.HasMinMax()) + assert.Equal(t, expectedMin, s.Min()) + assert.Equal(t, expectedMax, s.Max()) + case *metadata.Float64Statistics: + s.UpdateSpaced(values.([]float64), bitmap, 0, int64(nullCount)) + assert.True(t, s.HasMinMax()) + assert.Equal(t, expectedMin, s.Min()) + assert.Equal(t, expectedMax, s.Max()) + } + } + + f32Col := schema.NewColumn(schema.NewFloat32Node("f", parquet.Repetitions.Optional, -1), 1, 1) + f64Col := schema.NewColumn(schema.NewFloat64Node("f", parquet.Repetitions.Optional, -1), 1, 1) + // test values + someNanStats := metadata.NewStatistics(f64Col, memory.DefaultAllocator) + someNanStatsf32 := metadata.NewStatistics(f32Col, memory.DefaultAllocator) + // ingesting only nans should not yield a min or max + assertUnsetMinMax(someNanStats, allNans, nil) + assertUnsetMinMax(someNanStatsf32, allNansf32, nil) + // ingesting a mix should yield a valid min/max + assertMinMaxAre(someNanStats, someNans, min, max) + assertMinMaxAre(someNanStatsf32, someNansf32, float32(min), float32(max)) + // ingesting only nans after a valid min/max should have no effect + assertMinMaxAre(someNanStats, allNans, min, max) + assertMinMaxAre(someNanStatsf32, allNansf32, float32(min), float32(max)) + + someNanStats = metadata.NewStatistics(f64Col, memory.DefaultAllocator) + someNanStatsf32 = metadata.NewStatistics(f32Col, memory.DefaultAllocator) + assertUnsetMinMax(someNanStats, allNans, validBitmap) + assertUnsetMinMax(someNanStatsf32, allNansf32, validBitmap) + // nans should not pollute min/max when excluded via null bitmap + assertMinMaxAreSpaced(someNanStats, someNans, validBitmapNoNaNs, min, max) + assertMinMaxAreSpaced(someNanStatsf32, someNansf32, validBitmapNoNaNs, float32(min), float32(max)) + // ingesting nans with a null bitmap should not change the result + assertMinMaxAreSpaced(someNanStats, someNans, validBitmap, min, max) + assertMinMaxAreSpaced(someNanStatsf32, someNansf32, validBitmap, float32(min), float32(max)) +} + +func TestCheckNegativeZeroStats(t *testing.T) { + assertMinMaxZeroesSign := func(stats metadata.TypedStatistics, values interface{}) { + switch s := stats.(type) { + case *metadata.Float32Statistics: + s.Update(values.([]float32), 0) + assert.True(t, s.HasMinMax()) + var zero float32 + assert.Equal(t, zero, s.Min()) + assert.True(t, math.Signbit(float64(s.Min()))) + assert.Equal(t, zero, s.Max()) + assert.False(t, math.Signbit(float64(s.Max()))) + case *metadata.Float64Statistics: + s.Update(values.([]float64), 0) + assert.True(t, s.HasMinMax()) + var zero float64 + assert.Equal(t, zero, s.Min()) + assert.True(t, math.Signbit(s.Min())) + assert.Equal(t, zero, s.Max()) + assert.False(t, math.Signbit(s.Max())) + } + } + + fcol := schema.NewColumn(schema.NewFloat32Node("f", parquet.Repetitions.Optional, -1), 1, 1) + dcol := schema.NewColumn(schema.NewFloat64Node("d", parquet.Repetitions.Optional, -1), 1, 1) + + var f32zero float32 + var f64zero float64 + { + fstats := metadata.NewStatistics(fcol, memory.DefaultAllocator) + dstats := metadata.NewStatistics(dcol, memory.DefaultAllocator) + assertMinMaxZeroesSign(fstats, []float32{-f32zero, f32zero}) + assertMinMaxZeroesSign(dstats, []float64{-f64zero, f64zero}) + } + { + fstats := metadata.NewStatistics(fcol, memory.DefaultAllocator) + dstats := metadata.NewStatistics(dcol, memory.DefaultAllocator) + assertMinMaxZeroesSign(fstats, []float32{f32zero, -f32zero}) + assertMinMaxZeroesSign(dstats, []float64{f64zero, -f64zero}) + } + { + fstats := metadata.NewStatistics(fcol, memory.DefaultAllocator) + dstats := metadata.NewStatistics(dcol, memory.DefaultAllocator) + assertMinMaxZeroesSign(fstats, []float32{-f32zero, -f32zero}) + assertMinMaxZeroesSign(dstats, []float64{-f64zero, -f64zero}) + } + { + fstats := metadata.NewStatistics(fcol, memory.DefaultAllocator) + dstats := metadata.NewStatistics(dcol, memory.DefaultAllocator) + assertMinMaxZeroesSign(fstats, []float32{f32zero, f32zero}) + assertMinMaxZeroesSign(dstats, []float64{f64zero, f64zero}) + } +} diff --git a/go/parquet/metadata/statistics_types.gen.go b/go/parquet/metadata/statistics_types.gen.go new file mode 100644 index 00000000000..1625f6740e6 --- /dev/null +++ b/go/parquet/metadata/statistics_types.gen.go @@ -0,0 +1,2292 @@ +// Code generated by statistics_types.gen.go.tmpl. DO NOT EDIT. + +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "math" + + "github.com/apache/arrow/go/arrow" + "github.com/apache/arrow/go/arrow/memory" + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/internal/encoding" + "github.com/apache/arrow/go/parquet/internal/utils" + "github.com/apache/arrow/go/parquet/schema" + "golang.org/x/xerrors" +) + +type minmaxPairInt32 [2]int32 + +// Int32Statistics is the typed interface for managing stats for a column +// of Int32 type. +type Int32Statistics struct { + statistics + min int32 + max int32 + + bitSetReader utils.SetBitRunReader +} + +// NewInt32Statistics constructs an appropriate stat object type using the +// given column descriptor and allocator. +// +// Panics if the physical type of descr is not parquet.Type.Int32 +func NewInt32Statistics(descr *schema.Column, mem memory.Allocator) *Int32Statistics { + if descr.PhysicalType() != parquet.Types.Int32 { + panic(xerrors.Errorf("parquet: invalid type %s for constructing a Int32 stat object", descr.PhysicalType())) + } + + return &Int32Statistics{ + statistics: statistics{ + descr: descr, + hasNullCount: true, + hasDistinctCount: true, + order: descr.SortOrder(), + encoder: encoding.NewEncoder(descr.PhysicalType(), parquet.Encodings.Plain, false, descr, mem), + mem: mem, + }, + } +} + +// NewInt32StatisticsFromEncoded will construct a propertly typed statistics object +// initializing it with the provided information. +func NewInt32StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Int32Statistics { + ret := NewInt32Statistics(descr, mem) + ret.nvalues += nvalues + if hasNulls { + ret.incNulls(nulls) + } + if hasDistinct { + ret.incDistinct(distinct) + } + + if encodedMin != nil && len(encodedMin) > 0 { + ret.min = ret.plainDecode(encodedMin) + } + if encodedMax != nil && len(encodedMax) > 0 { + ret.max = ret.plainDecode(encodedMax) + } + ret.hasMinMax = hasMinMax + return ret +} + +func (s *Int32Statistics) plainEncode(src int32) []byte { + s.encoder.(encoding.Int32Encoder).Put([]int32{src}) + buf, err := s.encoder.FlushValues() + if err != nil { + panic(err) // recovered by Encode + } + defer buf.Release() + + out := make([]byte, buf.Len()) + copy(out, buf.Bytes()) + return out +} + +func (s *Int32Statistics) plainDecode(src []byte) int32 { + var buf [1]int32 + + decoder := encoding.NewDecoder(s.descr.PhysicalType(), parquet.Encodings.Plain, s.descr, s.mem) + decoder.SetData(1, src) + decoder.(encoding.Int32Decoder).Decode(buf[:]) + return buf[0] +} + +func (s *Int32Statistics) minval(a, b int32) int32 { + if s.less(a, b) { + return a + } + return b +} + +func (s *Int32Statistics) maxval(a, b int32) int32 { + if s.less(a, b) { + return b + } + return a +} + +// MinMaxEqual returns true if both stat objects have the same Min and Max values +func (s *Int32Statistics) MinMaxEqual(rhs *Int32Statistics) bool { + return s.equal(s.min, rhs.min) && s.equal(s.max, rhs.max) +} + +// Equals returns true only if both objects are the same type, have the same min and +// max values, null count, distinct count and number of values. +func (s *Int32Statistics) Equals(other TypedStatistics) bool { + if s.Type() != other.Type() { + return false + } + rhs, ok := other.(*Int32Statistics) + if !ok { + return false + } + + if s.HasMinMax() != rhs.HasMinMax() { + return false + } + return (s.hasMinMax && s.MinMaxEqual(rhs)) && + s.NullCount() == rhs.NullCount() && + s.DistinctCount() == rhs.DistinctCount() && + s.NumValues() == rhs.NumValues() +} + +func (s *Int32Statistics) getMinMax(values []int32) (min, max int32) { + if s.order == schema.SortSIGNED { + min, max = utils.GetMinMaxInt32(values) + } else { + umin, umax := utils.GetMinMaxUint32(arrow.Uint32Traits.CastFromBytes(arrow.Int32Traits.CastToBytes(values))) + min, max = int32(umin), int32(umax) + } + return +} + +func (s *Int32Statistics) getMinMaxSpaced(values []int32, validBits []byte, validBitsOffset int64) (min, max int32) { + min = s.defaultMin() + max = s.defaultMax() + var fn func([]int32) (int32, int32) + if s.order == schema.SortSIGNED { + fn = utils.GetMinMaxInt32 + } else { + fn = func(v []int32) (int32, int32) { + umin, umax := utils.GetMinMaxUint32(arrow.Uint32Traits.CastFromBytes(arrow.Int32Traits.CastToBytes(values))) + return int32(umin), int32(umax) + } + } + + if s.bitSetReader == nil { + s.bitSetReader = utils.NewSetBitRunReader(validBits, validBitsOffset, int64(len(values))) + } else { + s.bitSetReader.Reset(validBits, validBitsOffset, int64(len(values))) + } + + for { + run := s.bitSetReader.NextRun() + if run.Length == 0 { + break + } + localMin, localMax := fn(values[int(run.Pos):int(run.Pos+run.Length)]) + if min > localMin { + min = localMin + } + if max < localMax { + max = localMax + } + } + return +} + +func (s *Int32Statistics) Min() int32 { return s.min } +func (s *Int32Statistics) Max() int32 { return s.max } + +// Merge merges the stats from other into this stat object, updating +// the null count, distinct count, number of values and the min/max if +// appropriate. +func (s *Int32Statistics) Merge(other TypedStatistics) { + rhs, ok := other.(*Int32Statistics) + if !ok { + panic("incompatible stat type merge") + } + + s.statistics.merge(rhs) + if rhs.HasMinMax() { + s.SetMinMax(rhs.Min(), rhs.Max()) + } +} + +// Update is used to add more values to the current stat object, finding the +// min and max values etc. +func (s *Int32Statistics) Update(values []int32, numNull int64) { + s.incNulls(numNull) + s.nvalues += int64(len(values)) + + if len(values) == 0 { + return + } + + s.SetMinMax(s.getMinMax(values)) +} + +// UpdateSpaced is just like Update, but for spaced values using validBits to determine +// and skip null values. +func (s *Int32Statistics) UpdateSpaced(values []int32, validBits []byte, validBitsOffset, numNull int64) { + s.incNulls(numNull) + notnull := int64(len(values)) - numNull + s.nvalues += notnull + + if notnull == 0 { + return + } + + s.SetMinMax(s.getMinMaxSpaced(values, validBits, validBitsOffset)) +} + +// SetMinMax updates the min and max values only if they are not currently set +// or if argMin is less than the current min / argMax is greater than the current max +func (s *Int32Statistics) SetMinMax(argMin, argMax int32) { + maybeMinMax := s.cleanStat([2]int32{argMin, argMax}) + if maybeMinMax == nil { + return + } + + min := (*maybeMinMax)[0] + max := (*maybeMinMax)[1] + + if !s.hasMinMax { + s.hasMinMax = true + s.min = min + s.max = max + } else { + if !s.less(s.min, min) { + s.min = min + } + if s.less(s.max, max) { + s.max = max + } + } +} + +// EncodeMin returns the encoded min value with plain encoding. +// +// ByteArray stats do not include the length in the encoding. +func (s *Int32Statistics) EncodeMin() []byte { + if s.HasMinMax() { + return s.plainEncode(s.min) + } + return nil +} + +// EncodeMax returns the current encoded max value with plain encoding +// +// ByteArray stats do not include the length in the encoding +func (s *Int32Statistics) EncodeMax() []byte { + if s.HasMinMax() { + return s.plainEncode(s.max) + } + return nil +} + +// Encode returns a populated EncodedStatistics object +func (s *Int32Statistics) Encode() (enc EncodedStatistics, err error) { + defer func() { + if r := recover(); r != nil { + switch r := r.(type) { + case error: + err = r + case string: + err = xerrors.New(r) + default: + err = xerrors.Errorf("unknown error type thrown from panic: %v", r) + } + } + }() + if s.HasMinMax() { + enc.SetMax(s.EncodeMax()) + enc.SetMin(s.EncodeMin()) + } + if s.HasNullCount() { + enc.SetNullCount(s.NullCount()) + } + if s.HasDistinctCount() { + enc.SetDistinctCount(s.DistinctCount()) + } + return +} + +type minmaxPairInt64 [2]int64 + +// Int64Statistics is the typed interface for managing stats for a column +// of Int64 type. +type Int64Statistics struct { + statistics + min int64 + max int64 + + bitSetReader utils.SetBitRunReader +} + +// NewInt64Statistics constructs an appropriate stat object type using the +// given column descriptor and allocator. +// +// Panics if the physical type of descr is not parquet.Type.Int64 +func NewInt64Statistics(descr *schema.Column, mem memory.Allocator) *Int64Statistics { + if descr.PhysicalType() != parquet.Types.Int64 { + panic(xerrors.Errorf("parquet: invalid type %s for constructing a Int64 stat object", descr.PhysicalType())) + } + + return &Int64Statistics{ + statistics: statistics{ + descr: descr, + hasNullCount: true, + hasDistinctCount: true, + order: descr.SortOrder(), + encoder: encoding.NewEncoder(descr.PhysicalType(), parquet.Encodings.Plain, false, descr, mem), + mem: mem, + }, + } +} + +// NewInt64StatisticsFromEncoded will construct a propertly typed statistics object +// initializing it with the provided information. +func NewInt64StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Int64Statistics { + ret := NewInt64Statistics(descr, mem) + ret.nvalues += nvalues + if hasNulls { + ret.incNulls(nulls) + } + if hasDistinct { + ret.incDistinct(distinct) + } + + if encodedMin != nil && len(encodedMin) > 0 { + ret.min = ret.plainDecode(encodedMin) + } + if encodedMax != nil && len(encodedMax) > 0 { + ret.max = ret.plainDecode(encodedMax) + } + ret.hasMinMax = hasMinMax + return ret +} + +func (s *Int64Statistics) plainEncode(src int64) []byte { + s.encoder.(encoding.Int64Encoder).Put([]int64{src}) + buf, err := s.encoder.FlushValues() + if err != nil { + panic(err) // recovered by Encode + } + defer buf.Release() + + out := make([]byte, buf.Len()) + copy(out, buf.Bytes()) + return out +} + +func (s *Int64Statistics) plainDecode(src []byte) int64 { + var buf [1]int64 + + decoder := encoding.NewDecoder(s.descr.PhysicalType(), parquet.Encodings.Plain, s.descr, s.mem) + decoder.SetData(1, src) + decoder.(encoding.Int64Decoder).Decode(buf[:]) + return buf[0] +} + +func (s *Int64Statistics) minval(a, b int64) int64 { + if s.less(a, b) { + return a + } + return b +} + +func (s *Int64Statistics) maxval(a, b int64) int64 { + if s.less(a, b) { + return b + } + return a +} + +// MinMaxEqual returns true if both stat objects have the same Min and Max values +func (s *Int64Statistics) MinMaxEqual(rhs *Int64Statistics) bool { + return s.equal(s.min, rhs.min) && s.equal(s.max, rhs.max) +} + +// Equals returns true only if both objects are the same type, have the same min and +// max values, null count, distinct count and number of values. +func (s *Int64Statistics) Equals(other TypedStatistics) bool { + if s.Type() != other.Type() { + return false + } + rhs, ok := other.(*Int64Statistics) + if !ok { + return false + } + + if s.HasMinMax() != rhs.HasMinMax() { + return false + } + return (s.hasMinMax && s.MinMaxEqual(rhs)) && + s.NullCount() == rhs.NullCount() && + s.DistinctCount() == rhs.DistinctCount() && + s.NumValues() == rhs.NumValues() +} + +func (s *Int64Statistics) getMinMax(values []int64) (min, max int64) { + if s.order == schema.SortSIGNED { + min, max = utils.GetMinMaxInt64(values) + } else { + umin, umax := utils.GetMinMaxUint64(arrow.Uint64Traits.CastFromBytes(arrow.Int64Traits.CastToBytes(values))) + min, max = int64(umin), int64(umax) + } + return +} + +func (s *Int64Statistics) getMinMaxSpaced(values []int64, validBits []byte, validBitsOffset int64) (min, max int64) { + min = s.defaultMin() + max = s.defaultMax() + var fn func([]int64) (int64, int64) + if s.order == schema.SortSIGNED { + fn = utils.GetMinMaxInt64 + } else { + fn = func(v []int64) (int64, int64) { + umin, umax := utils.GetMinMaxUint64(arrow.Uint64Traits.CastFromBytes(arrow.Int64Traits.CastToBytes(values))) + return int64(umin), int64(umax) + } + } + + if s.bitSetReader == nil { + s.bitSetReader = utils.NewSetBitRunReader(validBits, validBitsOffset, int64(len(values))) + } else { + s.bitSetReader.Reset(validBits, validBitsOffset, int64(len(values))) + } + + for { + run := s.bitSetReader.NextRun() + if run.Length == 0 { + break + } + localMin, localMax := fn(values[int(run.Pos):int(run.Pos+run.Length)]) + if min > localMin { + min = localMin + } + if max < localMax { + max = localMax + } + } + return +} + +func (s *Int64Statistics) Min() int64 { return s.min } +func (s *Int64Statistics) Max() int64 { return s.max } + +// Merge merges the stats from other into this stat object, updating +// the null count, distinct count, number of values and the min/max if +// appropriate. +func (s *Int64Statistics) Merge(other TypedStatistics) { + rhs, ok := other.(*Int64Statistics) + if !ok { + panic("incompatible stat type merge") + } + + s.statistics.merge(rhs) + if rhs.HasMinMax() { + s.SetMinMax(rhs.Min(), rhs.Max()) + } +} + +// Update is used to add more values to the current stat object, finding the +// min and max values etc. +func (s *Int64Statistics) Update(values []int64, numNull int64) { + s.incNulls(numNull) + s.nvalues += int64(len(values)) + + if len(values) == 0 { + return + } + + s.SetMinMax(s.getMinMax(values)) +} + +// UpdateSpaced is just like Update, but for spaced values using validBits to determine +// and skip null values. +func (s *Int64Statistics) UpdateSpaced(values []int64, validBits []byte, validBitsOffset, numNull int64) { + s.incNulls(numNull) + notnull := int64(len(values)) - numNull + s.nvalues += notnull + + if notnull == 0 { + return + } + + s.SetMinMax(s.getMinMaxSpaced(values, validBits, validBitsOffset)) +} + +// SetMinMax updates the min and max values only if they are not currently set +// or if argMin is less than the current min / argMax is greater than the current max +func (s *Int64Statistics) SetMinMax(argMin, argMax int64) { + maybeMinMax := s.cleanStat([2]int64{argMin, argMax}) + if maybeMinMax == nil { + return + } + + min := (*maybeMinMax)[0] + max := (*maybeMinMax)[1] + + if !s.hasMinMax { + s.hasMinMax = true + s.min = min + s.max = max + } else { + if !s.less(s.min, min) { + s.min = min + } + if s.less(s.max, max) { + s.max = max + } + } +} + +// EncodeMin returns the encoded min value with plain encoding. +// +// ByteArray stats do not include the length in the encoding. +func (s *Int64Statistics) EncodeMin() []byte { + if s.HasMinMax() { + return s.plainEncode(s.min) + } + return nil +} + +// EncodeMax returns the current encoded max value with plain encoding +// +// ByteArray stats do not include the length in the encoding +func (s *Int64Statistics) EncodeMax() []byte { + if s.HasMinMax() { + return s.plainEncode(s.max) + } + return nil +} + +// Encode returns a populated EncodedStatistics object +func (s *Int64Statistics) Encode() (enc EncodedStatistics, err error) { + defer func() { + if r := recover(); r != nil { + switch r := r.(type) { + case error: + err = r + case string: + err = xerrors.New(r) + default: + err = xerrors.Errorf("unknown error type thrown from panic: %v", r) + } + } + }() + if s.HasMinMax() { + enc.SetMax(s.EncodeMax()) + enc.SetMin(s.EncodeMin()) + } + if s.HasNullCount() { + enc.SetNullCount(s.NullCount()) + } + if s.HasDistinctCount() { + enc.SetDistinctCount(s.DistinctCount()) + } + return +} + +type minmaxPairInt96 [2]parquet.Int96 + +// Int96Statistics is the typed interface for managing stats for a column +// of Int96 type. +type Int96Statistics struct { + statistics + min parquet.Int96 + max parquet.Int96 + + bitSetReader utils.SetBitRunReader +} + +// NewInt96Statistics constructs an appropriate stat object type using the +// given column descriptor and allocator. +// +// Panics if the physical type of descr is not parquet.Type.Int96 +func NewInt96Statistics(descr *schema.Column, mem memory.Allocator) *Int96Statistics { + if descr.PhysicalType() != parquet.Types.Int96 { + panic(xerrors.Errorf("parquet: invalid type %s for constructing a Int96 stat object", descr.PhysicalType())) + } + + return &Int96Statistics{ + statistics: statistics{ + descr: descr, + hasNullCount: true, + hasDistinctCount: true, + order: descr.SortOrder(), + encoder: encoding.NewEncoder(descr.PhysicalType(), parquet.Encodings.Plain, false, descr, mem), + mem: mem, + }, + } +} + +// NewInt96StatisticsFromEncoded will construct a propertly typed statistics object +// initializing it with the provided information. +func NewInt96StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Int96Statistics { + ret := NewInt96Statistics(descr, mem) + ret.nvalues += nvalues + if hasNulls { + ret.incNulls(nulls) + } + if hasDistinct { + ret.incDistinct(distinct) + } + + if encodedMin != nil && len(encodedMin) > 0 { + ret.min = ret.plainDecode(encodedMin) + } + if encodedMax != nil && len(encodedMax) > 0 { + ret.max = ret.plainDecode(encodedMax) + } + ret.hasMinMax = hasMinMax + return ret +} + +func (s *Int96Statistics) plainEncode(src parquet.Int96) []byte { + s.encoder.(encoding.Int96Encoder).Put([]parquet.Int96{src}) + buf, err := s.encoder.FlushValues() + if err != nil { + panic(err) // recovered by Encode + } + defer buf.Release() + + out := make([]byte, buf.Len()) + copy(out, buf.Bytes()) + return out +} + +func (s *Int96Statistics) plainDecode(src []byte) parquet.Int96 { + var buf [1]parquet.Int96 + + decoder := encoding.NewDecoder(s.descr.PhysicalType(), parquet.Encodings.Plain, s.descr, s.mem) + decoder.SetData(1, src) + decoder.(encoding.Int96Decoder).Decode(buf[:]) + return buf[0] +} + +func (s *Int96Statistics) minval(a, b parquet.Int96) parquet.Int96 { + if s.less(a, b) { + return a + } + return b +} + +func (s *Int96Statistics) maxval(a, b parquet.Int96) parquet.Int96 { + if s.less(a, b) { + return b + } + return a +} + +// MinMaxEqual returns true if both stat objects have the same Min and Max values +func (s *Int96Statistics) MinMaxEqual(rhs *Int96Statistics) bool { + return s.equal(s.min, rhs.min) && s.equal(s.max, rhs.max) +} + +// Equals returns true only if both objects are the same type, have the same min and +// max values, null count, distinct count and number of values. +func (s *Int96Statistics) Equals(other TypedStatistics) bool { + if s.Type() != other.Type() { + return false + } + rhs, ok := other.(*Int96Statistics) + if !ok { + return false + } + + if s.HasMinMax() != rhs.HasMinMax() { + return false + } + return (s.hasMinMax && s.MinMaxEqual(rhs)) && + s.NullCount() == rhs.NullCount() && + s.DistinctCount() == rhs.DistinctCount() && + s.NumValues() == rhs.NumValues() +} + +func (s *Int96Statistics) getMinMax(values []parquet.Int96) (min, max parquet.Int96) { + defMin := s.defaultMin() + defMax := s.defaultMax() + + min = defMin + max = defMax + + for _, v := range values { + min = s.minval(min, v) + max = s.maxval(max, v) + } + return +} + +func (s *Int96Statistics) getMinMaxSpaced(values []parquet.Int96, validBits []byte, validBitsOffset int64) (min, max parquet.Int96) { + min = s.defaultMin() + max = s.defaultMax() + + if s.bitSetReader == nil { + s.bitSetReader = utils.NewSetBitRunReader(validBits, validBitsOffset, int64(len(values))) + } else { + s.bitSetReader.Reset(validBits, validBitsOffset, int64(len(values))) + } + + for { + run := s.bitSetReader.NextRun() + if run.Length == 0 { + break + } + for _, v := range values[int(run.Pos):int(run.Pos+run.Length)] { + min = s.minval(min, v) + max = s.maxval(max, v) + } + } + return +} + +func (s *Int96Statistics) Min() parquet.Int96 { return s.min } +func (s *Int96Statistics) Max() parquet.Int96 { return s.max } + +// Merge merges the stats from other into this stat object, updating +// the null count, distinct count, number of values and the min/max if +// appropriate. +func (s *Int96Statistics) Merge(other TypedStatistics) { + rhs, ok := other.(*Int96Statistics) + if !ok { + panic("incompatible stat type merge") + } + + s.statistics.merge(rhs) + if rhs.HasMinMax() { + s.SetMinMax(rhs.Min(), rhs.Max()) + } +} + +// Update is used to add more values to the current stat object, finding the +// min and max values etc. +func (s *Int96Statistics) Update(values []parquet.Int96, numNull int64) { + s.incNulls(numNull) + s.nvalues += int64(len(values)) + + if len(values) == 0 { + return + } + + s.SetMinMax(s.getMinMax(values)) +} + +// UpdateSpaced is just like Update, but for spaced values using validBits to determine +// and skip null values. +func (s *Int96Statistics) UpdateSpaced(values []parquet.Int96, validBits []byte, validBitsOffset, numNull int64) { + s.incNulls(numNull) + notnull := int64(len(values)) - numNull + s.nvalues += notnull + + if notnull == 0 { + return + } + + s.SetMinMax(s.getMinMaxSpaced(values, validBits, validBitsOffset)) +} + +// SetMinMax updates the min and max values only if they are not currently set +// or if argMin is less than the current min / argMax is greater than the current max +func (s *Int96Statistics) SetMinMax(argMin, argMax parquet.Int96) { + maybeMinMax := s.cleanStat([2]parquet.Int96{argMin, argMax}) + if maybeMinMax == nil { + return + } + + min := (*maybeMinMax)[0] + max := (*maybeMinMax)[1] + + if !s.hasMinMax { + s.hasMinMax = true + s.min = min + s.max = max + } else { + if !s.less(s.min, min) { + s.min = min + } + if s.less(s.max, max) { + s.max = max + } + } +} + +// EncodeMin returns the encoded min value with plain encoding. +// +// ByteArray stats do not include the length in the encoding. +func (s *Int96Statistics) EncodeMin() []byte { + if s.HasMinMax() { + return s.plainEncode(s.min) + } + return nil +} + +// EncodeMax returns the current encoded max value with plain encoding +// +// ByteArray stats do not include the length in the encoding +func (s *Int96Statistics) EncodeMax() []byte { + if s.HasMinMax() { + return s.plainEncode(s.max) + } + return nil +} + +// Encode returns a populated EncodedStatistics object +func (s *Int96Statistics) Encode() (enc EncodedStatistics, err error) { + defer func() { + if r := recover(); r != nil { + switch r := r.(type) { + case error: + err = r + case string: + err = xerrors.New(r) + default: + err = xerrors.Errorf("unknown error type thrown from panic: %v", r) + } + } + }() + if s.HasMinMax() { + enc.SetMax(s.EncodeMax()) + enc.SetMin(s.EncodeMin()) + } + if s.HasNullCount() { + enc.SetNullCount(s.NullCount()) + } + if s.HasDistinctCount() { + enc.SetDistinctCount(s.DistinctCount()) + } + return +} + +type minmaxPairFloat32 [2]float32 + +// Float32Statistics is the typed interface for managing stats for a column +// of Float32 type. +type Float32Statistics struct { + statistics + min float32 + max float32 + + bitSetReader utils.SetBitRunReader +} + +// NewFloat32Statistics constructs an appropriate stat object type using the +// given column descriptor and allocator. +// +// Panics if the physical type of descr is not parquet.Type.Float +func NewFloat32Statistics(descr *schema.Column, mem memory.Allocator) *Float32Statistics { + if descr.PhysicalType() != parquet.Types.Float { + panic(xerrors.Errorf("parquet: invalid type %s for constructing a Float32 stat object", descr.PhysicalType())) + } + + return &Float32Statistics{ + statistics: statistics{ + descr: descr, + hasNullCount: true, + hasDistinctCount: true, + order: descr.SortOrder(), + encoder: encoding.NewEncoder(descr.PhysicalType(), parquet.Encodings.Plain, false, descr, mem), + mem: mem, + }, + } +} + +// NewFloat32StatisticsFromEncoded will construct a propertly typed statistics object +// initializing it with the provided information. +func NewFloat32StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Float32Statistics { + ret := NewFloat32Statistics(descr, mem) + ret.nvalues += nvalues + if hasNulls { + ret.incNulls(nulls) + } + if hasDistinct { + ret.incDistinct(distinct) + } + + if encodedMin != nil && len(encodedMin) > 0 { + ret.min = ret.plainDecode(encodedMin) + } + if encodedMax != nil && len(encodedMax) > 0 { + ret.max = ret.plainDecode(encodedMax) + } + ret.hasMinMax = hasMinMax + return ret +} + +func (s *Float32Statistics) plainEncode(src float32) []byte { + s.encoder.(encoding.Float32Encoder).Put([]float32{src}) + buf, err := s.encoder.FlushValues() + if err != nil { + panic(err) // recovered by Encode + } + defer buf.Release() + + out := make([]byte, buf.Len()) + copy(out, buf.Bytes()) + return out +} + +func (s *Float32Statistics) plainDecode(src []byte) float32 { + var buf [1]float32 + + decoder := encoding.NewDecoder(s.descr.PhysicalType(), parquet.Encodings.Plain, s.descr, s.mem) + decoder.SetData(1, src) + decoder.(encoding.Float32Decoder).Decode(buf[:]) + return buf[0] +} + +func (s *Float32Statistics) minval(a, b float32) float32 { + if s.less(a, b) { + return a + } + return b +} + +func (s *Float32Statistics) maxval(a, b float32) float32 { + if s.less(a, b) { + return b + } + return a +} + +// MinMaxEqual returns true if both stat objects have the same Min and Max values +func (s *Float32Statistics) MinMaxEqual(rhs *Float32Statistics) bool { + return s.equal(s.min, rhs.min) && s.equal(s.max, rhs.max) +} + +// Equals returns true only if both objects are the same type, have the same min and +// max values, null count, distinct count and number of values. +func (s *Float32Statistics) Equals(other TypedStatistics) bool { + if s.Type() != other.Type() { + return false + } + rhs, ok := other.(*Float32Statistics) + if !ok { + return false + } + + if s.HasMinMax() != rhs.HasMinMax() { + return false + } + return (s.hasMinMax && s.MinMaxEqual(rhs)) && + s.NullCount() == rhs.NullCount() && + s.DistinctCount() == rhs.DistinctCount() && + s.NumValues() == rhs.NumValues() +} + +func (s *Float32Statistics) coalesce(val, fallback float32) float32 { + if math.IsNaN(float64(val)) { + return fallback + } + return val +} + +func (s *Float32Statistics) getMinMax(values []float32) (min, max float32) { + defMin := s.defaultMin() + defMax := s.defaultMax() + + min = defMin + max = defMax + + for _, v := range values { + min = s.minval(min, s.coalesce(v, defMin)) + max = s.maxval(max, s.coalesce(v, defMax)) + } + return +} + +func (s *Float32Statistics) getMinMaxSpaced(values []float32, validBits []byte, validBitsOffset int64) (min, max float32) { + min = s.defaultMin() + max = s.defaultMax() + + if s.bitSetReader == nil { + s.bitSetReader = utils.NewSetBitRunReader(validBits, validBitsOffset, int64(len(values))) + } else { + s.bitSetReader.Reset(validBits, validBitsOffset, int64(len(values))) + } + + for { + run := s.bitSetReader.NextRun() + if run.Length == 0 { + break + } + for _, v := range values[int(run.Pos):int(run.Pos+run.Length)] { + min = s.minval(min, coalesce(v, s.defaultMin()).(float32)) + max = s.maxval(max, coalesce(v, s.defaultMax()).(float32)) + } + } + return +} + +func (s *Float32Statistics) Min() float32 { return s.min } +func (s *Float32Statistics) Max() float32 { return s.max } + +// Merge merges the stats from other into this stat object, updating +// the null count, distinct count, number of values and the min/max if +// appropriate. +func (s *Float32Statistics) Merge(other TypedStatistics) { + rhs, ok := other.(*Float32Statistics) + if !ok { + panic("incompatible stat type merge") + } + + s.statistics.merge(rhs) + if rhs.HasMinMax() { + s.SetMinMax(rhs.Min(), rhs.Max()) + } +} + +// Update is used to add more values to the current stat object, finding the +// min and max values etc. +func (s *Float32Statistics) Update(values []float32, numNull int64) { + s.incNulls(numNull) + s.nvalues += int64(len(values)) + + if len(values) == 0 { + return + } + + s.SetMinMax(s.getMinMax(values)) +} + +// UpdateSpaced is just like Update, but for spaced values using validBits to determine +// and skip null values. +func (s *Float32Statistics) UpdateSpaced(values []float32, validBits []byte, validBitsOffset, numNull int64) { + s.incNulls(numNull) + notnull := int64(len(values)) - numNull + s.nvalues += notnull + + if notnull == 0 { + return + } + + s.SetMinMax(s.getMinMaxSpaced(values, validBits, validBitsOffset)) +} + +// SetMinMax updates the min and max values only if they are not currently set +// or if argMin is less than the current min / argMax is greater than the current max +func (s *Float32Statistics) SetMinMax(argMin, argMax float32) { + maybeMinMax := s.cleanStat([2]float32{argMin, argMax}) + if maybeMinMax == nil { + return + } + + min := (*maybeMinMax)[0] + max := (*maybeMinMax)[1] + + if !s.hasMinMax { + s.hasMinMax = true + s.min = min + s.max = max + } else { + if !s.less(s.min, min) { + s.min = min + } + if s.less(s.max, max) { + s.max = max + } + } +} + +// EncodeMin returns the encoded min value with plain encoding. +// +// ByteArray stats do not include the length in the encoding. +func (s *Float32Statistics) EncodeMin() []byte { + if s.HasMinMax() { + return s.plainEncode(s.min) + } + return nil +} + +// EncodeMax returns the current encoded max value with plain encoding +// +// ByteArray stats do not include the length in the encoding +func (s *Float32Statistics) EncodeMax() []byte { + if s.HasMinMax() { + return s.plainEncode(s.max) + } + return nil +} + +// Encode returns a populated EncodedStatistics object +func (s *Float32Statistics) Encode() (enc EncodedStatistics, err error) { + defer func() { + if r := recover(); r != nil { + switch r := r.(type) { + case error: + err = r + case string: + err = xerrors.New(r) + default: + err = xerrors.Errorf("unknown error type thrown from panic: %v", r) + } + } + }() + if s.HasMinMax() { + enc.SetMax(s.EncodeMax()) + enc.SetMin(s.EncodeMin()) + } + if s.HasNullCount() { + enc.SetNullCount(s.NullCount()) + } + if s.HasDistinctCount() { + enc.SetDistinctCount(s.DistinctCount()) + } + return +} + +type minmaxPairFloat64 [2]float64 + +// Float64Statistics is the typed interface for managing stats for a column +// of Float64 type. +type Float64Statistics struct { + statistics + min float64 + max float64 + + bitSetReader utils.SetBitRunReader +} + +// NewFloat64Statistics constructs an appropriate stat object type using the +// given column descriptor and allocator. +// +// Panics if the physical type of descr is not parquet.Type.Double +func NewFloat64Statistics(descr *schema.Column, mem memory.Allocator) *Float64Statistics { + if descr.PhysicalType() != parquet.Types.Double { + panic(xerrors.Errorf("parquet: invalid type %s for constructing a Float64 stat object", descr.PhysicalType())) + } + + return &Float64Statistics{ + statistics: statistics{ + descr: descr, + hasNullCount: true, + hasDistinctCount: true, + order: descr.SortOrder(), + encoder: encoding.NewEncoder(descr.PhysicalType(), parquet.Encodings.Plain, false, descr, mem), + mem: mem, + }, + } +} + +// NewFloat64StatisticsFromEncoded will construct a propertly typed statistics object +// initializing it with the provided information. +func NewFloat64StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Float64Statistics { + ret := NewFloat64Statistics(descr, mem) + ret.nvalues += nvalues + if hasNulls { + ret.incNulls(nulls) + } + if hasDistinct { + ret.incDistinct(distinct) + } + + if encodedMin != nil && len(encodedMin) > 0 { + ret.min = ret.plainDecode(encodedMin) + } + if encodedMax != nil && len(encodedMax) > 0 { + ret.max = ret.plainDecode(encodedMax) + } + ret.hasMinMax = hasMinMax + return ret +} + +func (s *Float64Statistics) plainEncode(src float64) []byte { + s.encoder.(encoding.Float64Encoder).Put([]float64{src}) + buf, err := s.encoder.FlushValues() + if err != nil { + panic(err) // recovered by Encode + } + defer buf.Release() + + out := make([]byte, buf.Len()) + copy(out, buf.Bytes()) + return out +} + +func (s *Float64Statistics) plainDecode(src []byte) float64 { + var buf [1]float64 + + decoder := encoding.NewDecoder(s.descr.PhysicalType(), parquet.Encodings.Plain, s.descr, s.mem) + decoder.SetData(1, src) + decoder.(encoding.Float64Decoder).Decode(buf[:]) + return buf[0] +} + +func (s *Float64Statistics) minval(a, b float64) float64 { + if s.less(a, b) { + return a + } + return b +} + +func (s *Float64Statistics) maxval(a, b float64) float64 { + if s.less(a, b) { + return b + } + return a +} + +// MinMaxEqual returns true if both stat objects have the same Min and Max values +func (s *Float64Statistics) MinMaxEqual(rhs *Float64Statistics) bool { + return s.equal(s.min, rhs.min) && s.equal(s.max, rhs.max) +} + +// Equals returns true only if both objects are the same type, have the same min and +// max values, null count, distinct count and number of values. +func (s *Float64Statistics) Equals(other TypedStatistics) bool { + if s.Type() != other.Type() { + return false + } + rhs, ok := other.(*Float64Statistics) + if !ok { + return false + } + + if s.HasMinMax() != rhs.HasMinMax() { + return false + } + return (s.hasMinMax && s.MinMaxEqual(rhs)) && + s.NullCount() == rhs.NullCount() && + s.DistinctCount() == rhs.DistinctCount() && + s.NumValues() == rhs.NumValues() +} + +func (s *Float64Statistics) coalesce(val, fallback float64) float64 { + if math.IsNaN(float64(val)) { + return fallback + } + return val +} + +func (s *Float64Statistics) getMinMax(values []float64) (min, max float64) { + defMin := s.defaultMin() + defMax := s.defaultMax() + + min = defMin + max = defMax + + for _, v := range values { + min = s.minval(min, s.coalesce(v, defMin)) + max = s.maxval(max, s.coalesce(v, defMax)) + } + return +} + +func (s *Float64Statistics) getMinMaxSpaced(values []float64, validBits []byte, validBitsOffset int64) (min, max float64) { + min = s.defaultMin() + max = s.defaultMax() + + if s.bitSetReader == nil { + s.bitSetReader = utils.NewSetBitRunReader(validBits, validBitsOffset, int64(len(values))) + } else { + s.bitSetReader.Reset(validBits, validBitsOffset, int64(len(values))) + } + + for { + run := s.bitSetReader.NextRun() + if run.Length == 0 { + break + } + for _, v := range values[int(run.Pos):int(run.Pos+run.Length)] { + min = s.minval(min, coalesce(v, s.defaultMin()).(float64)) + max = s.maxval(max, coalesce(v, s.defaultMax()).(float64)) + } + } + return +} + +func (s *Float64Statistics) Min() float64 { return s.min } +func (s *Float64Statistics) Max() float64 { return s.max } + +// Merge merges the stats from other into this stat object, updating +// the null count, distinct count, number of values and the min/max if +// appropriate. +func (s *Float64Statistics) Merge(other TypedStatistics) { + rhs, ok := other.(*Float64Statistics) + if !ok { + panic("incompatible stat type merge") + } + + s.statistics.merge(rhs) + if rhs.HasMinMax() { + s.SetMinMax(rhs.Min(), rhs.Max()) + } +} + +// Update is used to add more values to the current stat object, finding the +// min and max values etc. +func (s *Float64Statistics) Update(values []float64, numNull int64) { + s.incNulls(numNull) + s.nvalues += int64(len(values)) + + if len(values) == 0 { + return + } + + s.SetMinMax(s.getMinMax(values)) +} + +// UpdateSpaced is just like Update, but for spaced values using validBits to determine +// and skip null values. +func (s *Float64Statistics) UpdateSpaced(values []float64, validBits []byte, validBitsOffset, numNull int64) { + s.incNulls(numNull) + notnull := int64(len(values)) - numNull + s.nvalues += notnull + + if notnull == 0 { + return + } + + s.SetMinMax(s.getMinMaxSpaced(values, validBits, validBitsOffset)) +} + +// SetMinMax updates the min and max values only if they are not currently set +// or if argMin is less than the current min / argMax is greater than the current max +func (s *Float64Statistics) SetMinMax(argMin, argMax float64) { + maybeMinMax := s.cleanStat([2]float64{argMin, argMax}) + if maybeMinMax == nil { + return + } + + min := (*maybeMinMax)[0] + max := (*maybeMinMax)[1] + + if !s.hasMinMax { + s.hasMinMax = true + s.min = min + s.max = max + } else { + if !s.less(s.min, min) { + s.min = min + } + if s.less(s.max, max) { + s.max = max + } + } +} + +// EncodeMin returns the encoded min value with plain encoding. +// +// ByteArray stats do not include the length in the encoding. +func (s *Float64Statistics) EncodeMin() []byte { + if s.HasMinMax() { + return s.plainEncode(s.min) + } + return nil +} + +// EncodeMax returns the current encoded max value with plain encoding +// +// ByteArray stats do not include the length in the encoding +func (s *Float64Statistics) EncodeMax() []byte { + if s.HasMinMax() { + return s.plainEncode(s.max) + } + return nil +} + +// Encode returns a populated EncodedStatistics object +func (s *Float64Statistics) Encode() (enc EncodedStatistics, err error) { + defer func() { + if r := recover(); r != nil { + switch r := r.(type) { + case error: + err = r + case string: + err = xerrors.New(r) + default: + err = xerrors.Errorf("unknown error type thrown from panic: %v", r) + } + } + }() + if s.HasMinMax() { + enc.SetMax(s.EncodeMax()) + enc.SetMin(s.EncodeMin()) + } + if s.HasNullCount() { + enc.SetNullCount(s.NullCount()) + } + if s.HasDistinctCount() { + enc.SetDistinctCount(s.DistinctCount()) + } + return +} + +type minmaxPairBoolean [2]bool + +// BooleanStatistics is the typed interface for managing stats for a column +// of Boolean type. +type BooleanStatistics struct { + statistics + min bool + max bool + + bitSetReader utils.SetBitRunReader +} + +// NewBooleanStatistics constructs an appropriate stat object type using the +// given column descriptor and allocator. +// +// Panics if the physical type of descr is not parquet.Type.Boolean +func NewBooleanStatistics(descr *schema.Column, mem memory.Allocator) *BooleanStatistics { + if descr.PhysicalType() != parquet.Types.Boolean { + panic(xerrors.Errorf("parquet: invalid type %s for constructing a Boolean stat object", descr.PhysicalType())) + } + + return &BooleanStatistics{ + statistics: statistics{ + descr: descr, + hasNullCount: true, + hasDistinctCount: true, + order: descr.SortOrder(), + encoder: encoding.NewEncoder(descr.PhysicalType(), parquet.Encodings.Plain, false, descr, mem), + mem: mem, + }, + } +} + +// NewBooleanStatisticsFromEncoded will construct a propertly typed statistics object +// initializing it with the provided information. +func NewBooleanStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *BooleanStatistics { + ret := NewBooleanStatistics(descr, mem) + ret.nvalues += nvalues + if hasNulls { + ret.incNulls(nulls) + } + if hasDistinct { + ret.incDistinct(distinct) + } + + if encodedMin != nil && len(encodedMin) > 0 { + ret.min = ret.plainDecode(encodedMin) + } + if encodedMax != nil && len(encodedMax) > 0 { + ret.max = ret.plainDecode(encodedMax) + } + ret.hasMinMax = hasMinMax + return ret +} + +func (s *BooleanStatistics) plainEncode(src bool) []byte { + s.encoder.(encoding.BooleanEncoder).Put([]bool{src}) + buf, err := s.encoder.FlushValues() + if err != nil { + panic(err) // recovered by Encode + } + defer buf.Release() + + out := make([]byte, buf.Len()) + copy(out, buf.Bytes()) + return out +} + +func (s *BooleanStatistics) plainDecode(src []byte) bool { + var buf [1]bool + + decoder := encoding.NewDecoder(s.descr.PhysicalType(), parquet.Encodings.Plain, s.descr, s.mem) + decoder.SetData(1, src) + decoder.(encoding.BooleanDecoder).Decode(buf[:]) + return buf[0] +} + +func (s *BooleanStatistics) minval(a, b bool) bool { + if s.less(a, b) { + return a + } + return b +} + +func (s *BooleanStatistics) maxval(a, b bool) bool { + if s.less(a, b) { + return b + } + return a +} + +// MinMaxEqual returns true if both stat objects have the same Min and Max values +func (s *BooleanStatistics) MinMaxEqual(rhs *BooleanStatistics) bool { + return s.equal(s.min, rhs.min) && s.equal(s.max, rhs.max) +} + +// Equals returns true only if both objects are the same type, have the same min and +// max values, null count, distinct count and number of values. +func (s *BooleanStatistics) Equals(other TypedStatistics) bool { + if s.Type() != other.Type() { + return false + } + rhs, ok := other.(*BooleanStatistics) + if !ok { + return false + } + + if s.HasMinMax() != rhs.HasMinMax() { + return false + } + return (s.hasMinMax && s.MinMaxEqual(rhs)) && + s.NullCount() == rhs.NullCount() && + s.DistinctCount() == rhs.DistinctCount() && + s.NumValues() == rhs.NumValues() +} + +func (s *BooleanStatistics) getMinMax(values []bool) (min, max bool) { + defMin := s.defaultMin() + defMax := s.defaultMax() + + min = defMin + max = defMax + + for _, v := range values { + min = s.minval(min, v) + max = s.maxval(max, v) + } + return +} + +func (s *BooleanStatistics) getMinMaxSpaced(values []bool, validBits []byte, validBitsOffset int64) (min, max bool) { + min = s.defaultMin() + max = s.defaultMax() + + if s.bitSetReader == nil { + s.bitSetReader = utils.NewSetBitRunReader(validBits, validBitsOffset, int64(len(values))) + } else { + s.bitSetReader.Reset(validBits, validBitsOffset, int64(len(values))) + } + + for { + run := s.bitSetReader.NextRun() + if run.Length == 0 { + break + } + for _, v := range values[int(run.Pos):int(run.Pos+run.Length)] { + min = s.minval(min, v) + max = s.maxval(max, v) + } + } + return +} + +func (s *BooleanStatistics) Min() bool { return s.min } +func (s *BooleanStatistics) Max() bool { return s.max } + +// Merge merges the stats from other into this stat object, updating +// the null count, distinct count, number of values and the min/max if +// appropriate. +func (s *BooleanStatistics) Merge(other TypedStatistics) { + rhs, ok := other.(*BooleanStatistics) + if !ok { + panic("incompatible stat type merge") + } + + s.statistics.merge(rhs) + if rhs.HasMinMax() { + s.SetMinMax(rhs.Min(), rhs.Max()) + } +} + +// Update is used to add more values to the current stat object, finding the +// min and max values etc. +func (s *BooleanStatistics) Update(values []bool, numNull int64) { + s.incNulls(numNull) + s.nvalues += int64(len(values)) + + if len(values) == 0 { + return + } + + s.SetMinMax(s.getMinMax(values)) +} + +// UpdateSpaced is just like Update, but for spaced values using validBits to determine +// and skip null values. +func (s *BooleanStatistics) UpdateSpaced(values []bool, validBits []byte, validBitsOffset, numNull int64) { + s.incNulls(numNull) + notnull := int64(len(values)) - numNull + s.nvalues += notnull + + if notnull == 0 { + return + } + + s.SetMinMax(s.getMinMaxSpaced(values, validBits, validBitsOffset)) +} + +// SetMinMax updates the min and max values only if they are not currently set +// or if argMin is less than the current min / argMax is greater than the current max +func (s *BooleanStatistics) SetMinMax(argMin, argMax bool) { + maybeMinMax := s.cleanStat([2]bool{argMin, argMax}) + if maybeMinMax == nil { + return + } + + min := (*maybeMinMax)[0] + max := (*maybeMinMax)[1] + + if !s.hasMinMax { + s.hasMinMax = true + s.min = min + s.max = max + } else { + if !s.less(s.min, min) { + s.min = min + } + if s.less(s.max, max) { + s.max = max + } + } +} + +// EncodeMin returns the encoded min value with plain encoding. +// +// ByteArray stats do not include the length in the encoding. +func (s *BooleanStatistics) EncodeMin() []byte { + if s.HasMinMax() { + return s.plainEncode(s.min) + } + return nil +} + +// EncodeMax returns the current encoded max value with plain encoding +// +// ByteArray stats do not include the length in the encoding +func (s *BooleanStatistics) EncodeMax() []byte { + if s.HasMinMax() { + return s.plainEncode(s.max) + } + return nil +} + +// Encode returns a populated EncodedStatistics object +func (s *BooleanStatistics) Encode() (enc EncodedStatistics, err error) { + defer func() { + if r := recover(); r != nil { + switch r := r.(type) { + case error: + err = r + case string: + err = xerrors.New(r) + default: + err = xerrors.Errorf("unknown error type thrown from panic: %v", r) + } + } + }() + if s.HasMinMax() { + enc.SetMax(s.EncodeMax()) + enc.SetMin(s.EncodeMin()) + } + if s.HasNullCount() { + enc.SetNullCount(s.NullCount()) + } + if s.HasDistinctCount() { + enc.SetDistinctCount(s.DistinctCount()) + } + return +} + +type minmaxPairByteArray [2]parquet.ByteArray + +// ByteArrayStatistics is the typed interface for managing stats for a column +// of ByteArray type. +type ByteArrayStatistics struct { + statistics + min parquet.ByteArray + max parquet.ByteArray + + bitSetReader utils.SetBitRunReader +} + +// NewByteArrayStatistics constructs an appropriate stat object type using the +// given column descriptor and allocator. +// +// Panics if the physical type of descr is not parquet.Type.ByteArray +func NewByteArrayStatistics(descr *schema.Column, mem memory.Allocator) *ByteArrayStatistics { + if descr.PhysicalType() != parquet.Types.ByteArray { + panic(xerrors.Errorf("parquet: invalid type %s for constructing a ByteArray stat object", descr.PhysicalType())) + } + + return &ByteArrayStatistics{ + statistics: statistics{ + descr: descr, + hasNullCount: true, + hasDistinctCount: true, + order: descr.SortOrder(), + encoder: encoding.NewEncoder(descr.PhysicalType(), parquet.Encodings.Plain, false, descr, mem), + mem: mem, + }, + + min: make([]byte, 0), + max: make([]byte, 0), + } +} + +// NewByteArrayStatisticsFromEncoded will construct a propertly typed statistics object +// initializing it with the provided information. +func NewByteArrayStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *ByteArrayStatistics { + ret := NewByteArrayStatistics(descr, mem) + ret.nvalues += nvalues + if hasNulls { + ret.incNulls(nulls) + } + if hasDistinct { + ret.incDistinct(distinct) + } + + if encodedMin != nil && len(encodedMin) > 0 { + ret.min = ret.plainDecode(encodedMin) + } + if encodedMax != nil && len(encodedMax) > 0 { + ret.max = ret.plainDecode(encodedMax) + } + ret.hasMinMax = hasMinMax + return ret +} + +func (s *ByteArrayStatistics) plainEncode(src parquet.ByteArray) []byte { + return src +} + +func (s *ByteArrayStatistics) plainDecode(src []byte) parquet.ByteArray { + return src +} + +func (s *ByteArrayStatistics) minval(a, b parquet.ByteArray) parquet.ByteArray { + switch { + case a == nil: + return b + case b == nil: + return a + case s.less(a, b): + return a + default: + return b + } +} + +func (s *ByteArrayStatistics) maxval(a, b parquet.ByteArray) parquet.ByteArray { + switch { + case a == nil: + return b + case b == nil: + return a + case s.less(a, b): + return b + default: + return a + } +} + +// MinMaxEqual returns true if both stat objects have the same Min and Max values +func (s *ByteArrayStatistics) MinMaxEqual(rhs *ByteArrayStatistics) bool { + return s.equal(s.min, rhs.min) && s.equal(s.max, rhs.max) +} + +// Equals returns true only if both objects are the same type, have the same min and +// max values, null count, distinct count and number of values. +func (s *ByteArrayStatistics) Equals(other TypedStatistics) bool { + if s.Type() != other.Type() { + return false + } + rhs, ok := other.(*ByteArrayStatistics) + if !ok { + return false + } + + if s.HasMinMax() != rhs.HasMinMax() { + return false + } + return (s.hasMinMax && s.MinMaxEqual(rhs)) && + s.NullCount() == rhs.NullCount() && + s.DistinctCount() == rhs.DistinctCount() && + s.NumValues() == rhs.NumValues() +} + +func (s *ByteArrayStatistics) getMinMax(values []parquet.ByteArray) (min, max parquet.ByteArray) { + defMin := s.defaultMin() + defMax := s.defaultMax() + + min = defMin + max = defMax + + for _, v := range values { + min = s.minval(min, v) + max = s.maxval(max, v) + } + return +} + +func (s *ByteArrayStatistics) getMinMaxSpaced(values []parquet.ByteArray, validBits []byte, validBitsOffset int64) (min, max parquet.ByteArray) { + min = s.defaultMin() + max = s.defaultMax() + + if s.bitSetReader == nil { + s.bitSetReader = utils.NewSetBitRunReader(validBits, validBitsOffset, int64(len(values))) + } else { + s.bitSetReader.Reset(validBits, validBitsOffset, int64(len(values))) + } + + for { + run := s.bitSetReader.NextRun() + if run.Length == 0 { + break + } + for _, v := range values[int(run.Pos):int(run.Pos+run.Length)] { + min = s.minval(min, v) + max = s.maxval(max, v) + } + } + return +} + +func (s *ByteArrayStatistics) Min() parquet.ByteArray { return s.min } +func (s *ByteArrayStatistics) Max() parquet.ByteArray { return s.max } + +// Merge merges the stats from other into this stat object, updating +// the null count, distinct count, number of values and the min/max if +// appropriate. +func (s *ByteArrayStatistics) Merge(other TypedStatistics) { + rhs, ok := other.(*ByteArrayStatistics) + if !ok { + panic("incompatible stat type merge") + } + + s.statistics.merge(rhs) + if rhs.HasMinMax() { + s.SetMinMax(rhs.Min(), rhs.Max()) + } +} + +// Update is used to add more values to the current stat object, finding the +// min and max values etc. +func (s *ByteArrayStatistics) Update(values []parquet.ByteArray, numNull int64) { + s.incNulls(numNull) + s.nvalues += int64(len(values)) + + if len(values) == 0 { + return + } + + s.SetMinMax(s.getMinMax(values)) +} + +// UpdateSpaced is just like Update, but for spaced values using validBits to determine +// and skip null values. +func (s *ByteArrayStatistics) UpdateSpaced(values []parquet.ByteArray, validBits []byte, validBitsOffset, numNull int64) { + s.incNulls(numNull) + notnull := int64(len(values)) - numNull + s.nvalues += notnull + + if notnull == 0 { + return + } + + s.SetMinMax(s.getMinMaxSpaced(values, validBits, validBitsOffset)) +} + +// SetMinMax updates the min and max values only if they are not currently set +// or if argMin is less than the current min / argMax is greater than the current max +func (s *ByteArrayStatistics) SetMinMax(argMin, argMax parquet.ByteArray) { + maybeMinMax := s.cleanStat([2]parquet.ByteArray{argMin, argMax}) + if maybeMinMax == nil { + return + } + + min := (*maybeMinMax)[0] + max := (*maybeMinMax)[1] + + if !s.hasMinMax { + s.hasMinMax = true + s.min = min + s.max = max + } else { + if !s.less(s.min, min) { + s.min = min + } + if s.less(s.max, max) { + s.max = max + } + } +} + +// EncodeMin returns the encoded min value with plain encoding. +// +// ByteArray stats do not include the length in the encoding. +func (s *ByteArrayStatistics) EncodeMin() []byte { + if s.HasMinMax() { + return s.plainEncode(s.min) + } + return nil +} + +// EncodeMax returns the current encoded max value with plain encoding +// +// ByteArray stats do not include the length in the encoding +func (s *ByteArrayStatistics) EncodeMax() []byte { + if s.HasMinMax() { + return s.plainEncode(s.max) + } + return nil +} + +// Encode returns a populated EncodedStatistics object +func (s *ByteArrayStatistics) Encode() (enc EncodedStatistics, err error) { + defer func() { + if r := recover(); r != nil { + switch r := r.(type) { + case error: + err = r + case string: + err = xerrors.New(r) + default: + err = xerrors.Errorf("unknown error type thrown from panic: %v", r) + } + } + }() + if s.HasMinMax() { + enc.SetMax(s.EncodeMax()) + enc.SetMin(s.EncodeMin()) + } + if s.HasNullCount() { + enc.SetNullCount(s.NullCount()) + } + if s.HasDistinctCount() { + enc.SetDistinctCount(s.DistinctCount()) + } + return +} + +type minmaxPairFixedLenByteArray [2]parquet.FixedLenByteArray + +// FixedLenByteArrayStatistics is the typed interface for managing stats for a column +// of FixedLenByteArray type. +type FixedLenByteArrayStatistics struct { + statistics + min parquet.FixedLenByteArray + max parquet.FixedLenByteArray + + bitSetReader utils.SetBitRunReader +} + +// NewFixedLenByteArrayStatistics constructs an appropriate stat object type using the +// given column descriptor and allocator. +// +// Panics if the physical type of descr is not parquet.Type.FixedLenByteArray +func NewFixedLenByteArrayStatistics(descr *schema.Column, mem memory.Allocator) *FixedLenByteArrayStatistics { + if descr.PhysicalType() != parquet.Types.FixedLenByteArray { + panic(xerrors.Errorf("parquet: invalid type %s for constructing a FixedLenByteArray stat object", descr.PhysicalType())) + } + + return &FixedLenByteArrayStatistics{ + statistics: statistics{ + descr: descr, + hasNullCount: true, + hasDistinctCount: true, + order: descr.SortOrder(), + encoder: encoding.NewEncoder(descr.PhysicalType(), parquet.Encodings.Plain, false, descr, mem), + mem: mem, + }, + } +} + +// NewFixedLenByteArrayStatisticsFromEncoded will construct a propertly typed statistics object +// initializing it with the provided information. +func NewFixedLenByteArrayStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *FixedLenByteArrayStatistics { + ret := NewFixedLenByteArrayStatistics(descr, mem) + ret.nvalues += nvalues + if hasNulls { + ret.incNulls(nulls) + } + if hasDistinct { + ret.incDistinct(distinct) + } + + if encodedMin != nil && len(encodedMin) > 0 { + ret.min = ret.plainDecode(encodedMin) + } + if encodedMax != nil && len(encodedMax) > 0 { + ret.max = ret.plainDecode(encodedMax) + } + ret.hasMinMax = hasMinMax + return ret +} + +func (s *FixedLenByteArrayStatistics) plainEncode(src parquet.FixedLenByteArray) []byte { + s.encoder.(encoding.FixedLenByteArrayEncoder).Put([]parquet.FixedLenByteArray{src}) + buf, err := s.encoder.FlushValues() + if err != nil { + panic(err) // recovered by Encode + } + defer buf.Release() + + out := make([]byte, buf.Len()) + copy(out, buf.Bytes()) + return out +} + +func (s *FixedLenByteArrayStatistics) plainDecode(src []byte) parquet.FixedLenByteArray { + var buf [1]parquet.FixedLenByteArray + + decoder := encoding.NewDecoder(s.descr.PhysicalType(), parquet.Encodings.Plain, s.descr, s.mem) + decoder.SetData(1, src) + decoder.(encoding.FixedLenByteArrayDecoder).Decode(buf[:]) + return buf[0] +} + +func (s *FixedLenByteArrayStatistics) minval(a, b parquet.FixedLenByteArray) parquet.FixedLenByteArray { + switch { + case a == nil: + return b + case b == nil: + return a + case s.less(a, b): + return a + default: + return b + } +} + +func (s *FixedLenByteArrayStatistics) maxval(a, b parquet.FixedLenByteArray) parquet.FixedLenByteArray { + switch { + case a == nil: + return b + case b == nil: + return a + case s.less(a, b): + return b + default: + return a + } +} + +// MinMaxEqual returns true if both stat objects have the same Min and Max values +func (s *FixedLenByteArrayStatistics) MinMaxEqual(rhs *FixedLenByteArrayStatistics) bool { + return s.equal(s.min, rhs.min) && s.equal(s.max, rhs.max) +} + +// Equals returns true only if both objects are the same type, have the same min and +// max values, null count, distinct count and number of values. +func (s *FixedLenByteArrayStatistics) Equals(other TypedStatistics) bool { + if s.Type() != other.Type() { + return false + } + rhs, ok := other.(*FixedLenByteArrayStatistics) + if !ok { + return false + } + + if s.HasMinMax() != rhs.HasMinMax() { + return false + } + return (s.hasMinMax && s.MinMaxEqual(rhs)) && + s.NullCount() == rhs.NullCount() && + s.DistinctCount() == rhs.DistinctCount() && + s.NumValues() == rhs.NumValues() +} + +func (s *FixedLenByteArrayStatistics) getMinMax(values []parquet.FixedLenByteArray) (min, max parquet.FixedLenByteArray) { + defMin := s.defaultMin() + defMax := s.defaultMax() + + min = defMin + max = defMax + + for _, v := range values { + min = s.minval(min, v) + max = s.maxval(max, v) + } + return +} + +func (s *FixedLenByteArrayStatistics) getMinMaxSpaced(values []parquet.FixedLenByteArray, validBits []byte, validBitsOffset int64) (min, max parquet.FixedLenByteArray) { + min = s.defaultMin() + max = s.defaultMax() + + if s.bitSetReader == nil { + s.bitSetReader = utils.NewSetBitRunReader(validBits, validBitsOffset, int64(len(values))) + } else { + s.bitSetReader.Reset(validBits, validBitsOffset, int64(len(values))) + } + + for { + run := s.bitSetReader.NextRun() + if run.Length == 0 { + break + } + for _, v := range values[int(run.Pos):int(run.Pos+run.Length)] { + min = s.minval(min, v) + max = s.maxval(max, v) + } + } + return +} + +func (s *FixedLenByteArrayStatistics) Min() parquet.FixedLenByteArray { return s.min } +func (s *FixedLenByteArrayStatistics) Max() parquet.FixedLenByteArray { return s.max } + +// Merge merges the stats from other into this stat object, updating +// the null count, distinct count, number of values and the min/max if +// appropriate. +func (s *FixedLenByteArrayStatistics) Merge(other TypedStatistics) { + rhs, ok := other.(*FixedLenByteArrayStatistics) + if !ok { + panic("incompatible stat type merge") + } + + s.statistics.merge(rhs) + if rhs.HasMinMax() { + s.SetMinMax(rhs.Min(), rhs.Max()) + } +} + +// Update is used to add more values to the current stat object, finding the +// min and max values etc. +func (s *FixedLenByteArrayStatistics) Update(values []parquet.FixedLenByteArray, numNull int64) { + s.incNulls(numNull) + s.nvalues += int64(len(values)) + + if len(values) == 0 { + return + } + + s.SetMinMax(s.getMinMax(values)) +} + +// UpdateSpaced is just like Update, but for spaced values using validBits to determine +// and skip null values. +func (s *FixedLenByteArrayStatistics) UpdateSpaced(values []parquet.FixedLenByteArray, validBits []byte, validBitsOffset, numNull int64) { + s.incNulls(numNull) + notnull := int64(len(values)) - numNull + s.nvalues += notnull + + if notnull == 0 { + return + } + + s.SetMinMax(s.getMinMaxSpaced(values, validBits, validBitsOffset)) +} + +// SetMinMax updates the min and max values only if they are not currently set +// or if argMin is less than the current min / argMax is greater than the current max +func (s *FixedLenByteArrayStatistics) SetMinMax(argMin, argMax parquet.FixedLenByteArray) { + maybeMinMax := s.cleanStat([2]parquet.FixedLenByteArray{argMin, argMax}) + if maybeMinMax == nil { + return + } + + min := (*maybeMinMax)[0] + max := (*maybeMinMax)[1] + + if !s.hasMinMax { + s.hasMinMax = true + s.min = min + s.max = max + } else { + if !s.less(s.min, min) { + s.min = min + } + if s.less(s.max, max) { + s.max = max + } + } +} + +// EncodeMin returns the encoded min value with plain encoding. +// +// ByteArray stats do not include the length in the encoding. +func (s *FixedLenByteArrayStatistics) EncodeMin() []byte { + if s.HasMinMax() { + return s.plainEncode(s.min) + } + return nil +} + +// EncodeMax returns the current encoded max value with plain encoding +// +// ByteArray stats do not include the length in the encoding +func (s *FixedLenByteArrayStatistics) EncodeMax() []byte { + if s.HasMinMax() { + return s.plainEncode(s.max) + } + return nil +} + +// Encode returns a populated EncodedStatistics object +func (s *FixedLenByteArrayStatistics) Encode() (enc EncodedStatistics, err error) { + defer func() { + if r := recover(); r != nil { + switch r := r.(type) { + case error: + err = r + case string: + err = xerrors.New(r) + default: + err = xerrors.Errorf("unknown error type thrown from panic: %v", r) + } + } + }() + if s.HasMinMax() { + enc.SetMax(s.EncodeMax()) + enc.SetMin(s.EncodeMin()) + } + if s.HasNullCount() { + enc.SetNullCount(s.NullCount()) + } + if s.HasDistinctCount() { + enc.SetDistinctCount(s.DistinctCount()) + } + return +} + +// NewStatistics uses the type in the column descriptor to construct the appropriate +// typed stats object. If mem is nil, then memory.DefaultAllocator will be used. +func NewStatistics(descr *schema.Column, mem memory.Allocator) TypedStatistics { + if mem == nil { + mem = memory.DefaultAllocator + } + switch descr.PhysicalType() { + case parquet.Types.Int32: + return NewInt32Statistics(descr, mem) + case parquet.Types.Int64: + return NewInt64Statistics(descr, mem) + case parquet.Types.Int96: + return NewInt96Statistics(descr, mem) + case parquet.Types.Float: + return NewFloat32Statistics(descr, mem) + case parquet.Types.Double: + return NewFloat64Statistics(descr, mem) + case parquet.Types.Boolean: + return NewBooleanStatistics(descr, mem) + case parquet.Types.ByteArray: + return NewByteArrayStatistics(descr, mem) + case parquet.Types.FixedLenByteArray: + return NewFixedLenByteArrayStatistics(descr, mem) + default: + panic("not implemented") + } +} + +// NewStatisticsFromEncoded uses the provided information to initialize a typed stat object +// by checking the type of the provided column descriptor. +// +// If mem is nil, then memory.DefaultAllocator is used. +func NewStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) TypedStatistics { + if mem == nil { + mem = memory.DefaultAllocator + } + switch descr.PhysicalType() { + case parquet.Types.Int32: + return NewInt32StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + case parquet.Types.Int64: + return NewInt64StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + case parquet.Types.Int96: + return NewInt96StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + case parquet.Types.Float: + return NewFloat32StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + case parquet.Types.Double: + return NewFloat64StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + case parquet.Types.Boolean: + return NewBooleanStatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + case parquet.Types.ByteArray: + return NewByteArrayStatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + case parquet.Types.FixedLenByteArray: + return NewFixedLenByteArrayStatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + default: + panic("not implemented") + } +} diff --git a/go/parquet/metadata/statistics_types.gen.go.tmpl b/go/parquet/metadata/statistics_types.gen.go.tmpl new file mode 100644 index 00000000000..66a483a3c59 --- /dev/null +++ b/go/parquet/metadata/statistics_types.gen.go.tmpl @@ -0,0 +1,418 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/schema" + "github.com/apache/arrow/go/parquet/internal/utils" + "github.com/apache/arrow/go/parquet/internal/encoding" +) + +{{range .In}} +type minmaxPair{{.Name}} [2]{{.name}} + +// {{.Name}}Statistics is the typed interface for managing stats for a column +// of {{.Name}} type. +type {{.Name}}Statistics struct { + statistics + min {{.name}} + max {{.name}} + + bitSetReader utils.SetBitRunReader +} + +// New{{.Name}}Statistics constructs an appropriate stat object type using the +// given column descriptor and allocator. +// +// Panics if the physical type of descr is not parquet.Type.{{if .physical}}{{.physical}}{{else}}{{.Name}}{{end}} +func New{{.Name}}Statistics(descr *schema.Column, mem memory.Allocator) *{{.Name}}Statistics { + if descr.PhysicalType() != parquet.Types.{{if .physical}}{{.physical}}{{else}}{{.Name}}{{end}} { + panic(xerrors.Errorf("parquet: invalid type %s for constructing a {{.Name}} stat object", descr.PhysicalType())) + } + + return &{{.Name}}Statistics{ + statistics: statistics{ + descr: descr, + hasNullCount: true, + hasDistinctCount: true, + order: descr.SortOrder(), + encoder: encoding.NewEncoder(descr.PhysicalType(), parquet.Encodings.Plain, false, descr, mem), + mem: mem, + }, + {{if eq .Name "ByteArray"}} + min: make([]byte, 0), + max: make([]byte, 0), + {{end}} + } +} + +// New{{.Name}}StatisticsFromEncoded will construct a propertly typed statistics object +// initializing it with the provided information. +func New{{.Name}}StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *{{.Name}}Statistics { + ret := New{{.Name}}Statistics(descr, mem) + ret.nvalues += nvalues + if hasNulls { + ret.incNulls(nulls) + } + if hasDistinct { + ret.incDistinct(distinct) + } + + if encodedMin != nil && len(encodedMin) > 0 { + ret.min = ret.plainDecode(encodedMin) + } + if encodedMax != nil && len(encodedMax) > 0 { + ret.max = ret.plainDecode(encodedMax) + } + ret.hasMinMax = hasMinMax + return ret +} + +func (s *{{.Name}}Statistics) plainEncode(src {{.name}}) []byte { +{{- if eq .Name "ByteArray"}} + return src +{{- else}} + s.encoder.(encoding.{{.Name}}Encoder).Put([]{{.name}}{src}) + buf, err := s.encoder.FlushValues() + if err != nil { + panic(err) // recovered by Encode + } + defer buf.Release() + + out := make([]byte, buf.Len()) + copy(out, buf.Bytes()) + return out +{{- end}} +} + +func (s *{{.Name}}Statistics) plainDecode(src []byte) {{.name}} { +{{- if eq .Name "ByteArray"}} + return src +{{- else}} + var buf [1]{{.name}} + + decoder := encoding.NewDecoder(s.descr.PhysicalType(), parquet.Encodings.Plain, s.descr, s.mem) + decoder.SetData(1, src) + decoder.(encoding.{{.Name}}Decoder).Decode(buf[:]) + return buf[0] +{{- end}} +} + +{{if and (ne .Name "ByteArray") (ne .Name "FixedLenByteArray")}} +func (s *{{.Name}}Statistics) minval(a, b {{.name}}) {{.name}} { + if s.less(a, b) { + return a + } + return b +} + +func (s *{{.Name}}Statistics) maxval(a, b {{.name}}) {{.name}} { + if s.less(a, b) { + return b + } + return a +} +{{else}} +func (s *{{.Name}}Statistics) minval(a, b {{.name}}) {{.name}} { + switch { + case a == nil: + return b + case b == nil: + return a + case s.less(a, b): + return a + default: + return b + } +} + +func (s *{{.Name}}Statistics) maxval(a, b {{.name}}) {{.name}} { + switch { + case a == nil: + return b + case b == nil: + return a + case s.less(a, b): + return b + default: + return a + } +} +{{end}} + +// MinMaxEqual returns true if both stat objects have the same Min and Max values +func (s *{{.Name}}Statistics) MinMaxEqual(rhs *{{.Name}}Statistics) bool { + return s.equal(s.min, rhs.min) && s.equal(s.max, rhs.max) +} + +// Equals returns true only if both objects are the same type, have the same min and +// max values, null count, distinct count and number of values. +func (s *{{.Name}}Statistics) Equals(other TypedStatistics) bool { + if s.Type() != other.Type() { + return false + } + rhs, ok := other.(*{{.Name}}Statistics) + if !ok { + return false + } + + if s.HasMinMax() != rhs.HasMinMax() { return false } + return (s.hasMinMax && s.MinMaxEqual(rhs)) && + s.NullCount() == rhs.NullCount() && + s.DistinctCount() == rhs.DistinctCount() && + s.NumValues() == rhs.NumValues() +} + +{{if or (eq .name "float32") (eq .name "float64")}} +func (s *{{.Name}}Statistics) coalesce(val, fallback {{.name}}) {{.name}} { + if math.IsNaN(float64(val)) { + return fallback + } + return val +} +{{end}} + +func (s *{{.Name}}Statistics) getMinMax(values []{{.name}}) (min, max {{.name}}) { +{{- if or (eq .name "int32") (eq .name "int64")}} + if s.order == schema.SortSIGNED { + min, max = utils.GetMinMax{{.Name}}(values) + } else { + umin, umax := utils.GetMinMaxU{{.name}}(arrow.U{{.name}}Traits.CastFromBytes(arrow.{{.Name}}Traits.CastToBytes(values))) + min, max = {{.name}}(umin), {{.name}}(umax) + } +{{- else}} + defMin := s.defaultMin() + defMax := s.defaultMax() + + min = defMin + max = defMax + + for _, v := range values { +{{- if or (eq .name "float32") (eq .name "float64") }} + min = s.minval(min, s.coalesce(v, defMin)) + max = s.maxval(max, s.coalesce(v, defMax)) +{{- else}} + min = s.minval(min, v) + max = s.maxval(max, v) +{{- end }} + } +{{- end}} + return +} + +func (s *{{.Name}}Statistics) getMinMaxSpaced(values []{{.name}}, validBits []byte, validBitsOffset int64) (min, max {{.name}}) { + min = s.defaultMin() + max = s.defaultMax() + +{{- if or (eq .name "int32") (eq .name "int64")}} + var fn func([]{{.name}}) ({{.name}}, {{.name}}) + if s.order == schema.SortSIGNED { + fn = utils.GetMinMax{{.Name}} + } else { + fn = func(v []{{.name}}) ({{.name}}, {{.name}}) { + umin, umax := utils.GetMinMaxU{{.name}}(arrow.U{{.name}}Traits.CastFromBytes(arrow.{{.Name}}Traits.CastToBytes(values))) + return {{.name}}(umin), {{.name}}(umax) + } + } +{{- end}} + + if s.bitSetReader == nil { + s.bitSetReader = utils.NewSetBitRunReader(validBits, validBitsOffset, int64(len(values))) + } else { + s.bitSetReader.Reset(validBits, validBitsOffset, int64(len(values))) + } + + for { + run := s.bitSetReader.NextRun() + if run.Length == 0 { + break + } +{{- if or (eq .name "int32") (eq .name "int64")}} + localMin, localMax := fn(values[int(run.Pos):int(run.Pos+run.Length)]) + if min > localMin { + min = localMin + } + if max < localMax { + max = localMax + } +{{- else}} + for _, v := range values[int(run.Pos):int(run.Pos+run.Length)] { +{{- if or (eq .name "float32") (eq .name "float64") }} + min = s.minval(min, coalesce(v, s.defaultMin()).({{.name}})) + max = s.maxval(max, coalesce(v, s.defaultMax()).({{.name}})) +{{- else}} + min = s.minval(min, v) + max = s.maxval(max, v) +{{- end }} + } +{{- end}} + } + return +} + +func (s *{{.Name}}Statistics) Min() {{.name}} { return s.min } +func (s *{{.Name}}Statistics) Max() {{.name}} { return s.max } + +// Merge merges the stats from other into this stat object, updating +// the null count, distinct count, number of values and the min/max if +// appropriate. +func (s *{{.Name}}Statistics) Merge(other TypedStatistics) { + rhs, ok := other.(*{{.Name}}Statistics) + if !ok { + panic("incompatible stat type merge") + } + + s.statistics.merge(rhs) + if rhs.HasMinMax() { + s.SetMinMax(rhs.Min(), rhs.Max()) + } +} + +// Update is used to add more values to the current stat object, finding the +// min and max values etc. +func (s *{{.Name}}Statistics) Update(values []{{.name}}, numNull int64) { + s.incNulls(numNull) + s.nvalues += int64(len(values)) + + if len(values) == 0 { + return + } + + s.SetMinMax(s.getMinMax(values)) +} + +// UpdateSpaced is just like Update, but for spaced values using validBits to determine +// and skip null values. +func (s *{{.Name}}Statistics) UpdateSpaced(values []{{.name}}, validBits []byte, validBitsOffset, numNull int64) { + s.incNulls(numNull) + notnull := int64(len(values)) - numNull + s.nvalues += notnull + + if notnull == 0 { + return + } + + s.SetMinMax(s.getMinMaxSpaced(values, validBits, validBitsOffset)) +} + +// SetMinMax updates the min and max values only if they are not currently set +// or if argMin is less than the current min / argMax is greater than the current max +func (s *{{.Name}}Statistics) SetMinMax(argMin, argMax {{.name}}) { + maybeMinMax := s.cleanStat([2]{{.name}}{argMin, argMax}) + if maybeMinMax == nil { + return + } + + min := (*maybeMinMax)[0] + max := (*maybeMinMax)[1] + + if !s.hasMinMax { + s.hasMinMax = true + s.min = min + s.max = max + } else { + if !s.less(s.min, min) { + s.min = min + } + if s.less(s.max, max) { + s.max = max + } + } +} + +// EncodeMin returns the encoded min value with plain encoding. +// +// ByteArray stats do not include the length in the encoding. +func (s *{{.Name}}Statistics) EncodeMin() []byte { + if s.HasMinMax() { + return s.plainEncode(s.min) + } + return nil +} + +// EncodeMax returns the current encoded max value with plain encoding +// +// ByteArray stats do not include the length in the encoding +func (s *{{.Name}}Statistics) EncodeMax() []byte{ + if s.HasMinMax() { + return s.plainEncode(s.max) + } + return nil +} + +// Encode returns a populated EncodedStatistics object +func (s *{{.Name}}Statistics) Encode() (enc EncodedStatistics, err error) { + defer func() { + if r := recover(); r != nil { + switch r := r.(type) { + case error: + err = r + case string: + err = xerrors.New(r) + default: + err = xerrors.Errorf("unknown error type thrown from panic: %v", r) + } + } + }() + if s.HasMinMax() { + enc.SetMax(s.EncodeMax()) + enc.SetMin(s.EncodeMin()) + } + if s.HasNullCount() { + enc.SetNullCount(s.NullCount()) + } + if s.HasDistinctCount() { + enc.SetDistinctCount(s.DistinctCount()) + } + return +} +{{end}} + +// NewStatistics uses the type in the column descriptor to construct the appropriate +// typed stats object. If mem is nil, then memory.DefaultAllocator will be used. +func NewStatistics(descr *schema.Column, mem memory.Allocator) TypedStatistics { + if mem == nil { + mem = memory.DefaultAllocator + } + switch descr.PhysicalType() { +{{- range .In}} + case parquet.Types.{{if .physical}}{{.physical}}{{else}}{{.Name}}{{end}}: + return New{{.Name}}Statistics(descr, mem) +{{- end}} + default: + panic("not implemented") + } +} + +// NewStatisticsFromEncoded uses the provided information to initialize a typed stat object +// by checking the type of the provided column descriptor. +// +// If mem is nil, then memory.DefaultAllocator is used. +func NewStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) TypedStatistics { + if mem == nil { + mem = memory.DefaultAllocator + } + switch descr.PhysicalType() { +{{- range .In}} + case parquet.Types.{{if .physical}}{{.physical}}{{else}}{{.Name}}{{end}}: + return New{{.Name}}StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) +{{- end}} + default: + panic("not implemented") + } +} From bb86c1fd0aed53586bcd9c6de0ae675d4b1dba79 Mon Sep 17 00:00:00 2001 From: Matthew Topol Date: Fri, 20 Aug 2021 12:24:34 -0400 Subject: [PATCH 2/7] small updates from PR feedback --- go/parquet/metadata/column_chunk.go | 2 +- go/parquet/metadata/file.go | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/go/parquet/metadata/column_chunk.go b/go/parquet/metadata/column_chunk.go index 4f55355baca..29316a9c2c3 100644 --- a/go/parquet/metadata/column_chunk.go +++ b/go/parquet/metadata/column_chunk.go @@ -299,7 +299,7 @@ func (c *ColumnChunkMetaDataBuilder) Finish(nvalues, dictPageOffset, indexPageOf c.chunk.MetaData.TotalUncompressedSize = uncompressed c.chunk.MetaData.TotalCompressedSize = compressed - thriftEncodings := make([]format.Encoding, 0) + thriftEncodings := make([]format.Encoding, 0, 3) if hasDict { thriftEncodings = append(thriftEncodings, format.Encoding(c.props.DictionaryIndexEncoding())) if c.props.Version() == parquet.V1 { diff --git a/go/parquet/metadata/file.go b/go/parquet/metadata/file.go index 8c57ed5f09a..12feea9063a 100644 --- a/go/parquet/metadata/file.go +++ b/go/parquet/metadata/file.go @@ -238,8 +238,6 @@ func NewFileMetaData(data []byte, fileDecryptor encryption.FileDecryptor) (*File f.initSchema() f.initColumnOrders() - // init keyvalue metadata - return f, nil } From 3faf6cbd35cabfe5573ac0fa5ed83ee93d1d9f34 Mon Sep 17 00:00:00 2001 From: Matthew Topol Date: Wed, 1 Sep 2021 13:24:12 -0400 Subject: [PATCH 3/7] updates from feedback and fixes for stats --- go/parquet/metadata/app_version.go | 16 ++- go/parquet/metadata/column_chunk.go | 67 +++++++--- go/parquet/metadata/file.go | 26 +++- go/parquet/metadata/metadata_test.go | 39 +++--- go/parquet/metadata/row_group.go | 7 +- go/parquet/metadata/stat_compare_test.go | 73 ++++++++--- go/parquet/metadata/statistics.go | 149 +++++++++++++---------- 7 files changed, 257 insertions(+), 120 deletions(-) diff --git a/go/parquet/metadata/app_version.go b/go/parquet/metadata/app_version.go index 2a2d076d67a..02d9a4aa87c 100644 --- a/go/parquet/metadata/app_version.go +++ b/go/parquet/metadata/app_version.go @@ -41,6 +41,9 @@ var ( parquet251FixedVersion = NewAppVersionExplicit("parquet-mr", 1, 8, 0) parquetCPPFixedStatsVersion = NewAppVersionExplicit("parquet-cpp", 1, 3, 0) parquetMRFixedStatsVersion = NewAppVersionExplicit("parquet-mr", 1, 10, 0) + // parquet1655FixedVersion is the version used for fixing PARQUET-1655 + // which fixed min/max stats comparisons for Decimal types + parquet1655FixedVersion = NewAppVersionExplicit("parquet-cpp-arrow", 4, 0, 0) ) // AppVersion represents a specific application version either read from @@ -139,8 +142,8 @@ func (v AppVersion) Equal(other *AppVersion) bool { // Reference: parquet-cpp/src/parquet/metadata.cc // // PARQUET-686 has more discussion on statistics -func (v AppVersion) HasCorrectStatistics(coltype parquet.Type, stats EncodedStatistics, sort schema.SortOrder) bool { - // parquet-cpp version 1.3.0 and parquet-mr 1.10.0 onwards stats are computed correctly for all types +func (v AppVersion) HasCorrectStatistics(coltype parquet.Type, logicalType schema.LogicalType, stats EncodedStatistics, sort schema.SortOrder) bool { + // parquet-cpp version 1.3.0 and parquet-mr 1.10.0 onwards stats are computed correctly for all types except decimal if (v.App == "parquet-cpp" && v.LessThan(parquetCPPFixedStatsVersion)) || (v.App == "parquet-mr" && v.LessThan(parquetMRFixedStatsVersion)) { // only SIGNED are valid unless max and min are the same (in which case the sort order doesn't matter) @@ -156,6 +159,15 @@ func (v AppVersion) HasCorrectStatistics(coltype parquet.Type, stats EncodedStat return true } } + + // parquet-cpp-arrow version 4.0.0 fixed Decimal comparisons for creating min/max stats + // parquet-cpp also becomes parquet-cpp-arrow as of version 4.0.0 + if v.App == "parquet-cpp" || (v.App == "parquet-cpp-arrow" && v.LessThan(parquet1655FixedVersion)) { + if _, ok := logicalType.(*schema.DecimalLogicalType); ok && coltype == parquet.Types.FixedLenByteArray { + return false + } + } + // created_by is not populated, which could have been caused by // parquet-mr during the same time as PARQUET-251, see PARQUET-297 if v.App == "unknown" { diff --git a/go/parquet/metadata/column_chunk.go b/go/parquet/metadata/column_chunk.go index 29316a9c2c3..9884c2c52bf 100644 --- a/go/parquet/metadata/column_chunk.go +++ b/go/parquet/metadata/column_chunk.go @@ -122,7 +122,7 @@ func (c *ColumnChunkMetaData) FilePath() string { return c.column.GetFilePath() // Type is the physical storage type used in the parquet file for this column chunk. func (c *ColumnChunkMetaData) Type() parquet.Type { return parquet.Type(c.columnMeta.Type) } -// NumValues is the number of values stored in just this chunk +// NumValues is the number of values stored in just this chunk including nulls. func (c *ColumnChunkMetaData) NumValues() int64 { return c.columnMeta.NumValues } // PathInSchema is the full path to this column from the root of the schema including @@ -180,6 +180,12 @@ func (c *ColumnChunkMetaData) TotalUncompressedSize() int64 { return c.columnMeta.GetTotalUncompressedSize() } +// BloomFilterOffset is the byte offset from the beginning of the file to the bloom +// filter data. +func (c *ColumnChunkMetaData) BloomFilterOffset() int64 { + return c.columnMeta.GetBloomFilterOffset() +} + // StatsSet returns true only if there are statistics set in the metadata and the column // descriptor has a sort order that is not SortUnknown // @@ -198,7 +204,8 @@ func (c *ColumnChunkMetaData) StatsSet() (bool, error) { if err != nil { return false, err } - return c.writerVersion.HasCorrectStatistics(c.Type(), encoded, c.descr.SortOrder()), nil + + return c.writerVersion.HasCorrectStatistics(c.Type(), c.descr.LogicalType(), encoded, c.descr.SortOrder()), nil } func (c *ColumnChunkMetaData) Equals(other *ColumnChunkMetaData) bool { @@ -279,27 +286,53 @@ func (c *ColumnChunkMetaDataBuilder) SetStats(val EncodedStatistics) { c.chunk.MetaData.Statistics = val.ToThrift() } +// ChunkMetaInfo is a helper struct for passing the offset and size information +// for finishing the building of column chunk metadata +type ChunkMetaInfo struct { + NumValues int64 + DictPageOffset int64 + IndexPageOffset int64 + DataPageOffset int64 + CompressedSize int64 + UncompressedSize int64 +} + +// EncodingStats is a helper struct for passing the encoding stat information +// for finishing up metadata for a column chunk. +type EncodingStats struct { + DictEncodingStats map[parquet.Encoding]int32 + DataEncodingStats map[parquet.Encoding]int32 +} + // Finish finalizes the metadata with the given offsets, // flushes any compression that needs to be done, and performs // any encryption if an encryptor is provided. -func (c *ColumnChunkMetaDataBuilder) Finish(nvalues, dictPageOffset, indexPageOffset, dataPageOffset, compressed, uncompressed int64, hasDict, dictFallback bool, dictEncodingStats, dataEncodingStats map[parquet.Encoding]int32, metaEncryptor encryption.Encryptor) error { - if dictPageOffset > 0 { - c.chunk.MetaData.DictionaryPageOffset = &dictPageOffset - c.chunk.FileOffset = dictPageOffset + compressed +func (c *ColumnChunkMetaDataBuilder) Finish(info ChunkMetaInfo, hasDict, dictFallback bool, encStats EncodingStats, metaEncryptor encryption.Encryptor) error { + if info.DictPageOffset > 0 { + c.chunk.MetaData.DictionaryPageOffset = &info.DictPageOffset + c.chunk.FileOffset = info.DictPageOffset + info.CompressedSize } else { - c.chunk.FileOffset = dataPageOffset + compressed + c.chunk.FileOffset = info.DataPageOffset + info.CompressedSize } - c.chunk.MetaData.NumValues = nvalues - if indexPageOffset >= 0 { - c.chunk.MetaData.IndexPageOffset = &indexPageOffset + c.chunk.MetaData.NumValues = info.NumValues + if info.IndexPageOffset >= 0 { + c.chunk.MetaData.IndexPageOffset = &info.IndexPageOffset } - c.chunk.MetaData.DataPageOffset = dataPageOffset - c.chunk.MetaData.TotalUncompressedSize = uncompressed - c.chunk.MetaData.TotalCompressedSize = compressed + c.chunk.MetaData.DataPageOffset = info.DataPageOffset + c.chunk.MetaData.TotalUncompressedSize = info.UncompressedSize + c.chunk.MetaData.TotalCompressedSize = info.CompressedSize + + // no matter the configuration, the maximum number of thrift encodings we'll + // populate is going to be 3: + // 1. potential dictionary index encoding + // 2. page encoding + // 3. RLE for repetition and definition levels + // so let's preallocate a capacity of 3 but initialize the slice at 0 len + const maxEncodings = 3 - thriftEncodings := make([]format.Encoding, 0, 3) + thriftEncodings := make([]format.Encoding, 0, maxEncodings) if hasDict { thriftEncodings = append(thriftEncodings, format.Encoding(c.props.DictionaryIndexEncoding())) if c.props.Version() == parquet.V1 { @@ -319,15 +352,15 @@ func (c *ColumnChunkMetaDataBuilder) Finish(nvalues, dictPageOffset, indexPageOf } c.chunk.MetaData.Encodings = thriftEncodings - thriftEncodingStats := make([]*format.PageEncodingStats, 0, len(dictEncodingStats)+len(dataEncodingStats)) - for k, v := range dictEncodingStats { + thriftEncodingStats := make([]*format.PageEncodingStats, 0, len(encStats.DictEncodingStats)+len(encStats.DataEncodingStats)) + for k, v := range encStats.DictEncodingStats { thriftEncodingStats = append(thriftEncodingStats, &format.PageEncodingStats{ PageType: format.PageType_DICTIONARY_PAGE, Encoding: format.Encoding(k), Count: v, }) } - for k, v := range dataEncodingStats { + for k, v := range encStats.DataEncodingStats { thriftEncodingStats = append(thriftEncodingStats, &format.PageEncodingStats{ PageType: format.PageType_DATA_PAGE, Encoding: format.Encoding(k), diff --git a/go/parquet/metadata/file.go b/go/parquet/metadata/file.go index 12feea9063a..366e7a58c3f 100644 --- a/go/parquet/metadata/file.go +++ b/go/parquet/metadata/file.go @@ -21,6 +21,7 @@ import ( "context" "io" "reflect" + "unicode/utf8" "github.com/apache/arrow/go/parquet" "github.com/apache/arrow/go/parquet/compress" @@ -114,8 +115,9 @@ func (f *FileMetaDataBuilder) Finish() (*FileMetaData, error) { createdBy := f.props.CreatedBy() f.metadata.CreatedBy = &createdBy - // Users cannot set the `ColumnOrder` since we donot not have user defined sort order + // Users cannot set the `ColumnOrder` since we do not not have user defined sort order // in the spec yet. + // // We always default to `TYPE_DEFINED_ORDER`. We can expose it in // the API once we have user defined sort orders in the Parquet format. // TypeDefinedOrder implies choose SortOrder based on ConvertedType/PhysicalType @@ -160,7 +162,9 @@ func (f *FileMetaDataBuilder) Finish() (*FileMetaData, error) { return out, nil } -// KeyValueMetadata is an alias for a slice of thrift keyvalue pairs +// KeyValueMetadata is an alias for a slice of thrift keyvalue pairs. +// +// It is presumed that the metadata should all be utf8 valid. type KeyValueMetadata []*format.KeyValue // NewKeyValueMetadata is equivalent to make(KeyValueMetadata, 0) @@ -168,8 +172,14 @@ func NewKeyValueMetadata() KeyValueMetadata { return make(KeyValueMetadata, 0) } -func (k *KeyValueMetadata) Append(key, value string) { +// Append adds the passed in key and value to the metadata, if either contains +// any invalid utf8 runes, then it is not added and an error is returned. +func (k *KeyValueMetadata) Append(key, value string) error { + if !utf8.ValidString(key) || !utf8.ValidString(value) { + return xerrors.Errorf("metadata must be valid utf8 strings, got key = '%s' and value = '%s'", key, value) + } *k = append(*k, &format.KeyValue{Key: key, Value: &value}) + return nil } func (k KeyValueMetadata) Len() int { return len(k) } @@ -208,10 +218,14 @@ func (k KeyValueMetadata) FindValue(key string) *string { // to make it easier to use and interact with. type FileMetaData struct { *format.FileMetaData - version *AppVersion Schema *schema.Schema FileDecryptor encryption.FileDecryptor - metadataLen int + + // app version of the writer for this file + version *AppVersion + // size of the raw bytes of the metadata in the file which were + // decoded by thrift, Size() getter returns the value. + metadataLen int } // NewFileMetaData takes in the raw bytes of the serialized metadata to deserialize @@ -354,9 +368,9 @@ func (f *FileMetaData) Subset(rowGroups []int) (*FileMetaData, error) { Version: f.Version, KeyValueMetadata: f.KeyValueMetadata(), }, - f.version, f.Schema, f.FileDecryptor, + f.version, 0, } diff --git a/go/parquet/metadata/metadata_test.go b/go/parquet/metadata/metadata_test.go index 8e1625b590a..629625df5cf 100644 --- a/go/parquet/metadata/metadata_test.go +++ b/go/parquet/metadata/metadata_test.go @@ -43,8 +43,8 @@ func generateTableMetaData(schema *schema.Schema, props *parquet.WriterPropertie statsFloat.Signed = true col2Builder.SetStats(statsFloat) - col1Builder.Finish(nrows/2, 4, 0, 10, 512, 600, true, false, dictEncodingStats, dataEncodingStats, nil) - col2Builder.Finish(nrows/2, 24, 0, 30, 512, 600, true, false, dictEncodingStats, dataEncodingStats, nil) + col1Builder.Finish(metadata.ChunkMetaInfo{nrows / 2, 4, 0, 10, 512, 600}, true, false, metadata.EncodingStats{dictEncodingStats, dataEncodingStats}, nil) + col2Builder.Finish(metadata.ChunkMetaInfo{nrows / 2, 24, 0, 30, 512, 600}, true, false, metadata.EncodingStats{dictEncodingStats, dataEncodingStats}, nil) rg1Builder.SetNumRows(nrows / 2) rg1Builder.Finish(1024, -1) @@ -56,8 +56,8 @@ func generateTableMetaData(schema *schema.Schema, props *parquet.WriterPropertie // column metadata col1Builder.SetStats(statsInt) col2Builder.SetStats(statsFloat) - col1Builder.Finish(nrows/2, 6, 0, 10, 512, 600, true, false, dictEncodingStats, dataEncodingStats, nil) - col2Builder.Finish(nrows/2, 16, 0, 26, 512, 600, true, false, dictEncodingStats, dataEncodingStats, nil) + col1Builder.Finish(metadata.ChunkMetaInfo{nrows / 2, 6, 0, 10, 512, 600}, true, false, metadata.EncodingStats{dictEncodingStats, dataEncodingStats}, nil) + col2Builder.Finish(metadata.ChunkMetaInfo{nrows / 2, 16, 0, 26, 512, 600}, true, false, metadata.EncodingStats{dictEncodingStats, dataEncodingStats}, nil) rg2Builder.SetNumRows(nrows / 2) rg2Builder.Finish(1024, -1) @@ -297,19 +297,19 @@ func TestApplicationVersion(t *testing.T) { assert.True(t, version.LessThan(version1)) var stats metadata.EncodedStatistics - assert.False(t, version1.HasCorrectStatistics(parquet.Types.Int96, stats, schema.SortUNKNOWN)) - assert.True(t, version.HasCorrectStatistics(parquet.Types.Int32, stats, schema.SortSIGNED)) - assert.False(t, version.HasCorrectStatistics(parquet.Types.ByteArray, stats, schema.SortSIGNED)) - assert.True(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, stats, schema.SortSIGNED)) - assert.False(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, stats, schema.SortUNSIGNED)) - assert.True(t, version3.HasCorrectStatistics(parquet.Types.FixedLenByteArray, stats, schema.SortSIGNED)) + assert.False(t, version1.HasCorrectStatistics(parquet.Types.Int96, schema.NoLogicalType{}, stats, schema.SortUNKNOWN)) + assert.True(t, version.HasCorrectStatistics(parquet.Types.Int32, schema.NoLogicalType{}, stats, schema.SortSIGNED)) + assert.False(t, version.HasCorrectStatistics(parquet.Types.ByteArray, schema.NoLogicalType{}, stats, schema.SortSIGNED)) + assert.True(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, schema.NoLogicalType{}, stats, schema.SortSIGNED)) + assert.False(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, schema.NoLogicalType{}, stats, schema.SortUNSIGNED)) + assert.True(t, version3.HasCorrectStatistics(parquet.Types.FixedLenByteArray, schema.NoLogicalType{}, stats, schema.SortSIGNED)) // check that the old stats are correct if min and max are the same regardless of sort order var statsStr metadata.EncodedStatistics statsStr.SetMin([]byte("a")).SetMax([]byte("b")) - assert.False(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, statsStr, schema.SortUNSIGNED)) + assert.False(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, schema.NoLogicalType{}, statsStr, schema.SortUNSIGNED)) statsStr.SetMax([]byte("a")) - assert.True(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, statsStr, schema.SortUNSIGNED)) + assert.True(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, schema.NoLogicalType{}, statsStr, schema.SortUNSIGNED)) // check that the same holds true for ints var ( @@ -319,7 +319,18 @@ func TestApplicationVersion(t *testing.T) { var statsInt metadata.EncodedStatistics statsInt.SetMin((*(*[4]byte)(unsafe.Pointer(&intMin)))[:]) statsInt.SetMax((*(*[4]byte)(unsafe.Pointer(&intMax)))[:]) - assert.False(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, statsInt, schema.SortUNSIGNED)) + assert.False(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, schema.NoLogicalType{}, statsInt, schema.SortUNSIGNED)) statsInt.SetMax((*(*[4]byte)(unsafe.Pointer(&intMin)))[:]) - assert.True(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, statsInt, schema.SortUNSIGNED)) + assert.True(t, version1.HasCorrectStatistics(parquet.Types.ByteArray, schema.NoLogicalType{}, statsInt, schema.SortUNSIGNED)) +} + +func TestCheckBadDecimalStats(t *testing.T) { + version1 := metadata.NewAppVersion("parquet-cpp version 3.0.0") + version2 := metadata.NewAppVersion("parquet-cpp-arrow version 3.0.0") + version3 := metadata.NewAppVersion("parquet-cpp-arrow version 4.0.0") + + var stats metadata.EncodedStatistics + assert.False(t, version1.HasCorrectStatistics(parquet.Types.FixedLenByteArray, schema.NewDecimalLogicalType(5, 0), stats, schema.SortSIGNED)) + assert.False(t, version2.HasCorrectStatistics(parquet.Types.FixedLenByteArray, schema.NewDecimalLogicalType(5, 0), stats, schema.SortSIGNED)) + assert.True(t, version3.HasCorrectStatistics(parquet.Types.FixedLenByteArray, schema.NewDecimalLogicalType(5, 0), stats, schema.SortSIGNED)) } diff --git a/go/parquet/metadata/row_group.go b/go/parquet/metadata/row_group.go index a252143ba22..12863138cd6 100644 --- a/go/parquet/metadata/row_group.go +++ b/go/parquet/metadata/row_group.go @@ -55,7 +55,8 @@ func (r *RowGroupMetaData) Equals(other *RowGroupMetaData) bool { return reflect.DeepEqual(r.rowGroup, other.rowGroup) } -// NumRows is just the number of rows in this row group +// NumRows is just the number of rows in this row group. All columns have the same +// number of rows for a row group regardless of repetition and definition levels. func (r *RowGroupMetaData) NumRows() int64 { return r.rowGroup.NumRows } // TotalByteSize is the total size of this rowgroup on disk @@ -138,6 +139,10 @@ func (r *RowGroupMetaDataBuilder) NextColumnChunk() *ColumnChunkMetaDataBuilder return colBldr } +// Finish should be called when complete and updates the metadata with the final +// file offset, and total compressed sizes. totalBytesWritten gets written as the +// TotalByteSize for the row group and Ordinal should be the index of the row group +// being written. e.g. first row group should be 0, second is 1, and so on... func (r *RowGroupMetaDataBuilder) Finish(totalBytesWritten int64, ordinal int16) error { if r.nextCol != r.NumColumns() { return xerrors.Errorf("parquet: only %d out of %d columns are initialized", r.nextCol-1, r.schema.NumColumns()) diff --git a/go/parquet/metadata/stat_compare_test.go b/go/parquet/metadata/stat_compare_test.go index 373657de867..c1fd11a8f69 100644 --- a/go/parquet/metadata/stat_compare_test.go +++ b/go/parquet/metadata/stat_compare_test.go @@ -33,15 +33,49 @@ func TestSignedByteArrayCompare(t *testing.T) { }, } - s1ba := parquet.ByteArray("12345") - s2ba := parquet.ByteArray("12345678") - assert.True(t, s.less(s1ba, s2ba)) + // signed byte array comparison is only used for Decimal comparison. + // when decimals are encoded as byte arrays they use twos compliment + // big-endian encoded values. Comparisons of byte arrays of unequal + // types need to handle sign extension. + + tests := []struct { + b []byte + order int + }{ + {[]byte{0x80, 0x80, 0, 0}, 0}, + {[]byte{ /*0xFF,*/ 0x80, 0, 0}, 1}, + {[]byte{0xFF, 0x80, 0, 0}, 1}, + {[]byte{ /*0xFF,*/ 0xFF, 0x01, 0}, 2}, + {[]byte{ /*0xFF, 0xFF,*/ 0x80, 0}, 3}, + {[]byte{ /*0xFF,*/ 0xFF, 0x80, 0}, 3}, + {[]byte{0xFF, 0xFF, 0x80, 0}, 3}, + {[]byte{ /*0xFF,0xFF,0xFF,*/ 0x80}, 4}, + {[]byte{ /*0xFF,0xFF,0xFF*/ 0xFF}, 5}, + {[]byte{ /*0, 0,*/ 0x01, 0x01}, 6}, + {[]byte{ /*0,*/ 0, 0x01, 0x01}, 6}, + {[]byte{0, 0, 0x01, 0x01}, 6}, + {[]byte{ /*0,*/ 0x01, 0x01, 0}, 7}, + {[]byte{0x01, 0x01, 0, 0}, 8}, + } - // This is case where signed comparison UTF-8 (PARQUET-686) is incorrect - // This example is to only check signed comparison and not UTF-8. - s1ba = parquet.ByteArray("bügeln") - s2ba = parquet.ByteArray("braten") - assert.True(t, s.less(s1ba, s2ba)) + for i, tt := range tests { + // empty array is always the smallest + assert.Truef(t, s.less(parquet.ByteArray{}, parquet.ByteArray(tt.b)), "case: %d", i) + assert.Falsef(t, s.less(parquet.ByteArray(tt.b), parquet.ByteArray{}), "case: %d", i) + // equals is always false + assert.Falsef(t, s.less(parquet.ByteArray(tt.b), parquet.ByteArray(tt.b)), "case: %d", i) + + for j, case2 := range tests { + var fn func(assert.TestingT, bool, string, ...interface{}) bool + if tt.order < case2.order { + fn = assert.Truef + } else { + fn = assert.Falsef + } + fn(t, s.less(parquet.ByteArray(tt.b), parquet.ByteArray(case2.b)), + "%d (order: %d) %d (order: %d)", i, tt.order, j, case2.order) + } + } } func TestUnsignedByteArrayCompare(t *testing.T) { @@ -70,13 +104,24 @@ func TestSignedCompareFLBA(t *testing.T) { statistics: statistics{order: schema.SortSIGNED}, } - s1flba := parquet.FixedLenByteArray("Anti123456") - s2flba := parquet.FixedLenByteArray("Bunkd123456") - assert.True(t, s.less(s1flba, s2flba)) + values := []parquet.FixedLenByteArray{ + []byte{0x80, 0, 0, 0}, + []byte{0xFF, 0xFF, 0x01, 0}, + []byte{0xFF, 0xFF, 0x80, 0}, + []byte{0xFF, 0xFF, 0xFF, 0x80}, + []byte{0xFF, 0xFF, 0xFF, 0xFF}, + []byte{0, 0, 0x01, 0x01}, + []byte{0, 0x01, 0x01, 0}, + []byte{0x01, 0x01, 0, 0}, + } - s1flba = parquet.FixedLenByteArray("Bünk123456") - s2flba = parquet.FixedLenByteArray("Bunk123456") - assert.True(t, s.less(s1flba, s2flba)) + for i, v := range values { + assert.Falsef(t, s.less(v, v), "%d", i) + for j, v2 := range values[i+1:] { + assert.Truef(t, s.less(v, v2), "%d %d", i, j) + assert.Falsef(t, s.less(v2, v), "%d %d", j, i) + } + } } func TestUnsignedCompareFLBA(t *testing.T) { diff --git a/go/parquet/metadata/statistics.go b/go/parquet/metadata/statistics.go index dd982e04f74..a22dba2dacb 100644 --- a/go/parquet/metadata/statistics.go +++ b/go/parquet/metadata/statistics.go @@ -23,12 +23,11 @@ import ( "unsafe" "github.com/apache/arrow/go/arrow" - "github.com/apache/arrow/go/arrow/array" "github.com/apache/arrow/go/arrow/memory" "github.com/apache/arrow/go/parquet" + "github.com/apache/arrow/go/parquet/internal/debug" "github.com/apache/arrow/go/parquet/internal/encoding" format "github.com/apache/arrow/go/parquet/internal/gen-go/parquet" - "github.com/apache/arrow/go/parquet/internal/utils" "github.com/apache/arrow/go/parquet/schema" ) @@ -200,6 +199,8 @@ func (s *statistics) Reset() { s.hasNullCount = false } +// base merge function for base non-typed stat object so we don't have to +// duplicate this in each of the typed implementations func (s *statistics) merge(other TypedStatistics) { s.nvalues += other.NumValues() if other.HasNullCount() { @@ -224,6 +225,84 @@ func coalesce(val, fallback interface{}) interface{} { return val } +func signedByteLess(a, b []byte) bool { + // signed comparison is used for integers encoded as big-endian twos complement + // integers (e.g. decimals) + + // if at least one of the lengths is zero, we can short circuit + if len(a) == 0 || len(b) == 0 { + return len(a) == 0 && len(b) > 0 + } + + sa := *(*[]int8)(unsafe.Pointer(&a)) + sb := *(*[]int8)(unsafe.Pointer(&b)) + + // we can short circuit for different signd numbers or for equal length byte + // arrays that have different first bytes. The equality requirement is necessary + // for sign extension cases. 0xFF10 should be equal to 0x10 (due to big endian sign extension) + if int8(0x80&uint8(sa[0])) != int8(0x80&uint8(sb[0])) || (len(sa) == len(sb) && sa[0] != sb[0]) { + return sa[0] < sb[0] + } + + // when the lengths are unequal and the numbers are of the same sign, we need + // to do comparison by sign extending the shorter value first, and once we get + // to equal sized arrays, lexicographical unsigned comparison of everything but + // the first byte is sufficient. + + if len(a) != len(b) { + var lead []byte + if len(a) > len(b) { + leadLen := len(a) - len(b) + lead = a[:leadLen] + a = a[leadLen:] + } else { + debug.Assert(len(a) < len(b), "something weird in byte slice signed comparison") + leadLen := len(b) - len(a) + lead = b[:leadLen] + b = b[leadLen:] + } + + // compare extra bytes to the sign extension of the first byte of the other number + var extension byte + if sa[0] < 0 { + extension = 0xFF + } + + notequal := false + for _, c := range lead { + if c != extension { + notequal = true + break + } + } + + if notequal { + // since sign extension are extrema values for unsigned bytes: + // + // Four cases exist: + // negative values: + // b is the longer value + // b must be the lesser value: return false + // else: + // a must be the lesser value: return true + // + // positive values: + // b is the longer value + // values in b must be greater than a: return true + // else: + // values in a must be greater than b: return false + neg := sa[0] < 0 + blonger := len(sa) < len(sb) + return neg != blonger + } + } else { + a = a[1:] + b = b[1:] + } + + return bytes.Compare(a, b) == -1 +} + func (BooleanStatistics) defaultMin() bool { return true } func (BooleanStatistics) defaultMax() bool { return false } func (s *Int32Statistics) defaultMin() int32 { @@ -354,18 +433,7 @@ func (s *ByteArrayStatistics) less(a, b parquet.ByteArray) bool { return bytes.Compare(a, b) == -1 } - sa := *(*[]int8)(unsafe.Pointer(&a)) - sb := *(*[]int8)(unsafe.Pointer(&b)) - i := 0 - for ; i < len(sa) && i < len(sb); i++ { - if sa[i] < sb[i] { - return true - } - if sb[i] < sa[i] { - return false - } - } - return i == len(sa) && i != len(sb) + return signedByteLess([]byte(a), []byte(b)) } func (s *FixedLenByteArrayStatistics) less(a, b parquet.FixedLenByteArray) bool { @@ -373,18 +441,7 @@ func (s *FixedLenByteArrayStatistics) less(a, b parquet.FixedLenByteArray) bool return bytes.Compare(a, b) == -1 } - sa := *(*[]int8)(unsafe.Pointer(&a)) - sb := *(*[]int8)(unsafe.Pointer(&b)) - i := 0 - for ; i < len(sa) && i < len(sb); i++ { - if sa[i] < sb[i] { - return true - } - if sb[i] < sa[i] { - return false - } - } - return i == len(sa) && i != len(sb) + return signedByteLess([]byte(a), []byte(b)) } func (BooleanStatistics) cleanStat(minMax minmaxPairBoolean) *minmaxPairBoolean { return &minMax } @@ -475,43 +532,3 @@ func GetStatValue(typ parquet.Type, val []byte) interface{} { } return nil } - -func (s *ByteArrayStatistics) UpdateWithArrow(values array.Interface) { - if _, ok := values.DataType().(arrow.BinaryDataType); !ok { - panic("can't update bytearray stats with non binary arrow data") - } - - s.incNulls(int64(values.NullN())) - s.nvalues += int64(values.Len()) - int64(values.NullN()) - if values.NullN() == values.Len() { - return - } - - var ( - min = s.defaultMin() - max = s.defaultMax() - - offsets []int32 - valueBuf []byte - ) - - switch arr := values.(type) { - case *array.Binary: - offsets = arr.ValueOffsets() - case *array.String: - offsets = arrow.Int32Traits.CastFromBytes(arr.Data().Buffers()[1].Bytes())[arr.Offset() : arr.Offset()+arr.Len()+1] - } - valueBuf = values.Data().Buffers()[2].Bytes() - - utils.VisitBitBlocks(values.NullBitmapBytes(), int64(values.Data().Offset()), int64(values.Len()), - func(_ int64) { - val := valueBuf[offsets[0]:offsets[1]] - min = s.minval(min, val) - max = s.maxval(max, val) - offsets = offsets[1:] - }, func() { - offsets = offsets[1:] - }) - - s.SetMinMax(min, max) -} From b2f1ffe5fc2ccde799c712e65ae27a50c18724c5 Mon Sep 17 00:00:00 2001 From: Matthew Topol Date: Wed, 1 Sep 2021 13:54:06 -0400 Subject: [PATCH 4/7] version stuff weirdness --- go/parquet/metadata/column_chunk.go | 2 +- go/parquet/metadata/file.go | 26 ++++++++++++---- go/parquet/metadata/metadata_test.go | 12 ++++---- go/parquet/reader_writer_properties_test.go | 2 +- go/parquet/types.go | 33 +++++++++++++++++++-- go/parquet/writer_properties.go | 6 ++-- 6 files changed, 62 insertions(+), 19 deletions(-) diff --git a/go/parquet/metadata/column_chunk.go b/go/parquet/metadata/column_chunk.go index 9884c2c52bf..06c04f44099 100644 --- a/go/parquet/metadata/column_chunk.go +++ b/go/parquet/metadata/column_chunk.go @@ -335,7 +335,7 @@ func (c *ColumnChunkMetaDataBuilder) Finish(info ChunkMetaInfo, hasDict, dictFal thriftEncodings := make([]format.Encoding, 0, maxEncodings) if hasDict { thriftEncodings = append(thriftEncodings, format.Encoding(c.props.DictionaryIndexEncoding())) - if c.props.Version() == parquet.V1 { + if c.props.Version() == parquet.V1_0 { thriftEncodings = append(thriftEncodings, format.Encoding_PLAIN) } else { thriftEncodings = append(thriftEncodings, format.Encoding(c.props.DictionaryPageEncoding())) diff --git a/go/parquet/metadata/file.go b/go/parquet/metadata/file.go index 366e7a58c3f..b97c4a4e5dc 100644 --- a/go/parquet/metadata/file.go +++ b/go/parquet/metadata/file.go @@ -105,12 +105,10 @@ func (f *FileMetaDataBuilder) Finish() (*FileMetaData, error) { f.metadata.NumRows = totalRows f.metadata.RowGroups = f.rowGroups switch f.props.Version() { - case parquet.V1: + case parquet.V1_0: f.metadata.Version = 1 - case parquet.V2: - f.metadata.Version = 2 default: - f.metadata.Version = 0 + f.metadata.Version = 2 } createdBy := f.props.CreatedBy() f.metadata.CreatedBy = &createdBy @@ -365,7 +363,7 @@ func (f *FileMetaData) Subset(rowGroups []int) (*FileMetaData, error) { ColumnOrders: f.GetColumnOrders(), EncryptionAlgorithm: f.FileMetaData.EncryptionAlgorithm, FooterSigningKeyMetadata: f.FooterSigningKeyMetadata, - Version: f.Version, + Version: f.FileMetaData.Version, KeyValueMetadata: f.KeyValueMetadata(), }, f.Schema, @@ -455,6 +453,24 @@ func (f *FileMetaData) WriteTo(w io.Writer, encryptor encryption.Encryptor) (int return int64(n), err } +// Version returns the "version" of the file +// +// WARNING: The value returned by this method is unreliable as 1) the +// parquet file metadata stores the version as a single integer and +// 2) some producers are known to always write a hardcoded value. Therefore +// you cannot use this value to know which features are used in the file. +func (f *FileMetaData) Version() parquet.Version { + switch f.FileMetaData.Version { + case 1: + return parquet.V1_0 + case 2: + return parquet.V2_LATEST + default: + // imporperly set version, assume parquet 1.0 + return parquet.V1_0 + } +} + // FileCryptoMetadata is a proxy for the thrift fileCryptoMetadata object type FileCryptoMetadata struct { metadata *format.FileCryptoMetaData diff --git a/go/parquet/metadata/metadata_test.go b/go/parquet/metadata/metadata_test.go index 629625df5cf..8e77d73ca2b 100644 --- a/go/parquet/metadata/metadata_test.go +++ b/go/parquet/metadata/metadata_test.go @@ -79,7 +79,7 @@ func assertStats(t *testing.T, m *metadata.ColumnChunkMetaData) metadata.TypedSt } func TestBuildAccess(t *testing.T) { - props := parquet.NewWriterProperties(parquet.WithVersion(parquet.V2)) + props := parquet.NewWriterProperties(parquet.WithVersion(parquet.V2_LATEST)) fields := schema.FieldList{ schema.NewInt32Node("int_col", parquet.Repetitions.Required, -1), @@ -119,7 +119,7 @@ func TestBuildAccess(t *testing.T) { for _, accessor := range []*metadata.FileMetaData{faccessor, faccessorCopy} { assert.Equal(t, nrows, accessor.NumRows) assert.Len(t, accessor.RowGroups, 2) - assert.EqualValues(t, parquet.V2, accessor.Version) + assert.EqualValues(t, parquet.V2_LATEST, accessor.Version()) assert.Equal(t, parquet.DefaultCreatedBy, accessor.GetCreatedBy()) assert.Equal(t, 3, accessor.NumSchemaElements()) @@ -205,7 +205,7 @@ func TestBuildAccess(t *testing.T) { faccessor.AppendRowGroups(faccessor2) assert.Len(t, faccessor.RowGroups, 4) assert.Equal(t, nrows*2, faccessor.NumRows) - assert.EqualValues(t, parquet.V2, faccessor.Version) + assert.EqualValues(t, parquet.V2_LATEST, faccessor.Version()) assert.Equal(t, parquet.DefaultCreatedBy, faccessor.GetCreatedBy()) assert.Equal(t, 3, faccessor.NumSchemaElements()) @@ -226,7 +226,7 @@ func TestBuildAccess(t *testing.T) { } func TestV1VersionMetadata(t *testing.T) { - props := parquet.NewWriterProperties(parquet.WithVersion(parquet.V1)) + props := parquet.NewWriterProperties(parquet.WithVersion(parquet.V1_0)) fields := schema.FieldList{ schema.NewInt32Node("int_col", parquet.Repetitions.Required, -1), @@ -239,11 +239,11 @@ func TestV1VersionMetadata(t *testing.T) { fbuilder := metadata.NewFileMetadataBuilder(schema, props, nil) faccessor, err := fbuilder.Finish() require.NoError(t, err) - assert.EqualValues(t, parquet.V1, faccessor.Version) + assert.EqualValues(t, parquet.V1_0, faccessor.Version()) } func TestKeyValueMetadata(t *testing.T) { - props := parquet.NewWriterProperties(parquet.WithVersion(parquet.V1)) + props := parquet.NewWriterProperties(parquet.WithVersion(parquet.V1_0)) fields := schema.FieldList{ schema.NewInt32Node("int_col", parquet.Repetitions.Required, -1), diff --git a/go/parquet/reader_writer_properties_test.go b/go/parquet/reader_writer_properties_test.go index ed10a76bf05..a8bffef3fdf 100644 --- a/go/parquet/reader_writer_properties_test.go +++ b/go/parquet/reader_writer_properties_test.go @@ -37,7 +37,7 @@ func TestWriterPropBasics(t *testing.T) { assert.Equal(t, parquet.DefaultDataPageSize, props.DataPageSize()) assert.Equal(t, parquet.DefaultDictionaryPageSizeLimit, props.DictionaryPageSizeLimit()) - assert.Equal(t, parquet.V1, props.Version()) + assert.Equal(t, parquet.V2_LATEST, props.Version()) assert.Equal(t, parquet.DataPageV1, props.DataPageVersion()) } diff --git a/go/parquet/types.go b/go/parquet/types.go index b55d664c91e..e568984ebe3 100644 --- a/go/parquet/types.go +++ b/go/parquet/types.go @@ -220,10 +220,37 @@ const ( AesCtr ) -// Constants for the parquet Version +// Constants for the parquet Version which governs which data types are allowed +// and how they are represented. For example, uint32 data will be written differently +// depending on this value (as INT64 for V1_0, as UINT32 for other versions). +// +// However, some features - such as compression algorithms, encryption, +// or the improved v2 data page format must be enabled separately in writer +// properties. const ( - V1 Version = 1 - V2 Version = 2 + // Enable only pre-2.2 parquet format features when writing. + // + // This is useful for maximum compatibility with legacy readers. + // Note that logical types may still be emitted, as long as they have + // a corresponding converted type. + V1_0 Version = iota + // Enable parquet format 2.4 and earlier features when writing. + // + // This enables uint32 as well as logical types which don't have a + // corresponding converted type. + // + // Note: Parquet format 2.4.0 was released in October 2017 + V2_4 + // Enable Parquet format 2.6 and earlier features when writing. + // + // This enables the nanos time unit in addition to the V2_4 features. + // + // Note: Parquet format 2.6.0 was released in September 2018 + V2_6 + // Enable the latest parquet format 2.x features. + // + // This is equal to the greatest 2.x version supported by this library. + V2_LATEST = V2_6 ) // constants for the parquet DataPage Version to use diff --git a/go/parquet/writer_properties.go b/go/parquet/writer_properties.go index ef11454a863..97bf3343f25 100644 --- a/go/parquet/writer_properties.go +++ b/go/parquet/writer_properties.go @@ -297,7 +297,7 @@ func defaultWriterProperties() *WriterProperties { batchSize: DefaultWriteBatchSize, maxRowGroupLen: DefaultMaxRowGroupLen, pageSize: DefaultDataPageSize, - parquetVersion: V1, + parquetVersion: V2_LATEST, dataPageVersion: DataPageV1, createdBy: DefaultCreatedBy, defColumnProps: DefaultColumnProperties(), @@ -434,7 +434,7 @@ func (w *WriterProperties) EncodingPath(path ColumnPath) Encoding { // DictionaryIndexEncoding returns which encoding will be used for the Dictionary Index values based on the // parquet version. V1 uses PlainDict and V2 uses RLEDict func (w *WriterProperties) DictionaryIndexEncoding() Encoding { - if w.parquetVersion == V1 { + if w.parquetVersion == V1_0 { return Encodings.PlainDict } return Encodings.RLEDict @@ -443,7 +443,7 @@ func (w *WriterProperties) DictionaryIndexEncoding() Encoding { // DictionaryPageEncoding returns the encoding that will be utilized for the DictionaryPage itself based on the parquet // version. V1 uses PlainDict, v2 uses Plain func (w *WriterProperties) DictionaryPageEncoding() Encoding { - if w.parquetVersion == V1 { + if w.parquetVersion == V1_0 { return Encodings.PlainDict } return Encodings.Plain From c9597c037307550bcc0141f3bbbba5824d3db67d Mon Sep 17 00:00:00 2001 From: Matthew Topol Date: Wed, 1 Sep 2021 14:20:04 -0400 Subject: [PATCH 5/7] ensure no misalignment of stat params --- go/parquet/metadata/column_chunk.go | 21 +-- go/parquet/metadata/statistics.go | 11 ++ go/parquet/metadata/statistics_types.gen.go | 130 ++++++++++-------- .../metadata/statistics_types.gen.go.tmpl | 18 +-- 4 files changed, 107 insertions(+), 73 deletions(-) diff --git a/go/parquet/metadata/column_chunk.go b/go/parquet/metadata/column_chunk.go index 06c04f44099..e3a7eae7d81 100644 --- a/go/parquet/metadata/column_chunk.go +++ b/go/parquet/metadata/column_chunk.go @@ -39,19 +39,24 @@ type PageEncodingStats struct { PageType format.PageType } +type statvalues struct { + *format.Statistics +} + +func (s *statvalues) GetMin() []byte { return s.GetMinValue() } +func (s *statvalues) GetMax() []byte { return s.GetMaxValue() } +func (s *statvalues) IsSetMin() bool { return s.IsSetMinValue() } +func (s *statvalues) IsSetMax() bool { return s.IsSetMaxValue() } + func makeColumnStats(metadata *format.ColumnMetaData, descr *schema.Column, mem memory.Allocator) TypedStatistics { if descr.ColumnOrder() == parquet.ColumnOrders.TypeDefinedOrder { - return NewStatisticsFromEncoded(descr, mem, metadata.Statistics.MinValue, metadata.Statistics.MaxValue, + return NewStatisticsFromEncoded(descr, mem, metadata.NumValues-metadata.Statistics.GetNullCount(), - metadata.Statistics.GetNullCount(), metadata.Statistics.GetDistinctCount(), - metadata.Statistics.IsSetMaxValue() || metadata.Statistics.IsSetMinValue(), - metadata.Statistics.IsSetNullCount(), metadata.Statistics.IsSetDistinctCount()) + &statvalues{metadata.Statistics}) } - return NewStatisticsFromEncoded(descr, mem, metadata.Statistics.Min, metadata.Statistics.Max, + return NewStatisticsFromEncoded(descr, mem, metadata.NumValues-metadata.Statistics.GetNullCount(), - metadata.Statistics.GetNullCount(), metadata.Statistics.GetDistinctCount(), - metadata.Statistics.IsSetMax() || metadata.Statistics.IsSetMin(), - metadata.Statistics.IsSetNullCount(), metadata.Statistics.IsSetDistinctCount()) + metadata.Statistics) } // ColumnChunkMetaData is a proxy around format.ColumnChunkMetaData diff --git a/go/parquet/metadata/statistics.go b/go/parquet/metadata/statistics.go index a22dba2dacb..71155b767fc 100644 --- a/go/parquet/metadata/statistics.go +++ b/go/parquet/metadata/statistics.go @@ -33,6 +33,17 @@ import ( //go:generate go run ../../arrow/_tools/tmpl/main.go -i -data=../internal/encoding/physical_types.tmpldata statistics_types.gen.go.tmpl +type StatProvider interface { + GetMin() []byte + GetMax() []byte + GetNullCount() int64 + GetDistinctCount() int64 + IsSetMax() bool + IsSetMin() bool + IsSetNullCount() bool + IsSetDistinctCount() bool +} + // EncodedStatistics are raw statistics with encoded values that will be written // to the parquet file, or was read from the parquet file. type EncodedStatistics struct { diff --git a/go/parquet/metadata/statistics_types.gen.go b/go/parquet/metadata/statistics_types.gen.go index 1625f6740e6..63971a0a714 100644 --- a/go/parquet/metadata/statistics_types.gen.go +++ b/go/parquet/metadata/statistics_types.gen.go @@ -65,23 +65,25 @@ func NewInt32Statistics(descr *schema.Column, mem memory.Allocator) *Int32Statis // NewInt32StatisticsFromEncoded will construct a propertly typed statistics object // initializing it with the provided information. -func NewInt32StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Int32Statistics { +func NewInt32StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) *Int32Statistics { ret := NewInt32Statistics(descr, mem) ret.nvalues += nvalues - if hasNulls { - ret.incNulls(nulls) + if encoded.IsSetNullCount() { + ret.incNulls(encoded.GetNullCount()) } - if hasDistinct { - ret.incDistinct(distinct) + if encoded.IsSetDistinctCount() { + ret.incDistinct(encoded.GetDistinctCount()) } + encodedMin := encoded.GetMin() if encodedMin != nil && len(encodedMin) > 0 { ret.min = ret.plainDecode(encodedMin) } + encodedMax := encoded.GetMax() if encodedMax != nil && len(encodedMax) > 0 { ret.max = ret.plainDecode(encodedMax) } - ret.hasMinMax = hasMinMax + ret.hasMinMax = encoded.IsSetMax() || encoded.IsSetMin() return ret } @@ -343,23 +345,25 @@ func NewInt64Statistics(descr *schema.Column, mem memory.Allocator) *Int64Statis // NewInt64StatisticsFromEncoded will construct a propertly typed statistics object // initializing it with the provided information. -func NewInt64StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Int64Statistics { +func NewInt64StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) *Int64Statistics { ret := NewInt64Statistics(descr, mem) ret.nvalues += nvalues - if hasNulls { - ret.incNulls(nulls) + if encoded.IsSetNullCount() { + ret.incNulls(encoded.GetNullCount()) } - if hasDistinct { - ret.incDistinct(distinct) + if encoded.IsSetDistinctCount() { + ret.incDistinct(encoded.GetDistinctCount()) } + encodedMin := encoded.GetMin() if encodedMin != nil && len(encodedMin) > 0 { ret.min = ret.plainDecode(encodedMin) } + encodedMax := encoded.GetMax() if encodedMax != nil && len(encodedMax) > 0 { ret.max = ret.plainDecode(encodedMax) } - ret.hasMinMax = hasMinMax + ret.hasMinMax = encoded.IsSetMax() || encoded.IsSetMin() return ret } @@ -621,23 +625,25 @@ func NewInt96Statistics(descr *schema.Column, mem memory.Allocator) *Int96Statis // NewInt96StatisticsFromEncoded will construct a propertly typed statistics object // initializing it with the provided information. -func NewInt96StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Int96Statistics { +func NewInt96StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) *Int96Statistics { ret := NewInt96Statistics(descr, mem) ret.nvalues += nvalues - if hasNulls { - ret.incNulls(nulls) + if encoded.IsSetNullCount() { + ret.incNulls(encoded.GetNullCount()) } - if hasDistinct { - ret.incDistinct(distinct) + if encoded.IsSetDistinctCount() { + ret.incDistinct(encoded.GetDistinctCount()) } + encodedMin := encoded.GetMin() if encodedMin != nil && len(encodedMin) > 0 { ret.min = ret.plainDecode(encodedMin) } + encodedMax := encoded.GetMax() if encodedMax != nil && len(encodedMax) > 0 { ret.max = ret.plainDecode(encodedMax) } - ret.hasMinMax = hasMinMax + ret.hasMinMax = encoded.IsSetMax() || encoded.IsSetMin() return ret } @@ -891,23 +897,25 @@ func NewFloat32Statistics(descr *schema.Column, mem memory.Allocator) *Float32St // NewFloat32StatisticsFromEncoded will construct a propertly typed statistics object // initializing it with the provided information. -func NewFloat32StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Float32Statistics { +func NewFloat32StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) *Float32Statistics { ret := NewFloat32Statistics(descr, mem) ret.nvalues += nvalues - if hasNulls { - ret.incNulls(nulls) + if encoded.IsSetNullCount() { + ret.incNulls(encoded.GetNullCount()) } - if hasDistinct { - ret.incDistinct(distinct) + if encoded.IsSetDistinctCount() { + ret.incDistinct(encoded.GetDistinctCount()) } + encodedMin := encoded.GetMin() if encodedMin != nil && len(encodedMin) > 0 { ret.min = ret.plainDecode(encodedMin) } + encodedMax := encoded.GetMax() if encodedMax != nil && len(encodedMax) > 0 { ret.max = ret.plainDecode(encodedMax) } - ret.hasMinMax = hasMinMax + ret.hasMinMax = encoded.IsSetMax() || encoded.IsSetMin() return ret } @@ -1168,23 +1176,25 @@ func NewFloat64Statistics(descr *schema.Column, mem memory.Allocator) *Float64St // NewFloat64StatisticsFromEncoded will construct a propertly typed statistics object // initializing it with the provided information. -func NewFloat64StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *Float64Statistics { +func NewFloat64StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) *Float64Statistics { ret := NewFloat64Statistics(descr, mem) ret.nvalues += nvalues - if hasNulls { - ret.incNulls(nulls) + if encoded.IsSetNullCount() { + ret.incNulls(encoded.GetNullCount()) } - if hasDistinct { - ret.incDistinct(distinct) + if encoded.IsSetDistinctCount() { + ret.incDistinct(encoded.GetDistinctCount()) } + encodedMin := encoded.GetMin() if encodedMin != nil && len(encodedMin) > 0 { ret.min = ret.plainDecode(encodedMin) } + encodedMax := encoded.GetMax() if encodedMax != nil && len(encodedMax) > 0 { ret.max = ret.plainDecode(encodedMax) } - ret.hasMinMax = hasMinMax + ret.hasMinMax = encoded.IsSetMax() || encoded.IsSetMin() return ret } @@ -1445,23 +1455,25 @@ func NewBooleanStatistics(descr *schema.Column, mem memory.Allocator) *BooleanSt // NewBooleanStatisticsFromEncoded will construct a propertly typed statistics object // initializing it with the provided information. -func NewBooleanStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *BooleanStatistics { +func NewBooleanStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) *BooleanStatistics { ret := NewBooleanStatistics(descr, mem) ret.nvalues += nvalues - if hasNulls { - ret.incNulls(nulls) + if encoded.IsSetNullCount() { + ret.incNulls(encoded.GetNullCount()) } - if hasDistinct { - ret.incDistinct(distinct) + if encoded.IsSetDistinctCount() { + ret.incDistinct(encoded.GetDistinctCount()) } + encodedMin := encoded.GetMin() if encodedMin != nil && len(encodedMin) > 0 { ret.min = ret.plainDecode(encodedMin) } + encodedMax := encoded.GetMax() if encodedMax != nil && len(encodedMax) > 0 { ret.max = ret.plainDecode(encodedMax) } - ret.hasMinMax = hasMinMax + ret.hasMinMax = encoded.IsSetMax() || encoded.IsSetMin() return ret } @@ -1718,23 +1730,25 @@ func NewByteArrayStatistics(descr *schema.Column, mem memory.Allocator) *ByteArr // NewByteArrayStatisticsFromEncoded will construct a propertly typed statistics object // initializing it with the provided information. -func NewByteArrayStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *ByteArrayStatistics { +func NewByteArrayStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) *ByteArrayStatistics { ret := NewByteArrayStatistics(descr, mem) ret.nvalues += nvalues - if hasNulls { - ret.incNulls(nulls) + if encoded.IsSetNullCount() { + ret.incNulls(encoded.GetNullCount()) } - if hasDistinct { - ret.incDistinct(distinct) + if encoded.IsSetDistinctCount() { + ret.incDistinct(encoded.GetDistinctCount()) } + encodedMin := encoded.GetMin() if encodedMin != nil && len(encodedMin) > 0 { ret.min = ret.plainDecode(encodedMin) } + encodedMax := encoded.GetMax() if encodedMax != nil && len(encodedMax) > 0 { ret.max = ret.plainDecode(encodedMax) } - ret.hasMinMax = hasMinMax + ret.hasMinMax = encoded.IsSetMax() || encoded.IsSetMin() return ret } @@ -1986,23 +2000,25 @@ func NewFixedLenByteArrayStatistics(descr *schema.Column, mem memory.Allocator) // NewFixedLenByteArrayStatisticsFromEncoded will construct a propertly typed statistics object // initializing it with the provided information. -func NewFixedLenByteArrayStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *FixedLenByteArrayStatistics { +func NewFixedLenByteArrayStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) *FixedLenByteArrayStatistics { ret := NewFixedLenByteArrayStatistics(descr, mem) ret.nvalues += nvalues - if hasNulls { - ret.incNulls(nulls) + if encoded.IsSetNullCount() { + ret.incNulls(encoded.GetNullCount()) } - if hasDistinct { - ret.incDistinct(distinct) + if encoded.IsSetDistinctCount() { + ret.incDistinct(encoded.GetDistinctCount()) } + encodedMin := encoded.GetMin() if encodedMin != nil && len(encodedMin) > 0 { ret.min = ret.plainDecode(encodedMin) } + encodedMax := encoded.GetMax() if encodedMax != nil && len(encodedMax) > 0 { ret.max = ret.plainDecode(encodedMax) } - ret.hasMinMax = hasMinMax + ret.hasMinMax = encoded.IsSetMax() || encoded.IsSetMin() return ret } @@ -2265,27 +2281,27 @@ func NewStatistics(descr *schema.Column, mem memory.Allocator) TypedStatistics { // by checking the type of the provided column descriptor. // // If mem is nil, then memory.DefaultAllocator is used. -func NewStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) TypedStatistics { +func NewStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) TypedStatistics { if mem == nil { mem = memory.DefaultAllocator } switch descr.PhysicalType() { case parquet.Types.Int32: - return NewInt32StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + return NewInt32StatisticsFromEncoded(descr, mem, nvalues, encoded) case parquet.Types.Int64: - return NewInt64StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + return NewInt64StatisticsFromEncoded(descr, mem, nvalues, encoded) case parquet.Types.Int96: - return NewInt96StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + return NewInt96StatisticsFromEncoded(descr, mem, nvalues, encoded) case parquet.Types.Float: - return NewFloat32StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + return NewFloat32StatisticsFromEncoded(descr, mem, nvalues, encoded) case parquet.Types.Double: - return NewFloat64StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + return NewFloat64StatisticsFromEncoded(descr, mem, nvalues, encoded) case parquet.Types.Boolean: - return NewBooleanStatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + return NewBooleanStatisticsFromEncoded(descr, mem, nvalues, encoded) case parquet.Types.ByteArray: - return NewByteArrayStatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + return NewByteArrayStatisticsFromEncoded(descr, mem, nvalues, encoded) case parquet.Types.FixedLenByteArray: - return NewFixedLenByteArrayStatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + return NewFixedLenByteArrayStatisticsFromEncoded(descr, mem, nvalues, encoded) default: panic("not implemented") } diff --git a/go/parquet/metadata/statistics_types.gen.go.tmpl b/go/parquet/metadata/statistics_types.gen.go.tmpl index 66a483a3c59..4ee47a91725 100644 --- a/go/parquet/metadata/statistics_types.gen.go.tmpl +++ b/go/parquet/metadata/statistics_types.gen.go.tmpl @@ -63,23 +63,25 @@ func New{{.Name}}Statistics(descr *schema.Column, mem memory.Allocator) *{{.Name // New{{.Name}}StatisticsFromEncoded will construct a propertly typed statistics object // initializing it with the provided information. -func New{{.Name}}StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) *{{.Name}}Statistics { +func New{{.Name}}StatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) *{{.Name}}Statistics { ret := New{{.Name}}Statistics(descr, mem) ret.nvalues += nvalues - if hasNulls { - ret.incNulls(nulls) + if encoded.IsSetNullCount() { + ret.incNulls(encoded.GetNullCount()) } - if hasDistinct { - ret.incDistinct(distinct) + if encoded.IsSetDistinctCount() { + ret.incDistinct(encoded.GetDistinctCount()) } + encodedMin := encoded.GetMin() if encodedMin != nil && len(encodedMin) > 0 { ret.min = ret.plainDecode(encodedMin) } + encodedMax := encoded.GetMax() if encodedMax != nil && len(encodedMax) > 0 { ret.max = ret.plainDecode(encodedMax) } - ret.hasMinMax = hasMinMax + ret.hasMinMax = encoded.IsSetMax() || encoded.IsSetMin() return ret } @@ -403,14 +405,14 @@ func NewStatistics(descr *schema.Column, mem memory.Allocator) TypedStatistics { // by checking the type of the provided column descriptor. // // If mem is nil, then memory.DefaultAllocator is used. -func NewStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, encodedMin, encodedMax []byte, nvalues, nulls, distinct int64, hasMinMax, hasNulls, hasDistinct bool) TypedStatistics { +func NewStatisticsFromEncoded(descr *schema.Column, mem memory.Allocator, nvalues int64, encoded StatProvider) TypedStatistics { if mem == nil { mem = memory.DefaultAllocator } switch descr.PhysicalType() { {{- range .In}} case parquet.Types.{{if .physical}}{{.physical}}{{else}}{{.Name}}{{end}}: - return New{{.Name}}StatisticsFromEncoded(descr, mem, encodedMin, encodedMax, nvalues, nulls, distinct, hasMinMax, hasNulls, hasDistinct) + return New{{.Name}}StatisticsFromEncoded(descr, mem, nvalues, encoded) {{- end}} default: panic("not implemented") From 0d70b7386c76a2b55d9a7389666db8a7934983c7 Mon Sep 17 00:00:00 2001 From: Matthew Topol Date: Wed, 1 Sep 2021 14:24:04 -0400 Subject: [PATCH 6/7] rename fileEncProps --- go/parquet/metadata/file.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/go/parquet/metadata/file.go b/go/parquet/metadata/file.go index b97c4a4e5dc..30dcc98e9f2 100644 --- a/go/parquet/metadata/file.go +++ b/go/parquet/metadata/file.go @@ -126,10 +126,10 @@ func (f *FileMetaDataBuilder) Finish() (*FileMetaData, error) { f.metadata.ColumnOrders[idx] = colOrder } - fileEncProps := f.props.FileEncryptionProperties() - if fileEncProps != nil && !fileEncProps.EncryptedFooter() { + encryptProps := f.props.FileEncryptionProperties() + if encryptProps != nil && !encryptProps.EncryptedFooter() { var signingAlgo parquet.Algorithm - algo := fileEncProps.Algorithm() + algo := encryptProps.Algorithm() signingAlgo.Aad.AadFileUnique = algo.Aad.AadFileUnique signingAlgo.Aad.SupplyAadPrefix = algo.Aad.SupplyAadPrefix if !algo.Aad.SupplyAadPrefix { From 0a938c1f60af6d6e04ec4f0513e205be53256943 Mon Sep 17 00:00:00 2001 From: Matthew Topol Date: Sun, 12 Sep 2021 17:51:46 -0400 Subject: [PATCH 7/7] adding comments for clarifications. --- go/parquet/metadata/statistics.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/go/parquet/metadata/statistics.go b/go/parquet/metadata/statistics.go index 71155b767fc..85910c8425f 100644 --- a/go/parquet/metadata/statistics.go +++ b/go/parquet/metadata/statistics.go @@ -218,6 +218,10 @@ func (s *statistics) merge(other TypedStatistics) { s.stats.NullCount += other.NullCount() } if other.HasDistinctCount() { + // this isn't technically correct as it should be keeping an actual set + // of the distinct values and then combining the sets to get a new count + // but for now we'll do this to match the C++ implementation at the current + // time. s.stats.DistinctCount += other.DistinctCount() } } @@ -464,6 +468,9 @@ func (Int96Statistics) cleanStat(minMax minmaxPairInt96) *minmaxPairInt96 // - if any of min/max is NaN, return nothing // - if min is 0.0f replace with -0.0f // - if max is -0.0f replace with 0.0f +// +// https://issues.apache.org/jira/browse/PARQUET-1222 tracks the official documenting of +// a well-defined order for floats and doubles. func (Float32Statistics) cleanStat(minMax minmaxPairFloat32) *minmaxPairFloat32 { if math.IsNaN(float64(minMax[0])) || math.IsNaN(float64(minMax[1])) { return nil