parquet

package module
v0.0.0-...-bc21918 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Nov 12, 2021 License: Apache-2.0, BSD-2-Clause, BSD-3-Clause, + 8 more Imports: 14 Imported by: 0

Documentation

Overview

Package parquet provides an implementation of Apache Parquet for Go.

Apache Parquet is an open-source columnar data storage format using the record shredding and assembly algorithm to accomodate complex data structures which can then be used to efficiently store the data.

This implementation is a native go implementation for reading and writing the parquet file format.

Install

You can download the library via:

go get -u github.com/apache/arrow/go/parquet

In addition, two cli utilities are provided:

go install github.com/apache/arrow/go/parquet/cmd/parquet_reader
go install github.com/apache/arrow/go/parquet/cmd/parquet_schema

Modules

This top level parquet package contains the basic common types and reader/writer properties along with some utilities that are used throughout the other modules.

The file module contains the functions for directly reading/writing parquet files including Column Readers and Column Writers.

The metadata module contains the types for managing the lower level file/rowgroup/column metadata inside of a ParquetFile including inspecting the statistics.

The pqarrow module contains helper functions and types for converting directly between Parquet and Apache Arrow formats.

The schema module contains the types for manipulating / inspecting / creating parquet file schemas.

Primitive Types

The Parquet Primitive Types and their corresponding Go types are Boolean (bool), Int32 (int32), Int64 (int64), Int96 (parquet.Int96), Float (float32), Double (float64), ByteArray (parquet.ByteArray) and FixedLenByteArray (parquet.FixedLenByteArray).

Encodings

The encoding types supported in this package are: Plain, Plain/RLE Dictionary, Delta Binary Packed (only integer types), Delta Byte Array (only ByteArray), Delta Length Byte Array (only ByteArray)

Tip: Some platforms don't necessarily support all kinds of encodings. If you're not sure what to use, just use Plain and Dictionary encoding.

Index

Constants

View Source
const (
	// By default we'll use AesGCM as our encryption algorithm
	DefaultEncryptionAlgorithm       = AesGcm
	MaximalAadMetadataLength   int32 = 256
	// if encryption is turned on, we will default to also encrypting the footer
	DefaultEncryptedFooter = true
	DefaultCheckSignature  = true
	// by default if you set the file decryption properties, we will error
	// on any plaintext files unless otherwise specified.
	DefaultAllowPlaintextFiles       = false
	AadFileUniqueLength        int32 = 8
)

Constants that will be used as the default values with encryption/decryption

View Source
const (
	// Default Buffer size used for the Reader
	DefaultBufSize int64 = 4096 * 4
	// Default data page size limit is 1K it's not guaranteed, but we will try to
	// cut data pages off at this size where possible.
	DefaultDataPageSize int64 = 1024 * 1024
	// Default is for dictionary encoding to be turned on, use WithDictionaryDefault
	// writer property to change that.
	DefaultDictionaryEnabled = true
	// If the dictionary reaches the size of this limitation, the writer will use
	// the fallback encoding (usually plain) instead of continuing to build the
	// dictionary index.
	DefaultDictionaryPageSizeLimit = DefaultDataPageSize
	// In order to attempt to facilitate data page size limits for writing,
	// data is written in batches. Increasing the batch size may improve performance
	// but the larger the batch size, the easier it is to overshoot the datapage limit.
	DefaultWriteBatchSize int64 = 1024
	// Default maximum number of rows for a single row group
	DefaultMaxRowGroupLen int64 = 64 * 1024 * 1024
	// Default is to have stats enabled for all columns, use writer properties to
	// change the default, or to enable/disable for specific columns.
	DefaultStatsEnabled = true
	// If the stats are larger than 4K the writer will skip writing them out anyways.
	DefaultMaxStatsSize int64 = 4096
	DefaultCreatedBy          = "parquet-go version 1.0.0"
)

Constants for default property values used for the default reader, writer and column props.

View Source
const (

	// Int96SizeBytes is the number of bytes that make up an Int96
	Int96SizeBytes int = 12
)

Variables

View Source
var (
	// Int96Traits provides information about the Int96 type
	Int96Traits int96Traits
	// ByteArrayTraits provides information about the ByteArray type, which is just an []byte
	ByteArrayTraits byteArrayTraits
	// FixedLenByteArrayTraits provides information about the FixedLenByteArray type which is just an []byte
	FixedLenByteArrayTraits fixedLenByteArrayTraits
	// ByteArraySizeBytes is the number of bytes returned by reflect.TypeOf(ByteArray{}).Size()
	ByteArraySizeBytes int = int(reflect.TypeOf(ByteArray{}).Size())
	// FixedLenByteArraySizeBytes is the number of bytes returned by reflect.TypeOf(FixedLenByteArray{}).Size()
	FixedLenByteArraySizeBytes int = int(reflect.TypeOf(FixedLenByteArray{}).Size())
)
View Source
var (
	// Types contains constants for the Physical Types that are used in the Parquet Spec
	//
	// They can be specified when needed as such: `parquet.Types.Int32` etc. The values
	// all correspond to the values in parquet.thrift
	Types = struct {
		Boolean           Type
		Int32             Type
		Int64             Type
		Int96             Type
		Float             Type
		Double            Type
		ByteArray         Type
		FixedLenByteArray Type
		// this only exists as a convienence so we can denote it when necessary
		// nearly all functions that take a parquet.Type will error/panic if given
		// Undefined
		Undefined Type
	}{
		Boolean:           Type(format.Type_BOOLEAN),
		Int32:             Type(format.Type_INT32),
		Int64:             Type(format.Type_INT64),
		Int96:             Type(format.Type_INT96),
		Float:             Type(format.Type_FLOAT),
		Double:            Type(format.Type_DOUBLE),
		ByteArray:         Type(format.Type_BYTE_ARRAY),
		FixedLenByteArray: Type(format.Type_FIXED_LEN_BYTE_ARRAY),
		Undefined:         Type(format.Type_FIXED_LEN_BYTE_ARRAY + 1),
	}

	// Encodings contains constants for the encoding types of the column data
	//
	// The values used all correspond to the values in parquet.thrift for the
	// corresponding encoding type.
	Encodings = struct {
		Plain                Encoding
		PlainDict            Encoding
		RLE                  Encoding
		RLEDict              Encoding
		BitPacked            Encoding // deprecated, not implemented
		DeltaByteArray       Encoding
		DeltaBinaryPacked    Encoding
		DeltaLengthByteArray Encoding
	}{
		Plain:                Encoding(format.Encoding_PLAIN),
		PlainDict:            Encoding(format.Encoding_PLAIN_DICTIONARY),
		RLE:                  Encoding(format.Encoding_RLE),
		RLEDict:              Encoding(format.Encoding_RLE_DICTIONARY),
		BitPacked:            Encoding(format.Encoding_BIT_PACKED),
		DeltaByteArray:       Encoding(format.Encoding_DELTA_BYTE_ARRAY),
		DeltaBinaryPacked:    Encoding(format.Encoding_DELTA_BINARY_PACKED),
		DeltaLengthByteArray: Encoding(format.Encoding_DELTA_LENGTH_BYTE_ARRAY),
	}

	// ColumnOrders contains constants for the Column Ordering fields
	ColumnOrders = struct {
		Undefined        ColumnOrder
		TypeDefinedOrder ColumnOrder
	}{
		Undefined:        format.NewColumnOrder(),
		TypeDefinedOrder: &format.ColumnOrder{TYPE_ORDER: format.NewTypeDefinedOrder()},
	}

	// DefaultColumnOrder is to use TypeDefinedOrder
	DefaultColumnOrder = ColumnOrders.TypeDefinedOrder

	// Repetitions contains the constants for Field Repetition Types
	Repetitions = struct {
		Required  Repetition
		Optional  Repetition
		Repeated  Repetition
		Undefined Repetition // convenience value
	}{
		Required:  Repetition(format.FieldRepetitionType_REQUIRED),
		Optional:  Repetition(format.FieldRepetitionType_OPTIONAL),
		Repeated:  Repetition(format.FieldRepetitionType_REPEATED),
		Undefined: Repetition(format.FieldRepetitionType_REPEATED + 1),
	}
)

Functions

This section is empty.

Types

type AADPrefixVerifier

type AADPrefixVerifier interface {
	// Verify identity of file. panic if bad
	Verify(string)
}

AADPrefixVerifier is an interface for any object that can be used to verify the identity of the file being decrypted. It should panic if the provided AAD identity is bad.

In a data set, AAD Prefixes should be collected, and then checked for missing files.

type Algorithm

type Algorithm struct {
	Algo Cipher
	Aad  struct {
		AadPrefix       []byte
		AadFileUnique   []byte
		SupplyAadPrefix bool
	}
}

Algorithm describes how something was encrypted, representing the EncryptionAlgorithm object from the parquet.thrift file.

func AlgorithmFromThrift

func AlgorithmFromThrift(enc *format.EncryptionAlgorithm) (ret Algorithm)

AlgorithmFromThrift converts the thrift object to the Algorithm struct for easier usage.

func (Algorithm) ToThrift

func (e Algorithm) ToThrift() *format.EncryptionAlgorithm

ToThrift returns an instance to be used for serializing when writing a file.

type ByteArray

type ByteArray []byte

ByteArray is a type to be utilized for representing the Parquet ByteArray physical type, represented as a byte slice

func (ByteArray) Len

func (b ByteArray) Len() int

Len returns the current length of the ByteArray, equivalent to len(bytearray)

func (ByteArray) String

func (b ByteArray) String() string

String returns a string representation of the ByteArray

type Cipher

type Cipher int

Cipher is the parquet Cipher Algorithms

const (
	AesGcm Cipher = iota
	AesCtr
)

constants for choosing the Aes Algorithm to use for encryption/decryption

type ColumnDecryptOption

type ColumnDecryptOption func(*columnDecryptConfig)

ColumnDecryptOption is the type of the options passed for constructing Decryption Properties

func WithDecryptKey

func WithDecryptKey(key string) ColumnDecryptOption

WithDecryptKey specifies the key to utilize for decryption

type ColumnDecryptionProperties

type ColumnDecryptionProperties struct {
	// contains filtered or unexported fields
}

ColumnDecryptionProperties are the specifications for how to decrypt a given column.

func NewColumnDecryptionProperties

func NewColumnDecryptionProperties(column string, opts ...ColumnDecryptOption) *ColumnDecryptionProperties

NewColumnDecryptionProperties constructs a new ColumnDecryptionProperties for the given column path, modified by the provided options

func (*ColumnDecryptionProperties) Clone

Clone returns a new instance of ColumnDecryptionProperties with the same key and column

func (*ColumnDecryptionProperties) ColumnPath

func (cd *ColumnDecryptionProperties) ColumnPath() string

ColumnPath returns which column these properties describe how to decrypt

func (*ColumnDecryptionProperties) IsUtilized

func (cd *ColumnDecryptionProperties) IsUtilized() bool

IsUtilized returns whether or not these properties have been used for decryption already

func (*ColumnDecryptionProperties) Key

Key returns the key specified to decrypt this column, or is empty if the Footer Key should be used.

func (*ColumnDecryptionProperties) SetUtilized

func (cd *ColumnDecryptionProperties) SetUtilized()

SetUtilized is used by the reader to specify when we've decrypted the column and have used the key so we know to wipe out the keys.

func (*ColumnDecryptionProperties) WipeOutDecryptionKey

func (cd *ColumnDecryptionProperties) WipeOutDecryptionKey()

WipeOutDecryptionKey is called after decryption to ensure the key doesn't stick around and get re-used.

type ColumnEncryptOption

type ColumnEncryptOption func(*colEncryptConfig)

ColumnEncryptOption how to specify options to the the NewColumnEncryptionProperties function.

func WithKey

func WithKey(key string) ColumnEncryptOption

WithKey sets a column specific key. If key is not set on an encrypted column, the column will be encrypted with the footer key. key length must be either 16, 24, or 32 bytes the key is cloned and will be wiped out (array values set to 0) upon completion of file writing. Caller is responsible for wiping out input key array

func WithKeyID

func WithKeyID(keyID string) ColumnEncryptOption

WithKeyID is a convenience function to set the key metadata using a string id. Set a key retrieval metadata (converted from String). and use either KeyMetadata or KeyID, not both. KeyID will be converted to metadata (UTF-8 Array)

func WithKeyMetadata

func WithKeyMetadata(keyMeta string) ColumnEncryptOption

WithKeyMetadata sets the key retrieval metadata, use either KeyMetadata or KeyID but not both

type ColumnEncryptionProperties

type ColumnEncryptionProperties struct {
	// contains filtered or unexported fields
}

ColumnEncryptionProperties specifies how to encrypt a given column

func NewColumnEncryptionProperties

func NewColumnEncryptionProperties(name string, opts ...ColumnEncryptOption) *ColumnEncryptionProperties

NewColumnEncryptionProperties constructs properties for the provided column path, modified by the options provided

func (*ColumnEncryptionProperties) Clone

Clone returns a instance of ColumnEncryptionProperties with the same key and metadata

func (*ColumnEncryptionProperties) ColumnPath

func (ce *ColumnEncryptionProperties) ColumnPath() string

ColumnPath returns which column these properties are for

func (*ColumnEncryptionProperties) IsEncrypted

func (ce *ColumnEncryptionProperties) IsEncrypted() bool

IsEncrypted returns true if this column is encrypted.

func (*ColumnEncryptionProperties) IsEncryptedWithFooterKey

func (ce *ColumnEncryptionProperties) IsEncryptedWithFooterKey() bool

IsEncryptedWithFooterKey returns if this column was encrypted with the footer key itself, or false if a separate key was used for encrypting this column.

func (*ColumnEncryptionProperties) IsUtilized

func (ce *ColumnEncryptionProperties) IsUtilized() bool

IsUtilized returns whether or not these properties have already been used, if the key is empty then this is always false

func (*ColumnEncryptionProperties) Key

Key returns the key used for encrypting this column if it isn't encrypted by the footer key

func (*ColumnEncryptionProperties) KeyMetadata

func (ce *ColumnEncryptionProperties) KeyMetadata() string

KeyMetadata returns the key identifier which is used with a KeyRetriever to get the key for this column if it is not encrypted using the footer key

func (*ColumnEncryptionProperties) SetUtilized

func (ce *ColumnEncryptionProperties) SetUtilized()

SetUtilized is used for marking it as utilized once it is used in FileEncryptionProperties as the encryption key will be wiped out on completion of writing

func (*ColumnEncryptionProperties) WipeOutEncryptionKey

func (ce *ColumnEncryptionProperties) WipeOutEncryptionKey()

WipeOutEncryptionKey Clears the encryption key, used after completion of file writing

type ColumnOrder

type ColumnOrder *format.ColumnOrder

ColumnOrder is the Column Order from the parquet.thrift

type ColumnPath

type ColumnPath []string

ColumnPath is the path from the root of the schema to a given column

func ColumnPathFromString

func ColumnPathFromString(s string) ColumnPath

ColumnPathFromString constructs a ColumnPath from a dot separated string

func (ColumnPath) Extend

func (c ColumnPath) Extend(s string) ColumnPath

Extend creates a new ColumnPath from an existing one, with the new ColumnPath having s appended to the end.

func (ColumnPath) String

func (c ColumnPath) String() string

type ColumnPathToDecryptionPropsMap

type ColumnPathToDecryptionPropsMap map[string]*ColumnDecryptionProperties

ColumnPathToDecryptionPropsMap maps column paths to decryption properties

type ColumnPathToEncryptionPropsMap

type ColumnPathToEncryptionPropsMap map[string]*ColumnEncryptionProperties

ColumnPathToEncryptionPropsMap maps column paths to encryption properties

type ColumnProperties

type ColumnProperties struct {
	Encoding          Encoding
	Codec             compress.Compression
	DictionaryEnabled bool
	StatsEnabled      bool
	MaxStatsSize      int64
	CompressionLevel  int
}

ColumnProperties defines the encoding, codec, and so on for a given column.

func DefaultColumnProperties

func DefaultColumnProperties() ColumnProperties

DefaultColumnProperties returns the default properties which get utilized for writing.

The default column properties are the following constants:

Encoding:						Encodings.Plain
Codec:							compress.Codecs.Uncompressed
DictionaryEnabled:	DefaultDictionaryEnabled
StatsEnabled:				DefaultStatsEnabled
MaxStatsSize:				DefaultMaxStatsSize
CompressionLevel:		compress.DefaultCompressionLevel

type DataPageVersion

type DataPageVersion int8

DataPageVersion is the version of the Parquet Data Pages

const (
	DataPageV1 DataPageVersion = iota
	DataPageV2
)

constants for the parquet DataPage Version to use

type DecryptionKeyRetriever

type DecryptionKeyRetriever interface {
	GetKey(keyMetadata []byte) string
}

DecryptionKeyRetriever is an interface for getting the desired key for decryption from metadata. It should take in some metadata identifier and return the actual Key to use for decryption.

type Encoding

type Encoding format.Encoding

Encoding is the parquet Encoding type

func (Encoding) String

func (e Encoding) String() string

type EncryptOption

type EncryptOption func(*configEncrypt)

EncryptOption is used for specifying values when building FileEncryptionProperties

func DisableAadPrefixStorage

func DisableAadPrefixStorage() EncryptOption

DisableAadPrefixStorage will set the properties to not store the AadPrefix in the file. If this isn't called and the AadPrefix is set, then it will be stored. This needs to in the options *after* WithAadPrefix to have an effect.

func WithAadPrefix

func WithAadPrefix(aadPrefix string) EncryptOption

WithAadPrefix sets the AAD prefix to use for encryption and by default will store it in the file

func WithAlg

func WithAlg(cipher Cipher) EncryptOption

WithAlg sets the encryption algorithm to utilize. (default is AesGcm)

func WithEncryptedColumns

func WithEncryptedColumns(encrypted ColumnPathToEncryptionPropsMap) EncryptOption

WithEncryptedColumns sets the map of columns and their properties (keys etc.) If not called, then all columns will be encrypted with the footer key. If called, then columns not in the map will be left unencrypted.

func WithFooterKeyID

func WithFooterKeyID(key string) EncryptOption

WithFooterKeyID sets a key retrieval metadata to use (converted from string), this must be a utf8 string.

use either WithFooterKeyID or WithFooterKeyMetadata, not both.

func WithFooterKeyMetadata

func WithFooterKeyMetadata(keyMeta string) EncryptOption

WithFooterKeyMetadata sets a key retrieval metadata to use for getting the key.

Use either WithFooterKeyID or WithFooterKeyMetadata, not both.

func WithPlaintextFooter

func WithPlaintextFooter() EncryptOption

WithPlaintextFooter sets the writer to write the footer in plain text, otherwise the footer will be encrypted too (which is the default behavior).

type FileDecryptionOption

type FileDecryptionOption func(*fileDecryptConfig)

FileDecryptionOption is how to supply options to constructing a new FileDecryptionProperties instance.

func DisableFooterSignatureVerification

func DisableFooterSignatureVerification() FileDecryptionOption

DisableFooterSignatureVerification skips integrity verification of plaintext footers.

If not called, integrity of plaintext footers will be checked in runtime, and will panic if the footer signing key is not available or if the footer content and signature don't match

func WithColumnKeys

WithColumnKeys sets explicit column keys.

It's also possible to set a key retriever on this property object.

Upon file decryption, availability of explicit keys is checked before invocation of the retreiver callback.

If an explicit key is available for a footer or a column, its key metadata will be ignored.

func WithDecryptAadPrefix

func WithDecryptAadPrefix(prefix string) FileDecryptionOption

WithDecryptAadPrefix explicitly supplies the file aad prefix.

A must when a prefix is used for file encryption, but not stored in the file.

func WithFooterKey

func WithFooterKey(key string) FileDecryptionOption

WithFooterKey sets an explicit footer key. If Applied on a file that contains footer key metadata the metadata will be ignored, the footer will be decrypted/verified with this key.

If the explicit key is not set, footer key will be fetched from the key retriever. With explcit keys or AAD prefix, new encryption properties object must be created for each encrypted file.

Explicit encryption keys (footer and column) are cloned. Upon completion of file reading, the cloned encryption keys in the properties will be wiped out Caller is responsible for wiping out the input key array footer key length must be either 16, 24, or 32 bytes

func WithKeyRetriever

func WithKeyRetriever(retriever DecryptionKeyRetriever) FileDecryptionOption

WithKeyRetriever sets a key retriever callback. It's also possible to set explicit footer or column keys.

func WithPlaintextAllowed

func WithPlaintextAllowed() FileDecryptionOption

WithPlaintextAllowed sets allowing plaintext files.

By default, reading plaintext (unencrypted) files is not allowed when using a decryptor.

In order to detect files that were not encrypted by mistake. However the default behavior can be overridden by using this method.

func WithPrefixVerifier

func WithPrefixVerifier(verifier AADPrefixVerifier) FileDecryptionOption

WithPrefixVerifier supplies a verifier object to use for verifying the AAD Prefixes stored in the file.

type FileDecryptionProperties

type FileDecryptionProperties struct {
	Verifier     AADPrefixVerifier
	KeyRetriever DecryptionKeyRetriever
	// contains filtered or unexported fields
}

FileDecryptionProperties define the File Level configuration for decrypting a parquet file. Once constructed they are read only.

func NewFileDecryptionProperties

func NewFileDecryptionProperties(opts ...FileDecryptionOption) *FileDecryptionProperties

NewFileDecryptionProperties takes in the options for constructing a new FileDecryptionProperties object, otherwise it will use the default configuration which will check footer integrity of a plaintext footer for an encrypted file for unencrypted parquet files, the decryption properties should not be set.

func (*FileDecryptionProperties) AadPrefix

func (fd *FileDecryptionProperties) AadPrefix() string

AadPrefix returns the prefix to be supplied for constructing the identification strings when decrypting

func (*FileDecryptionProperties) Clone

func (fd *FileDecryptionProperties) Clone(newAadPrefix string) *FileDecryptionProperties

Clone returns a new instance of these properties, changing the prefix if set (keeping the same prefix if left empty)

func (*FileDecryptionProperties) ColumnKey

func (fd *FileDecryptionProperties) ColumnKey(path string) string

ColumnKey returns the key to be used for decrypting the provided column.

func (*FileDecryptionProperties) FooterKey

func (fd *FileDecryptionProperties) FooterKey() string

FooterKey returns the key utilized for decrypting the Footer if encrypted and any columns that are encrypted with the footer key.

func (*FileDecryptionProperties) IsUtilized

func (fd *FileDecryptionProperties) IsUtilized() bool

IsUtilized returns whether or not this instance has been used to decrypt a file. If the footer key and prefix are empty and there are no column decryption properties, then this is always false.

func (*FileDecryptionProperties) PlaintextFilesAllowed

func (fd *FileDecryptionProperties) PlaintextFilesAllowed() bool

PlaintextFilesAllowed returns whether or not this instance of decryption properties are allowed on a plaintext file.

func (*FileDecryptionProperties) PlaintextFooterIntegrity

func (fd *FileDecryptionProperties) PlaintextFooterIntegrity() bool

PlaintextFooterIntegrity returns whether or not an integrity check will be performed on a plaintext footer for an encrypted file.

func (*FileDecryptionProperties) SetUtilized

func (fd *FileDecryptionProperties) SetUtilized()

SetUtilized is called to mark this instance as utilized once it is used to read a file. A single instance can be used for reading one file only. Setting this ensures the keys will be wiped out upon completion of file reading.

func (*FileDecryptionProperties) WipeOutDecryptionKeys

func (fd *FileDecryptionProperties) WipeOutDecryptionKeys()

WipeOutDecryptionKeys will clear all the keys for this instance including the column level ones, this will be called after this instance has been utilized.

type FileEncryptionProperties

type FileEncryptionProperties struct {
	// contains filtered or unexported fields
}

FileEncryptionProperties describe how to encrypt a parquet file when writing data.

func NewFileEncryptionProperties

func NewFileEncryptionProperties(footerKey string, opts ...EncryptOption) *FileEncryptionProperties

NewFileEncryptionProperties returns a new File Encryption description object using the options provided.

func (*FileEncryptionProperties) Algorithm

func (fe *FileEncryptionProperties) Algorithm() Algorithm

Algorithm returns the description of how we will perform the encryption, the algorithm, prefixes, and so on.

func (*FileEncryptionProperties) Clone

func (fe *FileEncryptionProperties) Clone(newAadPrefix string) *FileEncryptionProperties

Clone allows returning an identical property setup for another file with the option to update the aadPrefix, (if given the empty string, the current aad prefix will be used) since a single instance can only be used to encrypt one file before wiping out the keys.

func (*FileEncryptionProperties) ColumnEncryptionProperties

func (fe *FileEncryptionProperties) ColumnEncryptionProperties(path string) *ColumnEncryptionProperties

ColumnEncryptionProperties returns the properties for encrypting a given column.

This may be nil for columns that aren't encrypted or may be default properties.

func (*FileEncryptionProperties) EncryptedColumns

EncryptedColumns returns the mapping of column paths to column encryption properties

func (*FileEncryptionProperties) EncryptedFooter

func (fe *FileEncryptionProperties) EncryptedFooter() bool

EncryptedFooter returns if the footer for this file should be encrypted or left in plaintext.

func (*FileEncryptionProperties) FileAad

func (fe *FileEncryptionProperties) FileAad() string

FileAad returns the aad identification to be used at the file level which gets concatenated with the row and column information for encrypting data.

func (*FileEncryptionProperties) FooterKey

func (fe *FileEncryptionProperties) FooterKey() string

FooterKey returns the actual key used to encrypt the footer if it is encrypted, or to encrypt any columns which will be encrypted with it rather than their own keys.

func (*FileEncryptionProperties) FooterKeyMetadata

func (fe *FileEncryptionProperties) FooterKeyMetadata() string

FooterKeyMetadata is used for retrieving a key from the key retriever in order to set the footer key

func (*FileEncryptionProperties) IsUtilized

func (fe *FileEncryptionProperties) IsUtilized() bool

IsUtilized returns whether or not this instance has been used to encrypt a file

func (*FileEncryptionProperties) SetUtilized

func (fe *FileEncryptionProperties) SetUtilized()

SetUtilized is called after writing a file. A FileEncryptionProperties object can be used for writing one file only, the encryption keys will be wiped out upon completion of writing the file.

func (*FileEncryptionProperties) WipeOutEncryptionKeys

func (fe *FileEncryptionProperties) WipeOutEncryptionKeys()

WipeOutEncryptionKeys clears all of the encryption keys for this and the columns

type FixedLenByteArray

type FixedLenByteArray []byte

FixedLenByteArray is a go type to represent a FixedLengthByteArray as a byte slice

func (FixedLenByteArray) Len

func (b FixedLenByteArray) Len() int

Len returns the current length of this FixedLengthByteArray, equivalent to len(fixedlenbytearray)

func (FixedLenByteArray) String

func (b FixedLenByteArray) String() string

String returns a string representation of the FixedLenByteArray

type Int96

type Int96 [12]byte

Int96 is a 12 byte integer value utilized for representing timestamps as a 64 bit integer and a 32 bit integer.

func NewInt96

func NewInt96(v [3]uint32) (out Int96)

NewInt96 creates a new Int96 from the given 3 uint32 values.

func (*Int96) SetNanoSeconds

func (i96 *Int96) SetNanoSeconds(nanos int64)

SetNanoSeconds sets the Nanosecond field of the Int96 timestamp to the provided value

func (Int96) String

func (i96 Int96) String() string

String provides the string representation as a timestamp via converting to a time.Time and then calling String

func (Int96) ToTime

func (i96 Int96) ToTime() time.Time

ToTime returns a go time.Time object that represents the same time instant as the given Int96 value

type ReaderAtSeeker

type ReaderAtSeeker interface {
	io.ReaderAt
	io.ReadSeeker
}

ReaderAtSeeker is a combination of the ReaderAt and ReadSeeker interfaces from the io package defining the only functionality that is required in order for a parquet file to be read by the file functions. We just need to be able to call ReadAt, Read, and Seek

type ReaderProperties

type ReaderProperties struct {

	// Default buffer size to utilize when reading chunks
	BufferSize int64
	// create with NewFileDecryptionProperties if dealing with an encrypted file
	FileDecryptProps *FileDecryptionProperties
	// If this is set to true, then the reader will use SectionReader to
	// just use the read stream when reading data. Otherwise we will buffer
	// the data we're going to read into memory first and then read that buffer.
	//
	// If reading from higher latency IO, like S3, it might improve performance to
	// set this to true in order to read the entire row group in at once rather than
	// make multiple smaller data requests. For low latency IO streams or if only
	// reading small portions / subsets  of the parquet file, this can be set to false
	// to reduce the amount of IO performed in order to avoid reading excess amounts of data.
	BufferedStreamEnabled bool
	// contains filtered or unexported fields
}

ReaderProperties are used to define how the file reader will handle buffering and allocating buffers

func NewReaderProperties

func NewReaderProperties(alloc memory.Allocator) *ReaderProperties

NewReaderProperties returns the default Reader Properties using the provided allocator.

If nil is passed for the allocator, then memory.DefaultAllocator will be used.

func (*ReaderProperties) Allocator

func (r *ReaderProperties) Allocator() memory.Allocator

Allocator returns the allocator that the properties were initialized with

func (*ReaderProperties) GetStream

func (r *ReaderProperties) GetStream(source io.ReaderAt, start, nbytes int64) (ReaderAtSeeker, error)

GetStream returns a section of the underlying reader based on whether or not BufferedStream is enabled.

If BufferedStreamEnabled is true, it creates an io.SectionReader, otherwise it will read the entire section into a buffer in memory and return a bytes.NewReader for that buffer.

type Repetition

type Repetition format.FieldRepetitionType

Repetition is the underlying parquet field repetition type as in parquet.thrift

func (Repetition) String

func (r Repetition) String() string

type Type

type Type format.Type

Type is the physical type as in parquet.thrift

func (Type) ByteSize

func (t Type) ByteSize() int

ByteSize returns the number of bytes required to store a single value of the given parquet.Type in memory.

func (Type) String

func (t Type) String() string

type Version

type Version int8

Version is the parquet version type

const (
	// 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 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.

type WriterProperties

type WriterProperties struct {
	// contains filtered or unexported fields
}

WriterProperties is the collection of properties to use for writing a parquet file. The values are read only once it has been constructed.

func NewWriterProperties

func NewWriterProperties(opts ...WriterProperty) *WriterProperties

NewWriterProperties takes a list of options for building the properties. If multiple options are used which conflict then the last option is the one which will take effect. If no WriterProperty options are provided, then the default properties will be utilized for writing.

The Default properties use the following constants:

Allocator:					memory.DefaultAllocator
DictionaryPageSize: DefaultDictionaryPageSizeLimit
BatchSize:					DefaultWriteBatchSize
MaxRowGroupLength:	DefaultMaxRowGroupLen
PageSize:						DefaultDataPageSize
ParquetVersion:			V1
DataPageVersion:		DataPageV1
CreatedBy:					DefaultCreatedBy

func (*WriterProperties) Allocator

func (w *WriterProperties) Allocator() memory.Allocator

func (*WriterProperties) ColumnEncryptionProperties

func (w *WriterProperties) ColumnEncryptionProperties(path string) *ColumnEncryptionProperties

ColumnEncryptionProperties returns the specific properties for encryption that will be used for the given column path

func (*WriterProperties) Compression

func (w *WriterProperties) Compression() compress.Compression

Compression returns the default compression type that will be used for any columns that don't have a specific compression defined.

func (*WriterProperties) CompressionFor

func (w *WriterProperties) CompressionFor(path string) compress.Compression

CompressionFor will return the compression type that is specified for the given column path, or the default compression codec if there isn't one specific to this column.

func (*WriterProperties) CompressionLevel

func (w *WriterProperties) CompressionLevel() int

CompressionLevel returns the default compression level that will be used for any column that doesn't have a compression level specified for it.

func (*WriterProperties) CompressionLevelFor

func (w *WriterProperties) CompressionLevelFor(path string) int

CompressionLevelFor returns the compression level that will be utilized for the given column, or the default compression level if the column doesn't have a specific level specified.

func (*WriterProperties) CompressionLevelPath

func (w *WriterProperties) CompressionLevelPath(path ColumnPath) int

CompressionLevelPath is the same as CompressionLevelFor but takes a ColumnPath object

func (*WriterProperties) CompressionPath

func (w *WriterProperties) CompressionPath(path ColumnPath) compress.Compression

CompressionPath is the same as CompressionFor but takes a ColumnPath

func (*WriterProperties) CreatedBy

func (w *WriterProperties) CreatedBy() string

func (*WriterProperties) DataPageSize

func (w *WriterProperties) DataPageSize() int64

func (*WriterProperties) DataPageVersion

func (w *WriterProperties) DataPageVersion() DataPageVersion

func (*WriterProperties) DictionaryEnabled

func (w *WriterProperties) DictionaryEnabled() bool

DictionaryEnabled returns the default value as for whether or not dictionary encoding will be utilized for columns that aren't separately specified.

func (*WriterProperties) DictionaryEnabledFor

func (w *WriterProperties) DictionaryEnabledFor(path string) bool

DictionaryEnabledFor returns whether or not dictionary encoding will be used for the specified column when writing or the default value if the column was not separately specified.

func (*WriterProperties) DictionaryEnabledPath

func (w *WriterProperties) DictionaryEnabledPath(path ColumnPath) bool

DictionaryEnabledPath is the same as DictionaryEnabledFor but takes a ColumnPath object.

func (*WriterProperties) DictionaryIndexEncoding

func (w *WriterProperties) DictionaryIndexEncoding() 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 (*WriterProperties) DictionaryPageEncoding

func (w *WriterProperties) DictionaryPageEncoding() 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 (*WriterProperties) DictionaryPageSizeLimit

func (w *WriterProperties) DictionaryPageSizeLimit() int64

func (*WriterProperties) Encoding

func (w *WriterProperties) Encoding() Encoding

Encoding returns the default encoding that will be utilized for any columns which don't have a different value specified.

func (*WriterProperties) EncodingFor

func (w *WriterProperties) EncodingFor(path string) Encoding

EncodingFor returns the encoding that will be used for the given column path, or the default encoding if there isn't one specified for this column.

func (*WriterProperties) EncodingPath

func (w *WriterProperties) EncodingPath(path ColumnPath) Encoding

EncodingPath is the same as EncodingFor but takes a ColumnPath object

func (*WriterProperties) FileEncryptionProperties

func (w *WriterProperties) FileEncryptionProperties() *FileEncryptionProperties

FileEncryptionProperties returns the current encryption properties that were used to create the writer properties.

func (*WriterProperties) MaxRowGroupLength

func (w *WriterProperties) MaxRowGroupLength() int64

func (*WriterProperties) MaxStatsSize

func (w *WriterProperties) MaxStatsSize() int64

MaxStatsSize returns the default maximum size for stats

func (*WriterProperties) MaxStatsSizeFor

func (w *WriterProperties) MaxStatsSizeFor(path string) int64

MaxStatsSizeFor returns the maximum stat size for the given column path

func (*WriterProperties) MaxStatsSizePath

func (w *WriterProperties) MaxStatsSizePath(path ColumnPath) int64

MaxStatsSizePath is the same as MaxStatsSizeFor but takes a ColumnPath

func (*WriterProperties) StatisticsEnabled

func (w *WriterProperties) StatisticsEnabled() bool

StatisticsEnabled returns the default value for whether or not stats are enabled to be written for columns that aren't separately specified.

func (*WriterProperties) StatisticsEnabledFor

func (w *WriterProperties) StatisticsEnabledFor(path string) bool

StatisticsEnabledFor returns whether stats will be written for the given column path, or the default value if it wasn't separately specified.

func (*WriterProperties) StatisticsEnabledPath

func (w *WriterProperties) StatisticsEnabledPath(path ColumnPath) bool

StatisticsEnabledPath is the same as StatisticsEnabledFor but takes a ColumnPath object.

func (*WriterProperties) Version

func (w *WriterProperties) Version() Version

func (*WriterProperties) WriteBatchSize

func (w *WriterProperties) WriteBatchSize() int64

type WriterProperty

type WriterProperty func(*writerPropConfig)

WriterProperty is used as the options for building a writer properties instance

func WithAllocator

func WithAllocator(mem memory.Allocator) WriterProperty

WithAllocator specifies the writer to use the given allocator

func WithBatchSize

func WithBatchSize(batch int64) WriterProperty

WithBatchSize specifies the number of rows to use for batch writes to columns

func WithCompression

func WithCompression(codec compress.Compression) WriterProperty

WithCompression specifies the default compression type to use for column writing.

func WithCompressionFor

func WithCompressionFor(path string, codec compress.Compression) WriterProperty

WithCompressionFor specifies the compression type for the given column.

func WithCompressionLevel

func WithCompressionLevel(level int) WriterProperty

WithCompressionLevel specifies the default compression level for the compressor in every column.

The provided compression level is compressor specific. The user would have to know what the available levels are for the selected compressor. If the compressor does not allow for selecting different compression levels, then this function will have no effect. Parquet and Arrow will not validate the passed compression level. If no level is selected by the user or if the special compress.DefaultCompressionLevel value is used, then parquet will select the compression level.

func WithCompressionLevelFor

func WithCompressionLevelFor(path string, level int) WriterProperty

WithCompressionLevelFor is like WithCompressionLevel but only for the given column path.

func WithCompressionLevelPath

func WithCompressionLevelPath(path ColumnPath, level int) WriterProperty

WithCompressionLevelPath is the same as WithCompressionLevelFor but takes a ColumnPath

func WithCompressionPath

func WithCompressionPath(path ColumnPath, codec compress.Compression) WriterProperty

WithCompressionPath is the same as WithCompressionFor but takes a ColumnPath directly.

func WithCreatedBy

func WithCreatedBy(createdby string) WriterProperty

WithCreatedBy specifies the "created by" string to use for the writer

func WithDataPageSize

func WithDataPageSize(pgsize int64) WriterProperty

WithDataPageSize specifies the size to use for splitting data pages for column writing.

func WithDataPageVersion

func WithDataPageVersion(version DataPageVersion) WriterProperty

WithDataPageVersion specifies whether to use Version 1 or Version 2 of the DataPage spec

func WithDictionaryDefault

func WithDictionaryDefault(dict bool) WriterProperty

WithDictionaryDefault sets the default value for whether to enable dictionary encoding

func WithDictionaryFor

func WithDictionaryFor(path string, dict bool) WriterProperty

WithDictionaryFor allows enabling or disabling dictionary encoding for a given column path string

func WithDictionaryPageSizeLimit

func WithDictionaryPageSizeLimit(limit int64) WriterProperty

WithDictionaryPageSizeLimit is the limit of the dictionary at which the writer will fallback to plain encoding instead

func WithDictionaryPath

func WithDictionaryPath(path ColumnPath, dict bool) WriterProperty

WithDictionaryPath is like WithDictionaryFor, but takes a ColumnPath type

func WithEncoding

func WithEncoding(encoding Encoding) WriterProperty

WithEncoding defines the encoding that is used when we aren't using dictionary encoding.

This is either applied if dictionary encoding is disabled, or if we fallback if the dictionary grew too large.

func WithEncodingFor

func WithEncodingFor(path string, encoding Encoding) WriterProperty

WithEncodingFor is for defining the encoding only for a specific column path. This encoding will be used if dictionary encoding is disabled for the column or if we fallback because the dictionary grew too large

func WithEncodingPath

func WithEncodingPath(path ColumnPath, encoding Encoding) WriterProperty

WithEncodingPath is the same as WithEncodingFor but takes a ColumnPath directly.

func WithEncryptionProperties

func WithEncryptionProperties(props *FileEncryptionProperties) WriterProperty

WithEncryptionProperties specifies the file level encryption handling for writing the file.

func WithMaxRowGroupLength

func WithMaxRowGroupLength(nrows int64) WriterProperty

WithMaxRowGroupLength specifies the number of rows as the maximum number of rows for a given row group in the writer.

func WithMaxStatsSize

func WithMaxStatsSize(maxStatsSize int64) WriterProperty

WithMaxStatsSize sets a maximum size for the statistics before we decide not to include them.

func WithStats

func WithStats(enabled bool) WriterProperty

WithStats specifies a default for whether or not to enable column statistics.

func WithStatsFor

func WithStatsFor(path string, enabled bool) WriterProperty

WithStatsFor specifies a per column value as to enable or disable statistics in the resulting file.

func WithStatsPath

func WithStatsPath(path ColumnPath, enabled bool) WriterProperty

WithStatsPath is the same as WithStatsFor but takes a ColumnPath

func WithVersion

func WithVersion(version Version) WriterProperty

WithVersion specifies which Parquet Spec version to utilize for writing.

Directories

Path Synopsis
Package compress contains the interfaces and implementations for handling compression/decompression of parquet data at the column levels.
Package compress contains the interfaces and implementations for handling compression/decompression of parquet data at the column levels.
internal
bmi
Package bmi contains helpers for manipulating bitmaps via BMI2 extensions properly falling back to pure go implementations if the CPU doesn't support BMI2.
Package bmi contains helpers for manipulating bitmaps via BMI2 extensions properly falling back to pure go implementations if the CPU doesn't support BMI2.
debug
Package debug provides APIs for conditional runtime assertions and debug logging.
Package debug provides APIs for conditional runtime assertions and debug logging.
encryption
Package encryption contains the internal helpers for the parquet AES encryption/decryption handling.
Package encryption contains the internal helpers for the parquet AES encryption/decryption handling.
hashing
Package hashing provides utilities for and an implementation of a hash table which is more performant than the default go map implementation by leveraging xxh3 and some custom hash functions.
Package hashing provides utilities for and an implementation of a hash table which is more performant than the default go map implementation by leveraging xxh3 and some custom hash functions.
testutils
Package testutils contains utilities for generating random data and other helpers that are used for testing the various aspects of the parquet library.
Package testutils contains utilities for generating random data and other helpers that are used for testing the various aspects of the parquet library.
thrift
Package thrift is just some useful helpers for interacting with thrift to make other code easier to read/write and centralize interactions.
Package thrift is just some useful helpers for interacting with thrift to make other code easier to read/write and centralize interactions.
utils
Package utils contains various internal utilities for the parquet library that aren't intended to be exposed to external consumers such as interfaces and bitmap readers/writers including the RLE encoder/decoder and so on.
Package utils contains various internal utilities for the parquet library that aren't intended to be exposed to external consumers such as interfaces and bitmap readers/writers including the RLE encoder/decoder and so on.
Package schema provides types and functions for manipulating and building parquet file schemas.
Package schema provides types and functions for manipulating and building parquet file schemas.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL