should be it
This commit is contained in:
340
external/duckdb/extension/parquet/include/column_reader.hpp
vendored
Normal file
340
external/duckdb/extension/parquet/include/column_reader.hpp
vendored
Normal file
@@ -0,0 +1,340 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "parquet_bss_decoder.hpp"
|
||||
#include "parquet_statistics.hpp"
|
||||
#include "parquet_types.h"
|
||||
#include "resizable_buffer.hpp"
|
||||
#include "thrift_tools.hpp"
|
||||
#include "decoder/byte_stream_split_decoder.hpp"
|
||||
#include "decoder/delta_binary_packed_decoder.hpp"
|
||||
#include "decoder/dictionary_decoder.hpp"
|
||||
#include "decoder/rle_decoder.hpp"
|
||||
#include "decoder/delta_length_byte_array_decoder.hpp"
|
||||
#include "decoder/delta_byte_array_decoder.hpp"
|
||||
#include "parquet_column_schema.hpp"
|
||||
|
||||
#include "duckdb/common/operator/cast_operators.hpp"
|
||||
#include "duckdb/common/types/string_type.hpp"
|
||||
#include "duckdb/common/types/vector.hpp"
|
||||
#include "duckdb/common/types/vector_cache.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
class ParquetReader;
|
||||
struct TableFilterState;
|
||||
|
||||
using duckdb_apache::thrift::protocol::TProtocol;
|
||||
|
||||
using duckdb_parquet::ColumnChunk;
|
||||
using duckdb_parquet::CompressionCodec;
|
||||
using duckdb_parquet::FieldRepetitionType;
|
||||
using duckdb_parquet::PageHeader;
|
||||
using duckdb_parquet::SchemaElement;
|
||||
using duckdb_parquet::Type;
|
||||
|
||||
enum class ColumnEncoding {
|
||||
INVALID,
|
||||
DICTIONARY,
|
||||
DELTA_BINARY_PACKED,
|
||||
RLE,
|
||||
DELTA_LENGTH_BYTE_ARRAY,
|
||||
DELTA_BYTE_ARRAY,
|
||||
BYTE_STREAM_SPLIT,
|
||||
PLAIN
|
||||
};
|
||||
|
||||
class ColumnReader {
|
||||
friend class ByteStreamSplitDecoder;
|
||||
friend class DeltaBinaryPackedDecoder;
|
||||
friend class DeltaByteArrayDecoder;
|
||||
friend class DeltaLengthByteArrayDecoder;
|
||||
friend class DictionaryDecoder;
|
||||
friend class RLEDecoder;
|
||||
|
||||
public:
|
||||
ColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema_p);
|
||||
virtual ~ColumnReader();
|
||||
|
||||
public:
|
||||
static unique_ptr<ColumnReader> CreateReader(ParquetReader &reader, const ParquetColumnSchema &schema);
|
||||
virtual void InitializeRead(idx_t row_group_index, const vector<ColumnChunk> &columns, TProtocol &protocol_p);
|
||||
virtual idx_t Read(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result_out);
|
||||
virtual void Select(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result_out,
|
||||
const SelectionVector &sel, idx_t approved_tuple_count);
|
||||
virtual void Filter(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result_out,
|
||||
const TableFilter &filter, TableFilterState &filter_state, SelectionVector &sel,
|
||||
idx_t &approved_tuple_count, bool is_first_filter);
|
||||
static void ApplyFilter(Vector &v, const TableFilter &filter, TableFilterState &filter_state, idx_t scan_count,
|
||||
SelectionVector &sel, idx_t &approved_tuple_count);
|
||||
virtual void Skip(idx_t num_values);
|
||||
|
||||
ParquetReader &Reader();
|
||||
const LogicalType &Type() const {
|
||||
return column_schema.type;
|
||||
}
|
||||
const ParquetColumnSchema &Schema() const {
|
||||
return column_schema;
|
||||
}
|
||||
|
||||
inline idx_t ColumnIndex() const {
|
||||
return column_schema.column_index;
|
||||
}
|
||||
inline idx_t MaxDefine() const {
|
||||
return column_schema.max_define;
|
||||
}
|
||||
idx_t MaxRepeat() const {
|
||||
return column_schema.max_repeat;
|
||||
}
|
||||
|
||||
virtual idx_t FileOffset() const;
|
||||
virtual uint64_t TotalCompressedSize();
|
||||
virtual idx_t GroupRowsAvailable();
|
||||
|
||||
// register the range this reader will touch for prefetching
|
||||
virtual void RegisterPrefetch(ThriftFileTransport &transport, bool allow_merge);
|
||||
|
||||
unique_ptr<BaseStatistics> Stats(idx_t row_group_idx_p, const vector<ColumnChunk> &columns);
|
||||
|
||||
template <class VALUE_TYPE, class CONVERSION, bool HAS_DEFINES>
|
||||
void PlainTemplatedDefines(ByteBuffer &plain_data, const uint8_t *defines, uint64_t num_values, idx_t result_offset,
|
||||
Vector &result) {
|
||||
if (CONVERSION::PlainAvailable(plain_data, num_values)) {
|
||||
PlainTemplatedInternal<VALUE_TYPE, CONVERSION, HAS_DEFINES, false>(plain_data, defines, num_values,
|
||||
result_offset, result);
|
||||
} else {
|
||||
PlainTemplatedInternal<VALUE_TYPE, CONVERSION, HAS_DEFINES, true>(plain_data, defines, num_values,
|
||||
result_offset, result);
|
||||
}
|
||||
}
|
||||
template <class VALUE_TYPE, class CONVERSION>
|
||||
void PlainTemplated(ByteBuffer &plain_data, const uint8_t *defines, uint64_t num_values, idx_t result_offset,
|
||||
Vector &result) {
|
||||
if (HasDefines() && defines) {
|
||||
PlainTemplatedDefines<VALUE_TYPE, CONVERSION, true>(plain_data, defines, num_values, result_offset, result);
|
||||
} else {
|
||||
PlainTemplatedDefines<VALUE_TYPE, CONVERSION, false>(plain_data, defines, num_values, result_offset,
|
||||
result);
|
||||
}
|
||||
}
|
||||
|
||||
template <class CONVERSION, bool HAS_DEFINES>
|
||||
void PlainSkipTemplatedDefines(ByteBuffer &plain_data, const uint8_t *defines, uint64_t num_values) {
|
||||
if (CONVERSION::PlainAvailable(plain_data, num_values)) {
|
||||
PlainSkipTemplatedInternal<CONVERSION, HAS_DEFINES, false>(plain_data, defines, num_values);
|
||||
} else {
|
||||
PlainSkipTemplatedInternal<CONVERSION, HAS_DEFINES, true>(plain_data, defines, num_values);
|
||||
}
|
||||
}
|
||||
template <class CONVERSION>
|
||||
void PlainSkipTemplated(ByteBuffer &plain_data, const uint8_t *defines, uint64_t num_values) {
|
||||
if (HasDefines() && defines) {
|
||||
PlainSkipTemplatedDefines<CONVERSION, true>(plain_data, defines, num_values);
|
||||
} else {
|
||||
PlainSkipTemplatedDefines<CONVERSION, false>(plain_data, defines, num_values);
|
||||
}
|
||||
}
|
||||
|
||||
template <class VALUE_TYPE, class CONVERSION>
|
||||
void PlainSelectTemplated(ByteBuffer &plain_data, const uint8_t *defines, uint64_t num_values, Vector &result,
|
||||
const SelectionVector &sel, idx_t approved_tuple_count) {
|
||||
if (HasDefines() && defines) {
|
||||
PlainSelectTemplatedInternal<VALUE_TYPE, CONVERSION, true, true>(plain_data, defines, num_values, result,
|
||||
sel, approved_tuple_count);
|
||||
} else {
|
||||
PlainSelectTemplatedInternal<VALUE_TYPE, CONVERSION, false, true>(plain_data, defines, num_values, result,
|
||||
sel, approved_tuple_count);
|
||||
}
|
||||
}
|
||||
|
||||
idx_t GetValidCount(uint8_t *defines, idx_t count, idx_t offset = 0) const {
|
||||
if (!defines) {
|
||||
return count;
|
||||
}
|
||||
idx_t valid_count = 0;
|
||||
for (idx_t i = offset; i < offset + count; i++) {
|
||||
valid_count += defines[i] == MaxDefine();
|
||||
}
|
||||
return valid_count;
|
||||
}
|
||||
|
||||
protected:
|
||||
virtual bool SupportsDirectFilter() const {
|
||||
return false;
|
||||
}
|
||||
virtual bool SupportsDirectSelect() const {
|
||||
return false;
|
||||
}
|
||||
void DirectFilter(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result_out,
|
||||
const TableFilter &filter, TableFilterState &filter_state, SelectionVector &sel,
|
||||
idx_t &approved_tuple_count);
|
||||
void DirectSelect(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result,
|
||||
const SelectionVector &sel, idx_t approved_tuple_count);
|
||||
|
||||
private:
|
||||
//! Check if a previous table filter has filtered out this page
|
||||
bool PageIsFilteredOut(PageHeader &page_hdr);
|
||||
void BeginRead(data_ptr_t define_out, data_ptr_t repeat_out);
|
||||
void FinishRead(idx_t read_count);
|
||||
idx_t ReadPageHeaders(idx_t max_read, optional_ptr<const TableFilter> filter = nullptr,
|
||||
optional_ptr<TableFilterState> filter_state = nullptr);
|
||||
idx_t ReadInternal(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result);
|
||||
//! Prepare a read of up to "max_read" rows and read the defines/repeats.
|
||||
//! Returns whether all values are valid (i.e., not NULL)
|
||||
bool PrepareRead(idx_t read_count, data_ptr_t define_out, data_ptr_t repeat_out, idx_t result_offset);
|
||||
void ReadData(idx_t read_now, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result, idx_t result_offset);
|
||||
|
||||
template <class VALUE_TYPE, class CONVERSION, bool HAS_DEFINES, bool CHECKED>
|
||||
void PlainTemplatedInternal(ByteBuffer &plain_data, const uint8_t *__restrict defines, const uint64_t num_values,
|
||||
const idx_t result_offset, Vector &result) {
|
||||
const auto result_ptr = FlatVector::GetData<VALUE_TYPE>(result);
|
||||
if (!HAS_DEFINES && !CHECKED && CONVERSION::PlainConstantSize() == sizeof(VALUE_TYPE)) {
|
||||
// we can memcpy
|
||||
idx_t copy_count = num_values * CONVERSION::PlainConstantSize();
|
||||
memcpy(result_ptr + result_offset, plain_data.ptr, copy_count);
|
||||
plain_data.unsafe_inc(copy_count);
|
||||
return;
|
||||
}
|
||||
auto &result_mask = FlatVector::Validity(result);
|
||||
for (idx_t row_idx = result_offset; row_idx < result_offset + num_values; row_idx++) {
|
||||
if (HAS_DEFINES && defines[row_idx] != MaxDefine()) {
|
||||
result_mask.SetInvalid(row_idx);
|
||||
continue;
|
||||
}
|
||||
result_ptr[row_idx] = CONVERSION::template PlainRead<CHECKED>(plain_data, *this);
|
||||
}
|
||||
}
|
||||
|
||||
template <class CONVERSION, bool HAS_DEFINES, bool CHECKED>
|
||||
void PlainSkipTemplatedInternal(ByteBuffer &plain_data, const uint8_t *__restrict defines,
|
||||
const uint64_t num_values, idx_t row_offset = 0) {
|
||||
if (!HAS_DEFINES && CONVERSION::PlainConstantSize() > 0) {
|
||||
if (CHECKED) {
|
||||
plain_data.inc(num_values * CONVERSION::PlainConstantSize());
|
||||
} else {
|
||||
plain_data.unsafe_inc(num_values * CONVERSION::PlainConstantSize());
|
||||
}
|
||||
return;
|
||||
}
|
||||
for (idx_t row_idx = row_offset; row_idx < row_offset + num_values; row_idx++) {
|
||||
if (HAS_DEFINES && defines[row_idx] != MaxDefine()) {
|
||||
continue;
|
||||
}
|
||||
CONVERSION::template PlainSkip<CHECKED>(plain_data, *this);
|
||||
}
|
||||
}
|
||||
|
||||
template <class VALUE_TYPE, class CONVERSION, bool HAS_DEFINES, bool CHECKED>
|
||||
void PlainSelectTemplatedInternal(ByteBuffer &plain_data, const uint8_t *__restrict defines,
|
||||
const uint64_t num_values, Vector &result, const SelectionVector &sel,
|
||||
idx_t approved_tuple_count) {
|
||||
const auto result_ptr = FlatVector::GetData<VALUE_TYPE>(result);
|
||||
auto &result_mask = FlatVector::Validity(result);
|
||||
idx_t current_entry = 0;
|
||||
for (idx_t i = 0; i < approved_tuple_count; i++) {
|
||||
auto next_entry = sel.get_index(i);
|
||||
D_ASSERT(current_entry <= next_entry);
|
||||
// perform any skips forward if required
|
||||
PlainSkipTemplatedInternal<CONVERSION, HAS_DEFINES, CHECKED>(plain_data, defines,
|
||||
next_entry - current_entry, current_entry);
|
||||
// read this row
|
||||
if (HAS_DEFINES && defines[next_entry] != MaxDefine()) {
|
||||
result_mask.SetInvalid(next_entry);
|
||||
} else {
|
||||
result_ptr[next_entry] = CONVERSION::template PlainRead<CHECKED>(plain_data, *this);
|
||||
}
|
||||
current_entry = next_entry + 1;
|
||||
}
|
||||
if (current_entry < num_values) {
|
||||
// skip forward to the end of where we are selecting
|
||||
PlainSkipTemplatedInternal<CONVERSION, HAS_DEFINES, CHECKED>(plain_data, defines,
|
||||
num_values - current_entry, current_entry);
|
||||
}
|
||||
}
|
||||
|
||||
protected:
|
||||
Allocator &GetAllocator();
|
||||
// readers that use the default Read() need to implement those
|
||||
virtual void PlainSkip(ByteBuffer &plain_data, uint8_t *defines, idx_t num_values);
|
||||
virtual void Plain(ByteBuffer &plain_data, uint8_t *defines, idx_t num_values, idx_t result_offset, Vector &result);
|
||||
virtual void Plain(shared_ptr<ResizeableBuffer> &plain_data, uint8_t *defines, idx_t num_values,
|
||||
idx_t result_offset, Vector &result);
|
||||
virtual void PlainSelect(shared_ptr<ResizeableBuffer> &plain_data, uint8_t *defines, idx_t num_values,
|
||||
Vector &result, const SelectionVector &sel, idx_t count);
|
||||
|
||||
// applies any skips that were registered using Skip()
|
||||
virtual void ApplyPendingSkips(data_ptr_t define_out, data_ptr_t repeat_out);
|
||||
|
||||
inline bool HasDefines() const {
|
||||
return MaxDefine() > 0;
|
||||
}
|
||||
|
||||
inline bool HasRepeats() const {
|
||||
return MaxRepeat() > 0;
|
||||
}
|
||||
|
||||
protected:
|
||||
const ParquetColumnSchema &column_schema;
|
||||
|
||||
ParquetReader &reader;
|
||||
idx_t pending_skips = 0;
|
||||
bool page_is_filtered_out = false;
|
||||
|
||||
virtual void ResetPage();
|
||||
|
||||
private:
|
||||
void AllocateBlock(idx_t size);
|
||||
void PrepareRead(optional_ptr<const TableFilter> filter, optional_ptr<TableFilterState> filter_state);
|
||||
void PreparePage(PageHeader &page_hdr);
|
||||
void PrepareDataPage(PageHeader &page_hdr);
|
||||
void PreparePageV2(PageHeader &page_hdr);
|
||||
void DecompressInternal(CompressionCodec::type codec, const_data_ptr_t src, idx_t src_size, data_ptr_t dst,
|
||||
idx_t dst_size);
|
||||
const ColumnChunk *chunk = nullptr;
|
||||
|
||||
TProtocol *protocol;
|
||||
idx_t page_rows_available;
|
||||
idx_t group_rows_available;
|
||||
idx_t chunk_read_offset;
|
||||
|
||||
shared_ptr<ResizeableBuffer> block;
|
||||
|
||||
ColumnEncoding encoding = ColumnEncoding::INVALID;
|
||||
unique_ptr<RleBpDecoder> defined_decoder;
|
||||
unique_ptr<RleBpDecoder> repeated_decoder;
|
||||
DictionaryDecoder dictionary_decoder;
|
||||
DeltaBinaryPackedDecoder delta_binary_packed_decoder;
|
||||
RLEDecoder rle_decoder;
|
||||
DeltaLengthByteArrayDecoder delta_length_byte_array_decoder;
|
||||
DeltaByteArrayDecoder delta_byte_array_decoder;
|
||||
ByteStreamSplitDecoder byte_stream_split_decoder;
|
||||
|
||||
//! Resizeable buffers used for the various encodings above
|
||||
ResizeableBuffer encoding_buffers[2];
|
||||
|
||||
public:
|
||||
template <class TARGET>
|
||||
TARGET &Cast() {
|
||||
if (TARGET::TYPE != PhysicalType::INVALID && Type().InternalType() != TARGET::TYPE) {
|
||||
throw InternalException("Failed to cast column reader to type - type mismatch");
|
||||
}
|
||||
return reinterpret_cast<TARGET &>(*this);
|
||||
}
|
||||
|
||||
template <class TARGET>
|
||||
const TARGET &Cast() const {
|
||||
if (TARGET::TYPE != PhysicalType::INVALID && Type().InternalType() != TARGET::TYPE) {
|
||||
throw InternalException("Failed to cast column reader to type - type mismatch");
|
||||
}
|
||||
return reinterpret_cast<const TARGET &>(*this);
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
145
external/duckdb/extension/parquet/include/column_writer.hpp
vendored
Normal file
145
external/duckdb/extension/parquet/include/column_writer.hpp
vendored
Normal file
@@ -0,0 +1,145 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "parquet_types.h"
|
||||
#include "parquet_column_schema.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
class MemoryStream;
|
||||
class ParquetWriter;
|
||||
class ColumnWriterPageState;
|
||||
class PrimitiveColumnWriterState;
|
||||
struct ChildFieldIDs;
|
||||
struct ShreddingType;
|
||||
class ResizeableBuffer;
|
||||
class ParquetBloomFilter;
|
||||
|
||||
class ColumnWriterState {
|
||||
public:
|
||||
virtual ~ColumnWriterState();
|
||||
|
||||
unsafe_vector<uint16_t> definition_levels;
|
||||
unsafe_vector<uint16_t> repetition_levels;
|
||||
unsafe_vector<uint8_t> is_empty;
|
||||
idx_t parent_null_count = 0;
|
||||
idx_t null_count = 0;
|
||||
|
||||
public:
|
||||
template <class TARGET>
|
||||
TARGET &Cast() {
|
||||
DynamicCastCheck<TARGET>(this);
|
||||
return reinterpret_cast<TARGET &>(*this);
|
||||
}
|
||||
template <class TARGET>
|
||||
const TARGET &Cast() const {
|
||||
D_ASSERT(dynamic_cast<const TARGET *>(this));
|
||||
return reinterpret_cast<const TARGET &>(*this);
|
||||
}
|
||||
};
|
||||
|
||||
class ColumnWriterPageState {
|
||||
public:
|
||||
virtual ~ColumnWriterPageState() {
|
||||
}
|
||||
|
||||
public:
|
||||
template <class TARGET>
|
||||
TARGET &Cast() {
|
||||
DynamicCastCheck<TARGET>(this);
|
||||
return reinterpret_cast<TARGET &>(*this);
|
||||
}
|
||||
template <class TARGET>
|
||||
const TARGET &Cast() const {
|
||||
D_ASSERT(dynamic_cast<const TARGET *>(this));
|
||||
return reinterpret_cast<const TARGET &>(*this);
|
||||
}
|
||||
};
|
||||
|
||||
class ColumnWriter {
|
||||
protected:
|
||||
static constexpr uint16_t PARQUET_DEFINE_VALID = UINT16_C(65535);
|
||||
|
||||
public:
|
||||
ColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema, vector<string> schema_path,
|
||||
bool can_have_nulls);
|
||||
virtual ~ColumnWriter();
|
||||
|
||||
public:
|
||||
const LogicalType &Type() const {
|
||||
return column_schema.type;
|
||||
}
|
||||
const ParquetColumnSchema &Schema() const {
|
||||
return column_schema;
|
||||
}
|
||||
inline idx_t SchemaIndex() const {
|
||||
return column_schema.schema_index;
|
||||
}
|
||||
inline idx_t MaxDefine() const {
|
||||
return column_schema.max_define;
|
||||
}
|
||||
idx_t MaxRepeat() const {
|
||||
return column_schema.max_repeat;
|
||||
}
|
||||
|
||||
static ParquetColumnSchema FillParquetSchema(vector<duckdb_parquet::SchemaElement> &schemas,
|
||||
const LogicalType &type, const string &name, bool allow_geometry,
|
||||
optional_ptr<const ChildFieldIDs> field_ids,
|
||||
optional_ptr<const ShreddingType> shredding_types,
|
||||
idx_t max_repeat = 0, idx_t max_define = 1,
|
||||
bool can_have_nulls = true);
|
||||
//! Create the column writer for a specific type recursively
|
||||
static unique_ptr<ColumnWriter> CreateWriterRecursive(ClientContext &context, ParquetWriter &writer,
|
||||
const vector<duckdb_parquet::SchemaElement> &parquet_schemas,
|
||||
const ParquetColumnSchema &schema,
|
||||
vector<string> path_in_schema);
|
||||
|
||||
virtual unique_ptr<ColumnWriterState> InitializeWriteState(duckdb_parquet::RowGroup &row_group) = 0;
|
||||
|
||||
//! indicates whether the write need to analyse the data before preparing it
|
||||
virtual bool HasAnalyze() {
|
||||
return false;
|
||||
}
|
||||
|
||||
virtual void Analyze(ColumnWriterState &state, ColumnWriterState *parent, Vector &vector, idx_t count) {
|
||||
throw NotImplementedException("Writer does not need analysis");
|
||||
}
|
||||
|
||||
//! Called after all data has been passed to Analyze
|
||||
virtual void FinalizeAnalyze(ColumnWriterState &state) {
|
||||
throw NotImplementedException("Writer does not need analysis");
|
||||
}
|
||||
|
||||
virtual void Prepare(ColumnWriterState &state, ColumnWriterState *parent, Vector &vector, idx_t count,
|
||||
bool vector_can_span_multiple_pages) = 0;
|
||||
|
||||
virtual void BeginWrite(ColumnWriterState &state) = 0;
|
||||
virtual void Write(ColumnWriterState &state, Vector &vector, idx_t count) = 0;
|
||||
virtual void FinalizeWrite(ColumnWriterState &state) = 0;
|
||||
|
||||
protected:
|
||||
void HandleDefineLevels(ColumnWriterState &state, ColumnWriterState *parent, const ValidityMask &validity,
|
||||
const idx_t count, const uint16_t define_value, const uint16_t null_value) const;
|
||||
void HandleRepeatLevels(ColumnWriterState &state_p, ColumnWriterState *parent, idx_t count) const;
|
||||
|
||||
void CompressPage(MemoryStream &temp_writer, size_t &compressed_size, data_ptr_t &compressed_data,
|
||||
AllocatedData &compressed_buf);
|
||||
|
||||
public:
|
||||
ParquetWriter &writer;
|
||||
const ParquetColumnSchema &column_schema;
|
||||
vector<string> schema_path;
|
||||
bool can_have_nulls;
|
||||
|
||||
protected:
|
||||
vector<unique_ptr<ColumnWriter>> child_writers;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
221
external/duckdb/extension/parquet/include/decode_utils.hpp
vendored
Normal file
221
external/duckdb/extension/parquet/include/decode_utils.hpp
vendored
Normal file
@@ -0,0 +1,221 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// decode_utils.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb/common/fast_mem.hpp"
|
||||
#include "duckdb/common/bitpacking.hpp"
|
||||
#include "resizable_buffer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ParquetDecodeUtils {
|
||||
//===--------------------------------------------------------------------===//
|
||||
// Bitpacking
|
||||
//===--------------------------------------------------------------------===//
|
||||
private:
|
||||
static const uint64_t BITPACK_MASKS[];
|
||||
static const uint64_t BITPACK_MASKS_SIZE;
|
||||
static const uint8_t BITPACK_DLEN;
|
||||
|
||||
static void CheckWidth(const uint8_t width) {
|
||||
if (width >= BITPACK_MASKS_SIZE) {
|
||||
throw InvalidInputException("The width (%d) of the bitpacked data exceeds the supported max width (%d), "
|
||||
"the file might be corrupted.",
|
||||
width, BITPACK_MASKS_SIZE);
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
template <class T>
|
||||
static void BitUnpack(ByteBuffer &src, bitpacking_width_t &bitpack_pos, T *dst, idx_t count,
|
||||
const bitpacking_width_t width) {
|
||||
CheckWidth(width);
|
||||
const auto mask = BITPACK_MASKS[width];
|
||||
src.available(count * width / BITPACK_DLEN); // check if buffer has enough space available once
|
||||
if (bitpack_pos == 0 && count >= BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE) {
|
||||
idx_t remainder = count % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE;
|
||||
idx_t aligned_count = count - remainder;
|
||||
BitUnpackAlignedInternal(src, dst, aligned_count, width);
|
||||
dst += aligned_count;
|
||||
count = remainder;
|
||||
}
|
||||
for (idx_t i = 0; i < count; i++) {
|
||||
auto val = (src.unsafe_get<uint8_t>() >> bitpack_pos) & mask;
|
||||
bitpack_pos += width;
|
||||
while (bitpack_pos > BITPACK_DLEN) {
|
||||
src.unsafe_inc(1);
|
||||
val |= (static_cast<T>(src.unsafe_get<uint8_t>())
|
||||
<< static_cast<T>(BITPACK_DLEN - (bitpack_pos - width))) &
|
||||
mask;
|
||||
bitpack_pos -= BITPACK_DLEN;
|
||||
}
|
||||
dst[i] = val;
|
||||
}
|
||||
}
|
||||
|
||||
static void Skip(ByteBuffer &src, bitpacking_width_t &bitpack_pos, idx_t count, const bitpacking_width_t width) {
|
||||
CheckWidth(width);
|
||||
src.available(count * width / BITPACK_DLEN); // check if buffer has enough space available once
|
||||
if (bitpack_pos == 0 && count >= BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE) {
|
||||
idx_t remainder = count % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE;
|
||||
idx_t aligned_count = count - remainder;
|
||||
SkipAligned(src, aligned_count, width);
|
||||
count = remainder;
|
||||
}
|
||||
// FIXME: we should be able to just do this in one go instead of having this loop
|
||||
for (idx_t i = 0; i < count; i++) {
|
||||
bitpack_pos += width;
|
||||
while (bitpack_pos > BITPACK_DLEN) {
|
||||
src.unsafe_inc(1);
|
||||
bitpack_pos -= BITPACK_DLEN;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <class T>
|
||||
static void BitPackAligned(T *src, data_ptr_t dst, const idx_t count, const bitpacking_width_t width) {
|
||||
D_ASSERT(width < BITPACK_MASKS_SIZE);
|
||||
D_ASSERT(count % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE == 0);
|
||||
BitpackingPrimitives::PackBuffer<T, true>(dst, src, count, width);
|
||||
}
|
||||
|
||||
template <class T>
|
||||
static void BitUnpackAlignedInternal(ByteBuffer &src, T *dst, const idx_t count, const bitpacking_width_t width) {
|
||||
D_ASSERT(count % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE == 0);
|
||||
if (cast_pointer_to_uint64(src.ptr) % sizeof(T) == 0) {
|
||||
// Fast path: aligned
|
||||
BitpackingPrimitives::UnPackBuffer<T>(data_ptr_cast(dst), src.ptr, count, width);
|
||||
src.unsafe_inc(count * width / BITPACK_DLEN);
|
||||
return;
|
||||
}
|
||||
|
||||
for (idx_t i = 0; i < count; i += BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE) {
|
||||
const auto next_read = BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE * width / BITPACK_DLEN;
|
||||
|
||||
// Buffer for alignment
|
||||
T aligned_data[BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE];
|
||||
|
||||
// Copy over to aligned buffer
|
||||
FastMemcpy(aligned_data, src.ptr, next_read);
|
||||
|
||||
// Unpack
|
||||
BitpackingPrimitives::UnPackBlock<T>(data_ptr_cast(dst), data_ptr_cast(aligned_data), width, true);
|
||||
|
||||
src.unsafe_inc(next_read);
|
||||
dst += BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE;
|
||||
}
|
||||
}
|
||||
|
||||
template <class T>
|
||||
static void BitUnpackAligned(ByteBuffer &src, T *dst, const idx_t count, const bitpacking_width_t width) {
|
||||
CheckWidth(width);
|
||||
if (count % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE != 0) {
|
||||
throw InvalidInputException("Aligned bitpacking count must be a multiple of %llu",
|
||||
BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE);
|
||||
}
|
||||
const auto read_size = count * width / BITPACK_DLEN;
|
||||
src.available(read_size); // check if buffer has enough space available once
|
||||
BitUnpackAlignedInternal(src, dst, count, width);
|
||||
}
|
||||
|
||||
static void SkipAligned(ByteBuffer &src, const idx_t count, const bitpacking_width_t width) {
|
||||
CheckWidth(width);
|
||||
if (count % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE != 0) {
|
||||
throw InvalidInputException("Aligned bitpacking count must be a multiple of %llu",
|
||||
BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE);
|
||||
}
|
||||
const auto read_size = count * width / BITPACK_DLEN;
|
||||
src.inc(read_size);
|
||||
}
|
||||
|
||||
//===--------------------------------------------------------------------===//
|
||||
// Zigzag
|
||||
//===--------------------------------------------------------------------===//
|
||||
private:
|
||||
//! https://lemire.me/blog/2022/11/25/making-all-your-integers-positive-with-zigzag-encoding/
|
||||
template <class UNSIGNED>
|
||||
static typename std::enable_if<std::is_unsigned<UNSIGNED>::value, typename std::make_signed<UNSIGNED>::type>::type
|
||||
ZigzagToIntInternal(UNSIGNED x) {
|
||||
return (x >> 1) ^ (-(x & 1));
|
||||
}
|
||||
|
||||
template <typename SIGNED>
|
||||
static typename std::enable_if<std::is_signed<SIGNED>::value, typename std::make_unsigned<SIGNED>::type>::type
|
||||
IntToZigzagInternal(SIGNED x) {
|
||||
using UNSIGNED = typename std::make_unsigned<SIGNED>::type;
|
||||
return (static_cast<UNSIGNED>(x) << 1) ^ static_cast<UNSIGNED>(x >> (sizeof(SIGNED) * 8 - 1));
|
||||
}
|
||||
|
||||
public:
|
||||
template <class UNSIGNED>
|
||||
static typename std::enable_if<std::is_unsigned<UNSIGNED>::value, typename std::make_signed<UNSIGNED>::type>::type
|
||||
ZigzagToInt(UNSIGNED x) {
|
||||
auto integer = ZigzagToIntInternal(x);
|
||||
D_ASSERT(x == IntToZigzagInternal(integer)); // test roundtrip
|
||||
return integer;
|
||||
}
|
||||
|
||||
template <typename SIGNED>
|
||||
static typename std::enable_if<std::is_signed<SIGNED>::value, typename std::make_unsigned<SIGNED>::type>::type
|
||||
IntToZigzag(SIGNED x) {
|
||||
auto zigzag = IntToZigzagInternal(x);
|
||||
D_ASSERT(x == ZigzagToIntInternal(zigzag)); // test roundtrip
|
||||
return zigzag;
|
||||
}
|
||||
|
||||
//===--------------------------------------------------------------------===//
|
||||
// Varint
|
||||
//===--------------------------------------------------------------------===//
|
||||
public:
|
||||
template <class T>
|
||||
static uint8_t GetVarintSize(T val) {
|
||||
uint8_t res = 0;
|
||||
do {
|
||||
val >>= 7;
|
||||
res++;
|
||||
} while (val != 0);
|
||||
return res;
|
||||
}
|
||||
|
||||
template <class T>
|
||||
static void VarintEncode(T val, WriteStream &ser) {
|
||||
do {
|
||||
uint8_t byte = val & 127;
|
||||
val >>= 7;
|
||||
if (val != 0) {
|
||||
byte |= 128;
|
||||
}
|
||||
ser.Write<uint8_t>(byte);
|
||||
} while (val != 0);
|
||||
}
|
||||
|
||||
template <class T, bool CHECKED = true>
|
||||
static T VarintDecode(ByteBuffer &buf) {
|
||||
T result = 0;
|
||||
uint8_t shift = 0;
|
||||
while (true) {
|
||||
uint8_t byte;
|
||||
if (CHECKED) {
|
||||
byte = buf.read<uint8_t>();
|
||||
} else {
|
||||
byte = buf.unsafe_read<uint8_t>();
|
||||
}
|
||||
result |= T(byte & 127) << shift;
|
||||
if ((byte & 128) == 0) {
|
||||
break;
|
||||
}
|
||||
shift += 7;
|
||||
if (shift > sizeof(T) * 8) {
|
||||
throw std::runtime_error("Varint-decoding found too large number");
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
};
|
||||
} // namespace duckdb
|
||||
32
external/duckdb/extension/parquet/include/decoder/byte_stream_split_decoder.hpp
vendored
Normal file
32
external/duckdb/extension/parquet/include/decoder/byte_stream_split_decoder.hpp
vendored
Normal file
@@ -0,0 +1,32 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// decoder/byte_stream_split_decoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "parquet_bss_decoder.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
class ColumnReader;
|
||||
|
||||
class ByteStreamSplitDecoder {
|
||||
public:
|
||||
explicit ByteStreamSplitDecoder(ColumnReader &reader);
|
||||
|
||||
public:
|
||||
void InitializePage();
|
||||
void Read(uint8_t *defines, idx_t read_count, Vector &result, idx_t result_offset);
|
||||
void Skip(uint8_t *defines, idx_t skip_count);
|
||||
|
||||
private:
|
||||
ColumnReader &reader;
|
||||
ResizeableBuffer &decoded_data_buffer;
|
||||
unique_ptr<BssDecoder> bss_decoder;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
33
external/duckdb/extension/parquet/include/decoder/delta_binary_packed_decoder.hpp
vendored
Normal file
33
external/duckdb/extension/parquet/include/decoder/delta_binary_packed_decoder.hpp
vendored
Normal file
@@ -0,0 +1,33 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// decoder/delta_binary_packed_decoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "parquet_dbp_decoder.hpp"
|
||||
#include "resizable_buffer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
class ColumnReader;
|
||||
|
||||
class DeltaBinaryPackedDecoder {
|
||||
public:
|
||||
explicit DeltaBinaryPackedDecoder(ColumnReader &reader);
|
||||
|
||||
public:
|
||||
void InitializePage();
|
||||
void Read(uint8_t *defines, idx_t read_count, Vector &result, idx_t result_offset);
|
||||
void Skip(uint8_t *defines, idx_t skip_count);
|
||||
|
||||
private:
|
||||
ColumnReader &reader;
|
||||
ResizeableBuffer &decoded_data_buffer;
|
||||
unique_ptr<DbpDecoder> dbp_decoder;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
38
external/duckdb/extension/parquet/include/decoder/delta_byte_array_decoder.hpp
vendored
Normal file
38
external/duckdb/extension/parquet/include/decoder/delta_byte_array_decoder.hpp
vendored
Normal file
@@ -0,0 +1,38 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// decoder/delta_byte_array_decoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "parquet_dbp_decoder.hpp"
|
||||
#include "resizable_buffer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
class ColumnReader;
|
||||
|
||||
class DeltaByteArrayDecoder {
|
||||
public:
|
||||
explicit DeltaByteArrayDecoder(ColumnReader &reader);
|
||||
|
||||
public:
|
||||
void InitializePage();
|
||||
|
||||
void Read(uint8_t *defines, idx_t read_count, Vector &result, idx_t result_offset);
|
||||
void Skip(uint8_t *defines, idx_t skip_count);
|
||||
|
||||
static void ReadDbpData(Allocator &allocator, ResizeableBuffer &buffer, ResizeableBuffer &result_buffer,
|
||||
idx_t &value_count);
|
||||
|
||||
private:
|
||||
ColumnReader &reader;
|
||||
unique_ptr<Vector> byte_array_data;
|
||||
idx_t byte_array_count = 0;
|
||||
idx_t delta_offset = 0;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
43
external/duckdb/extension/parquet/include/decoder/delta_length_byte_array_decoder.hpp
vendored
Normal file
43
external/duckdb/extension/parquet/include/decoder/delta_length_byte_array_decoder.hpp
vendored
Normal file
@@ -0,0 +1,43 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// decoder/delta_length_byte_array_decoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "parquet_dbp_decoder.hpp"
|
||||
#include "resizable_buffer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
class ColumnReader;
|
||||
|
||||
class DeltaLengthByteArrayDecoder {
|
||||
public:
|
||||
explicit DeltaLengthByteArrayDecoder(ColumnReader &reader);
|
||||
|
||||
public:
|
||||
void InitializePage();
|
||||
|
||||
void Read(shared_ptr<ResizeableBuffer> &block, uint8_t *defines, idx_t read_count, Vector &result,
|
||||
idx_t result_offset);
|
||||
void Skip(uint8_t *defines, idx_t skip_count);
|
||||
|
||||
private:
|
||||
template <bool HAS_DEFINES>
|
||||
void ReadInternal(shared_ptr<ResizeableBuffer> &block, uint8_t *defines, idx_t read_count, Vector &result,
|
||||
idx_t result_offset);
|
||||
template <bool HAS_DEFINES>
|
||||
void SkipInternal(uint8_t *defines, idx_t skip_count);
|
||||
|
||||
private:
|
||||
ColumnReader &reader;
|
||||
ResizeableBuffer &length_buffer;
|
||||
idx_t byte_array_count = 0;
|
||||
idx_t length_idx;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
56
external/duckdb/extension/parquet/include/decoder/dictionary_decoder.hpp
vendored
Normal file
56
external/duckdb/extension/parquet/include/decoder/dictionary_decoder.hpp
vendored
Normal file
@@ -0,0 +1,56 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// decoder/dictionary_decoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "parquet_rle_bp_decoder.hpp"
|
||||
#include "resizable_buffer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
class ColumnReader;
|
||||
struct TableFilterState;
|
||||
|
||||
class DictionaryDecoder {
|
||||
public:
|
||||
explicit DictionaryDecoder(ColumnReader &reader);
|
||||
|
||||
public:
|
||||
void InitializeDictionary(idx_t dictionary_size, optional_ptr<const TableFilter> filter,
|
||||
optional_ptr<TableFilterState> filter_state, bool has_defines);
|
||||
void InitializePage();
|
||||
idx_t Read(uint8_t *defines, idx_t read_count, Vector &result, idx_t result_offset);
|
||||
void Skip(uint8_t *defines, idx_t skip_count);
|
||||
bool CanFilter(const TableFilter &filter, TableFilterState &filter_state);
|
||||
bool DictionarySupportsFilter(const TableFilter &filter, TableFilterState &filter_state);
|
||||
void Filter(uint8_t *defines, idx_t read_count, Vector &result, SelectionVector &sel, idx_t &approved_tuple_count);
|
||||
bool HasFilter() const {
|
||||
return filter_result.get();
|
||||
}
|
||||
bool HasFilteredOutAllValues() const {
|
||||
return HasFilter() && filter_count == 0;
|
||||
}
|
||||
|
||||
private:
|
||||
idx_t GetValidValues(uint8_t *defines, idx_t read_count, idx_t result_offset);
|
||||
void ConvertDictToSelVec(uint32_t *offsets, const SelectionVector &rows, idx_t count);
|
||||
|
||||
private:
|
||||
ColumnReader &reader;
|
||||
ResizeableBuffer &offset_buffer;
|
||||
unique_ptr<RleBpDecoder> dict_decoder;
|
||||
SelectionVector valid_sel;
|
||||
SelectionVector dictionary_selection_vector;
|
||||
idx_t dictionary_size;
|
||||
buffer_ptr<VectorChildBuffer> dictionary;
|
||||
unsafe_unique_array<bool> filter_result;
|
||||
idx_t filter_count;
|
||||
bool can_have_nulls;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
32
external/duckdb/extension/parquet/include/decoder/rle_decoder.hpp
vendored
Normal file
32
external/duckdb/extension/parquet/include/decoder/rle_decoder.hpp
vendored
Normal file
@@ -0,0 +1,32 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// decoder/rle_decoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "parquet_rle_bp_decoder.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
class ColumnReader;
|
||||
|
||||
class RLEDecoder {
|
||||
public:
|
||||
explicit RLEDecoder(ColumnReader &reader);
|
||||
|
||||
public:
|
||||
void InitializePage();
|
||||
void Read(uint8_t *defines, idx_t read_count, Vector &result, idx_t result_offset);
|
||||
void Skip(uint8_t *defines, idx_t skip_count);
|
||||
|
||||
private:
|
||||
ColumnReader &reader;
|
||||
ResizeableBuffer &decoded_data_buffer;
|
||||
unique_ptr<RleBpDecoder> rle_decoder;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
102
external/duckdb/extension/parquet/include/geo_parquet.hpp
vendored
Normal file
102
external/duckdb/extension/parquet/include/geo_parquet.hpp
vendored
Normal file
@@ -0,0 +1,102 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// geo_parquet.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_writer.hpp"
|
||||
#include "duckdb/common/string.hpp"
|
||||
#include "duckdb/common/types/data_chunk.hpp"
|
||||
#include "duckdb/common/unordered_map.hpp"
|
||||
#include "duckdb/common/unordered_set.hpp"
|
||||
#include "parquet_types.h"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
struct ParquetColumnSchema;
|
||||
class ParquetReader;
|
||||
class ColumnReader;
|
||||
class ClientContext;
|
||||
class ExpressionExecutor;
|
||||
|
||||
enum class GeoParquetColumnEncoding : uint8_t {
|
||||
WKB = 1,
|
||||
POINT,
|
||||
LINESTRING,
|
||||
POLYGON,
|
||||
MULTIPOINT,
|
||||
MULTILINESTRING,
|
||||
MULTIPOLYGON,
|
||||
};
|
||||
|
||||
enum class GeoParquetVersion : uint8_t {
|
||||
// Write GeoParquet 1.0 metadata
|
||||
// GeoParquet 1.0 has the widest support among readers and writers
|
||||
V1,
|
||||
|
||||
// Write GeoParquet 2.0
|
||||
// The GeoParquet 2.0 options is identical to GeoParquet 1.0 except the underlying storage
|
||||
// of spatial columns is Parquet native geometry, where the Parquet writer will include
|
||||
// native statistics according to the underlying Parquet options. Compared to 'BOTH', this will
|
||||
// actually write the metadata as containing GeoParquet version 2.0.0
|
||||
// However, V2 isnt standardized yet, so this option is still a bit experimental
|
||||
V2,
|
||||
|
||||
// Write GeoParquet 1.0 metadata, with native Parquet geometry types
|
||||
// This is a bit of a hold-over option for compatibility with systems that
|
||||
// reject GeoParquet 2.0 metadata, but can read Parquet native geometry types as they simply ignore the extra
|
||||
// logical type. DuckDB v1.4.0 falls into this category.
|
||||
BOTH,
|
||||
|
||||
// Do not write GeoParquet metadata
|
||||
// This option suppresses GeoParquet metadata; however, spatial types will be written as
|
||||
// Parquet native Geometry/Geography.
|
||||
NONE,
|
||||
};
|
||||
|
||||
struct GeoParquetColumnMetadata {
|
||||
// The encoding of the geometry column
|
||||
GeoParquetColumnEncoding geometry_encoding;
|
||||
|
||||
// The statistics of the geometry column
|
||||
GeometryStatsData stats;
|
||||
|
||||
// The crs of the geometry column (if any) in PROJJSON format
|
||||
string projjson;
|
||||
|
||||
// Used to track the "primary" geometry column (if any)
|
||||
idx_t insertion_index = 0;
|
||||
};
|
||||
|
||||
class GeoParquetFileMetadata {
|
||||
public:
|
||||
explicit GeoParquetFileMetadata(GeoParquetVersion geo_parquet_version) : version(geo_parquet_version) {
|
||||
}
|
||||
void AddGeoParquetStats(const string &column_name, const LogicalType &type, const GeometryStatsData &stats);
|
||||
void Write(duckdb_parquet::FileMetaData &file_meta_data);
|
||||
|
||||
// Try to read GeoParquet metadata. Returns nullptr if not found, invalid or the required spatial extension is not
|
||||
// available.
|
||||
static unique_ptr<GeoParquetFileMetadata> TryRead(const duckdb_parquet::FileMetaData &file_meta_data,
|
||||
const ClientContext &context);
|
||||
const unordered_map<string, GeoParquetColumnMetadata> &GetColumnMeta() const;
|
||||
|
||||
static unique_ptr<ColumnReader> CreateColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema,
|
||||
ClientContext &context);
|
||||
|
||||
bool IsGeometryColumn(const string &column_name) const;
|
||||
|
||||
static bool IsGeoParquetConversionEnabled(const ClientContext &context);
|
||||
static LogicalType GeometryType();
|
||||
|
||||
private:
|
||||
mutex write_lock;
|
||||
unordered_map<string, GeoParquetColumnMetadata> geometry_columns;
|
||||
GeoParquetVersion version;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
196
external/duckdb/extension/parquet/include/parquet.json
vendored
Normal file
196
external/duckdb/extension/parquet/include/parquet.json
vendored
Normal file
@@ -0,0 +1,196 @@
|
||||
[
|
||||
{
|
||||
"class": "ParquetColumnDefinition",
|
||||
"includes": [
|
||||
"parquet_reader.hpp"
|
||||
],
|
||||
"members": [
|
||||
{
|
||||
"id": 100,
|
||||
"name": "field_id",
|
||||
"type": "int32_t"
|
||||
},
|
||||
{
|
||||
"id": 101,
|
||||
"name": "name",
|
||||
"type": "string"
|
||||
},
|
||||
{
|
||||
"id": 103,
|
||||
"name": "type",
|
||||
"type": "LogicalType"
|
||||
},
|
||||
{
|
||||
"id": 104,
|
||||
"name": "default_value",
|
||||
"type": "Value"
|
||||
},
|
||||
{
|
||||
"id": 105,
|
||||
"name": "identifier",
|
||||
"type": "Value",
|
||||
"default": "Value()"
|
||||
}
|
||||
],
|
||||
"pointer_type": "none"
|
||||
},
|
||||
{
|
||||
"class": "ParquetEncryptionConfig",
|
||||
"includes": [
|
||||
"parquet_crypto.hpp"
|
||||
],
|
||||
"members": [
|
||||
{
|
||||
"id": 100,
|
||||
"name": "footer_key",
|
||||
"type": "string"
|
||||
},
|
||||
{
|
||||
"id": 101,
|
||||
"name": "column_keys",
|
||||
"type": "unordered_map<string, string>"
|
||||
}
|
||||
],
|
||||
"pointer_type": "shared_ptr"
|
||||
},
|
||||
{
|
||||
"class": "ParquetOptionsSerialization",
|
||||
"includes": [
|
||||
"parquet_reader.hpp"
|
||||
],
|
||||
"members": [
|
||||
{
|
||||
"id": 100,
|
||||
"name": "binary_as_string",
|
||||
"type": "bool",
|
||||
"property": "parquet_options.binary_as_string"
|
||||
},
|
||||
{
|
||||
"id": 101,
|
||||
"name": "file_row_number",
|
||||
"type": "bool",
|
||||
"property": "parquet_options.file_row_number"
|
||||
},
|
||||
{
|
||||
"id": 102,
|
||||
"name": "file_options",
|
||||
"type": "MultiFileOptions"
|
||||
},
|
||||
{
|
||||
"id": 103,
|
||||
"name": "schema",
|
||||
"type": "vector<ParquetColumnDefinition>",
|
||||
"property": "parquet_options.schema"
|
||||
},
|
||||
{
|
||||
"id": 104,
|
||||
"name": "encryption_config",
|
||||
"type": "shared_ptr<ParquetEncryptionConfig>",
|
||||
"default": "nullptr",
|
||||
"property": "parquet_options.encryption_config"
|
||||
},
|
||||
{
|
||||
"id": 105,
|
||||
"name": "debug_use_openssl",
|
||||
"type": "bool",
|
||||
"default": "true",
|
||||
"property": "parquet_options.debug_use_openssl"
|
||||
},
|
||||
{
|
||||
"id": 106,
|
||||
"name": "explicit_cardinality",
|
||||
"type": "idx_t",
|
||||
"default": "0",
|
||||
"property": "parquet_options.explicit_cardinality"
|
||||
},
|
||||
{
|
||||
"id": 107,
|
||||
"name": "can_have_nan",
|
||||
"type": "bool",
|
||||
"default": "false",
|
||||
"property": "parquet_options.can_have_nan"
|
||||
}
|
||||
],
|
||||
"pointer_type": "none"
|
||||
},
|
||||
{
|
||||
"class": "FieldID",
|
||||
"includes": [
|
||||
"parquet_field_id.hpp"
|
||||
],
|
||||
"members": [
|
||||
{
|
||||
"id": 100,
|
||||
"name": "set",
|
||||
"type": "bool"
|
||||
},
|
||||
{
|
||||
"id": 101,
|
||||
"name": "field_id",
|
||||
"type": "int32_t"
|
||||
},
|
||||
{
|
||||
"id": 102,
|
||||
"name": "child_field_ids",
|
||||
"type": "ChildFieldIDs"
|
||||
}
|
||||
],
|
||||
"pointer_type": "none"
|
||||
},
|
||||
{
|
||||
"class": "ChildFieldIDs",
|
||||
"includes": [
|
||||
"parquet_field_id.hpp"
|
||||
],
|
||||
"members": [
|
||||
{
|
||||
"id": 100,
|
||||
"name": "ids",
|
||||
"type": "case_insensitive_map_t<FieldID>",
|
||||
"serialize_property": "ids.operator*()",
|
||||
"deserialize_property": "ids.operator*()"
|
||||
}
|
||||
],
|
||||
"pointer_type": "none"
|
||||
},
|
||||
{
|
||||
"class": "ShreddingType",
|
||||
"includes": [
|
||||
"parquet_shredding.hpp"
|
||||
],
|
||||
"members": [
|
||||
{
|
||||
"id": 100,
|
||||
"name": "set",
|
||||
"type": "bool"
|
||||
},
|
||||
{
|
||||
"id": 101,
|
||||
"name": "type",
|
||||
"type": "LogicalType"
|
||||
},
|
||||
{
|
||||
"id": 102,
|
||||
"name": "children",
|
||||
"type": "ChildShreddingTypes"
|
||||
}
|
||||
],
|
||||
"pointer_type": "none"
|
||||
},
|
||||
{
|
||||
"class": "ChildShreddingTypes",
|
||||
"includes": [
|
||||
"parquet_shredding.hpp"
|
||||
],
|
||||
"members": [
|
||||
{
|
||||
"id": 100,
|
||||
"name": "types",
|
||||
"type": "case_insensitive_map_t<ShreddingType>",
|
||||
"serialize_property": "types.operator*()",
|
||||
"deserialize_property": "types.operator*()"
|
||||
}
|
||||
],
|
||||
"pointer_type": "none"
|
||||
}
|
||||
]
|
||||
61
external/duckdb/extension/parquet/include/parquet_bss_decoder.hpp
vendored
Normal file
61
external/duckdb/extension/parquet/include/parquet_bss_decoder.hpp
vendored
Normal file
@@ -0,0 +1,61 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_bss_decoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
#include "parquet_types.h"
|
||||
#include "resizable_buffer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
/// Decoder for the Byte Stream Split encoding
|
||||
class BssDecoder {
|
||||
public:
|
||||
/// Create a decoder object. buffer/buffer_len is the encoded data.
|
||||
BssDecoder(data_ptr_t buffer, uint32_t buffer_len) : buffer_(buffer, buffer_len), value_offset_(0) {
|
||||
}
|
||||
|
||||
public:
|
||||
template <typename T>
|
||||
void GetBatch(data_ptr_t values_target_ptr, uint32_t batch_size) {
|
||||
if (buffer_.len % sizeof(T) != 0) {
|
||||
duckdb::stringstream error;
|
||||
error << "Data buffer size for the BYTE_STREAM_SPLIT encoding (" << buffer_.len
|
||||
<< ") should be a multiple of the type size (" << sizeof(T) << ")";
|
||||
throw std::runtime_error(error.str());
|
||||
}
|
||||
uint32_t num_buffer_values = buffer_.len / sizeof(T);
|
||||
|
||||
buffer_.available((value_offset_ + batch_size) * sizeof(T));
|
||||
|
||||
for (uint32_t byte_offset = 0; byte_offset < sizeof(T); ++byte_offset) {
|
||||
data_ptr_t input_bytes = buffer_.ptr + byte_offset * num_buffer_values + value_offset_;
|
||||
for (uint32_t i = 0; i < batch_size; ++i) {
|
||||
values_target_ptr[byte_offset + i * sizeof(T)] = *(input_bytes + i);
|
||||
}
|
||||
}
|
||||
value_offset_ += batch_size;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void Skip(uint32_t batch_size) {
|
||||
if (buffer_.len % sizeof(T) != 0) {
|
||||
duckdb::stringstream error;
|
||||
error << "Data buffer size for the BYTE_STREAM_SPLIT encoding (" << buffer_.len
|
||||
<< ") should be a multiple of the type size (" << sizeof(T) << ")";
|
||||
throw std::runtime_error(error.str());
|
||||
}
|
||||
buffer_.available((value_offset_ + batch_size) * sizeof(T));
|
||||
value_offset_ += batch_size;
|
||||
}
|
||||
|
||||
private:
|
||||
ByteBuffer buffer_;
|
||||
uint32_t value_offset_;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
47
external/duckdb/extension/parquet/include/parquet_bss_encoder.hpp
vendored
Normal file
47
external/duckdb/extension/parquet/include/parquet_bss_encoder.hpp
vendored
Normal file
@@ -0,0 +1,47 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_bss_encoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "decode_utils.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class BssEncoder {
|
||||
public:
|
||||
explicit BssEncoder(const idx_t total_value_count_p, const idx_t bit_width_p)
|
||||
: total_value_count(total_value_count_p), bit_width(bit_width_p), count(0) {
|
||||
}
|
||||
|
||||
public:
|
||||
void BeginWrite(Allocator &allocator) {
|
||||
buffer = allocator.Allocate(total_value_count * bit_width + 1);
|
||||
}
|
||||
|
||||
template <class T>
|
||||
void WriteValue(const T &value) {
|
||||
D_ASSERT(sizeof(T) == bit_width);
|
||||
for (idx_t i = 0; i < sizeof(T); i++) {
|
||||
buffer.get()[i * total_value_count + count] = reinterpret_cast<const_data_ptr_t>(&value)[i];
|
||||
}
|
||||
count++;
|
||||
}
|
||||
|
||||
void FinishWrite(WriteStream &writer) {
|
||||
writer.WriteData(buffer.get(), total_value_count * bit_width);
|
||||
}
|
||||
|
||||
private:
|
||||
const idx_t total_value_count;
|
||||
const idx_t bit_width;
|
||||
|
||||
idx_t count;
|
||||
AllocatedData buffer;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
58
external/duckdb/extension/parquet/include/parquet_column_schema.hpp
vendored
Normal file
58
external/duckdb/extension/parquet/include/parquet_column_schema.hpp
vendored
Normal file
@@ -0,0 +1,58 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_column_schema.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "parquet_types.h"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
using duckdb_parquet::FileMetaData;
|
||||
struct ParquetOptions;
|
||||
|
||||
enum class ParquetColumnSchemaType { COLUMN, FILE_ROW_NUMBER, GEOMETRY, EXPRESSION, VARIANT };
|
||||
|
||||
enum class ParquetExtraTypeInfo {
|
||||
NONE,
|
||||
IMPALA_TIMESTAMP,
|
||||
UNIT_NS,
|
||||
UNIT_MS,
|
||||
UNIT_MICROS,
|
||||
DECIMAL_BYTE_ARRAY,
|
||||
DECIMAL_INT32,
|
||||
DECIMAL_INT64,
|
||||
FLOAT16
|
||||
};
|
||||
|
||||
struct ParquetColumnSchema {
|
||||
ParquetColumnSchema() = default;
|
||||
ParquetColumnSchema(idx_t max_define, idx_t max_repeat, idx_t schema_index, idx_t file_index,
|
||||
ParquetColumnSchemaType schema_type = ParquetColumnSchemaType::COLUMN);
|
||||
ParquetColumnSchema(string name, LogicalType type, idx_t max_define, idx_t max_repeat, idx_t schema_index,
|
||||
idx_t column_index, ParquetColumnSchemaType schema_type = ParquetColumnSchemaType::COLUMN);
|
||||
ParquetColumnSchema(ParquetColumnSchema parent, LogicalType result_type, ParquetColumnSchemaType schema_type);
|
||||
|
||||
ParquetColumnSchemaType schema_type;
|
||||
string name;
|
||||
LogicalType type;
|
||||
idx_t max_define;
|
||||
idx_t max_repeat;
|
||||
idx_t schema_index;
|
||||
idx_t column_index;
|
||||
optional_idx parent_schema_index;
|
||||
uint32_t type_length = 0;
|
||||
uint32_t type_scale = 0;
|
||||
duckdb_parquet::Type::type parquet_type = duckdb_parquet::Type::INT32;
|
||||
ParquetExtraTypeInfo type_info = ParquetExtraTypeInfo::NONE;
|
||||
vector<ParquetColumnSchema> children;
|
||||
|
||||
unique_ptr<BaseStatistics> Stats(const FileMetaData &file_meta_data, const ParquetOptions &parquet_options,
|
||||
idx_t row_group_idx_p, const vector<duckdb_parquet::ColumnChunk> &columns) const;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
89
external/duckdb/extension/parquet/include/parquet_crypto.hpp
vendored
Normal file
89
external/duckdb/extension/parquet/include/parquet_crypto.hpp
vendored
Normal file
@@ -0,0 +1,89 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_crypto.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "parquet_types.h"
|
||||
#include "duckdb/common/encryption_state.hpp"
|
||||
#include "duckdb/storage/object_cache.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
using duckdb_apache::thrift::TBase;
|
||||
using duckdb_apache::thrift::protocol::TProtocol;
|
||||
|
||||
class BufferedFileWriter;
|
||||
|
||||
class ParquetKeys : public ObjectCacheEntry {
|
||||
public:
|
||||
static ParquetKeys &Get(ClientContext &context);
|
||||
|
||||
public:
|
||||
void AddKey(const string &key_name, const string &key);
|
||||
bool HasKey(const string &key_name) const;
|
||||
const string &GetKey(const string &key_name) const;
|
||||
|
||||
public:
|
||||
static string ObjectType();
|
||||
string GetObjectType() override;
|
||||
|
||||
private:
|
||||
unordered_map<string, string> keys;
|
||||
};
|
||||
|
||||
class ParquetEncryptionConfig {
|
||||
public:
|
||||
explicit ParquetEncryptionConfig();
|
||||
ParquetEncryptionConfig(ClientContext &context, const Value &arg);
|
||||
ParquetEncryptionConfig(string footer_key);
|
||||
|
||||
public:
|
||||
static shared_ptr<ParquetEncryptionConfig> Create(ClientContext &context, const Value &arg);
|
||||
const string &GetFooterKey() const;
|
||||
|
||||
public:
|
||||
void Serialize(Serializer &serializer) const;
|
||||
static shared_ptr<ParquetEncryptionConfig> Deserialize(Deserializer &deserializer);
|
||||
|
||||
private:
|
||||
//! The encryption key used for the footer
|
||||
string footer_key;
|
||||
//! Mapping from column name to key name
|
||||
unordered_map<string, string> column_keys;
|
||||
};
|
||||
|
||||
class ParquetCrypto {
|
||||
public:
|
||||
//! Encrypted modules
|
||||
static constexpr idx_t LENGTH_BYTES = 4;
|
||||
static constexpr idx_t NONCE_BYTES = 12;
|
||||
static constexpr idx_t TAG_BYTES = 16;
|
||||
|
||||
//! Block size we encrypt/decrypt
|
||||
static constexpr idx_t CRYPTO_BLOCK_SIZE = 4096;
|
||||
static constexpr idx_t BLOCK_SIZE = 16;
|
||||
|
||||
public:
|
||||
//! Decrypt and read a Thrift object from the transport protocol
|
||||
static uint32_t Read(TBase &object, TProtocol &iprot, const string &key, const EncryptionUtil &encryption_util_p);
|
||||
//! Encrypt and write a Thrift object to the transport protocol
|
||||
static uint32_t Write(const TBase &object, TProtocol &oprot, const string &key,
|
||||
const EncryptionUtil &encryption_util_p);
|
||||
//! Decrypt and read a buffer
|
||||
static uint32_t ReadData(TProtocol &iprot, const data_ptr_t buffer, const uint32_t buffer_size, const string &key,
|
||||
const EncryptionUtil &encryption_util_p);
|
||||
//! Encrypt and write a buffer to a file
|
||||
static uint32_t WriteData(TProtocol &oprot, const const_data_ptr_t buffer, const uint32_t buffer_size,
|
||||
const string &key, const EncryptionUtil &encryption_util_p);
|
||||
|
||||
public:
|
||||
static void AddKey(ClientContext &context, const FunctionParameters ¶meters);
|
||||
static bool ValidKey(const std::string &key);
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
163
external/duckdb/extension/parquet/include/parquet_dbp_decoder.hpp
vendored
Normal file
163
external/duckdb/extension/parquet/include/parquet_dbp_decoder.hpp
vendored
Normal file
@@ -0,0 +1,163 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_dbp_deccoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "decode_utils.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class DbpDecoder {
|
||||
public:
|
||||
DbpDecoder(const data_ptr_t buffer, const uint32_t buffer_len)
|
||||
: buffer_(buffer, buffer_len),
|
||||
//<block size in values> <number of miniblocks in a block> <total value count> <first value>
|
||||
block_size_in_values(ParquetDecodeUtils::VarintDecode<uint64_t>(buffer_)),
|
||||
number_of_miniblocks_per_block(DecodeNumberOfMiniblocksPerBlock(buffer_)),
|
||||
number_of_values_in_a_miniblock(block_size_in_values / number_of_miniblocks_per_block),
|
||||
total_value_count(ParquetDecodeUtils::VarintDecode<uint64_t>(buffer_)),
|
||||
previous_value(ParquetDecodeUtils::ZigzagToInt(ParquetDecodeUtils::VarintDecode<uint64_t>(buffer_))),
|
||||
// init state to something sane
|
||||
is_first_value(true), read_values(0), min_delta(NumericLimits<int64_t>::Maximum()),
|
||||
miniblock_index(number_of_miniblocks_per_block - 1), list_of_bitwidths_of_miniblocks(nullptr),
|
||||
miniblock_offset(number_of_values_in_a_miniblock),
|
||||
unpacked_data_offset(BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE) {
|
||||
if (!(block_size_in_values % number_of_miniblocks_per_block == 0 &&
|
||||
number_of_values_in_a_miniblock % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE == 0)) {
|
||||
throw InvalidInputException("Parquet file has invalid block sizes for DELTA_BINARY_PACKED");
|
||||
}
|
||||
}
|
||||
|
||||
ByteBuffer BufferPtr() const {
|
||||
return buffer_;
|
||||
}
|
||||
|
||||
uint64_t TotalValues() const {
|
||||
return total_value_count;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void GetBatch(const data_ptr_t target_values_ptr, const idx_t batch_size) {
|
||||
if (read_values + batch_size > total_value_count) {
|
||||
throw std::runtime_error("DBP decode did not find enough values");
|
||||
}
|
||||
read_values += batch_size;
|
||||
GetBatchInternal<T>(target_values_ptr, batch_size);
|
||||
}
|
||||
|
||||
template <class T>
|
||||
void Skip(idx_t skip_count) {
|
||||
if (read_values + skip_count > total_value_count) {
|
||||
throw std::runtime_error("DBP decode did not find enough values");
|
||||
}
|
||||
read_values += skip_count;
|
||||
GetBatchInternal<T, true>(nullptr, skip_count);
|
||||
}
|
||||
|
||||
void Finalize() {
|
||||
if (miniblock_offset == number_of_values_in_a_miniblock) {
|
||||
return;
|
||||
}
|
||||
auto data = make_unsafe_uniq_array<int64_t>(number_of_values_in_a_miniblock);
|
||||
GetBatchInternal<int64_t>(data_ptr_cast(data.get()), number_of_values_in_a_miniblock - miniblock_offset);
|
||||
}
|
||||
|
||||
private:
|
||||
static idx_t DecodeNumberOfMiniblocksPerBlock(ByteBuffer &buffer) {
|
||||
auto res = ParquetDecodeUtils::VarintDecode<uint64_t>(buffer);
|
||||
if (res == 0) {
|
||||
throw InvalidInputException(
|
||||
"Parquet file has invalid number of miniblocks per block for DELTA_BINARY_PACKED");
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
template <typename T, bool SKIP_READ = false>
|
||||
void GetBatchInternal(const data_ptr_t target_values_ptr, const idx_t batch_size) {
|
||||
if (batch_size == 0) {
|
||||
return;
|
||||
}
|
||||
D_ASSERT(target_values_ptr || SKIP_READ);
|
||||
|
||||
T *target_values = nullptr;
|
||||
if (!SKIP_READ) {
|
||||
target_values = reinterpret_cast<T *>(target_values_ptr);
|
||||
}
|
||||
idx_t target_values_offset = 0;
|
||||
if (is_first_value) {
|
||||
if (!SKIP_READ) {
|
||||
target_values[0] = static_cast<T>(previous_value);
|
||||
}
|
||||
target_values_offset++;
|
||||
is_first_value = false;
|
||||
}
|
||||
|
||||
while (target_values_offset < batch_size) {
|
||||
// Copy over any remaining data
|
||||
const idx_t next = MinValue(batch_size - target_values_offset,
|
||||
BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE - unpacked_data_offset);
|
||||
if (next != 0) {
|
||||
for (idx_t i = 0; i < next; i++) {
|
||||
const auto &unpacked_value = unpacked_data[unpacked_data_offset + i];
|
||||
auto current_value = static_cast<T>(static_cast<uint64_t>(previous_value) +
|
||||
static_cast<uint64_t>(min_delta) + unpacked_value);
|
||||
if (!SKIP_READ) {
|
||||
target_values[target_values_offset + i] = current_value;
|
||||
}
|
||||
previous_value = static_cast<int64_t>(current_value);
|
||||
}
|
||||
target_values_offset += next;
|
||||
unpacked_data_offset += next;
|
||||
continue;
|
||||
}
|
||||
|
||||
// Move to next miniblock / block
|
||||
D_ASSERT(unpacked_data_offset == BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE);
|
||||
D_ASSERT(miniblock_index < number_of_miniblocks_per_block);
|
||||
D_ASSERT(miniblock_offset <= number_of_values_in_a_miniblock);
|
||||
if (miniblock_offset == number_of_values_in_a_miniblock) {
|
||||
miniblock_offset = 0;
|
||||
if (++miniblock_index == number_of_miniblocks_per_block) {
|
||||
// <min delta> <list of bitwidths of miniblocks> <miniblocks>
|
||||
min_delta = ParquetDecodeUtils::ZigzagToInt(ParquetDecodeUtils::VarintDecode<uint64_t>(buffer_));
|
||||
buffer_.available(number_of_miniblocks_per_block);
|
||||
list_of_bitwidths_of_miniblocks = buffer_.ptr;
|
||||
buffer_.unsafe_inc(number_of_miniblocks_per_block);
|
||||
miniblock_index = 0;
|
||||
}
|
||||
}
|
||||
|
||||
// Unpack from current miniblock
|
||||
ParquetDecodeUtils::BitUnpackAligned(buffer_, unpacked_data,
|
||||
BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE,
|
||||
list_of_bitwidths_of_miniblocks[miniblock_index]);
|
||||
unpacked_data_offset = 0;
|
||||
miniblock_offset += BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE;
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
ByteBuffer buffer_;
|
||||
const idx_t block_size_in_values;
|
||||
const idx_t number_of_miniblocks_per_block;
|
||||
const idx_t number_of_values_in_a_miniblock;
|
||||
const idx_t total_value_count;
|
||||
int64_t previous_value;
|
||||
|
||||
bool is_first_value;
|
||||
idx_t read_values;
|
||||
|
||||
//! Block stuff
|
||||
int64_t min_delta;
|
||||
idx_t miniblock_index;
|
||||
bitpacking_width_t *list_of_bitwidths_of_miniblocks;
|
||||
idx_t miniblock_offset;
|
||||
uint64_t unpacked_data[BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE];
|
||||
idx_t unpacked_data_offset;
|
||||
};
|
||||
} // namespace duckdb
|
||||
229
external/duckdb/extension/parquet/include/parquet_dbp_encoder.hpp
vendored
Normal file
229
external/duckdb/extension/parquet/include/parquet_dbp_encoder.hpp
vendored
Normal file
@@ -0,0 +1,229 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_dbp_encoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "decode_utils.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class DbpEncoder {
|
||||
private:
|
||||
static constexpr uint64_t BLOCK_SIZE_IN_VALUES = 2048;
|
||||
static constexpr uint64_t NUMBER_OF_MINIBLOCKS_IN_A_BLOCK = 8;
|
||||
static constexpr uint64_t NUMBER_OF_VALUES_IN_A_MINIBLOCK = BLOCK_SIZE_IN_VALUES / NUMBER_OF_MINIBLOCKS_IN_A_BLOCK;
|
||||
|
||||
public:
|
||||
explicit DbpEncoder(const idx_t total_value_count_p) : total_value_count(total_value_count_p), count(0) {
|
||||
}
|
||||
|
||||
public:
|
||||
template <class T>
|
||||
void BeginWrite(WriteStream &writer, const T &first_value) {
|
||||
throw InternalException("DbpEncoder should only be used with integers");
|
||||
}
|
||||
|
||||
template <class T>
|
||||
void WriteValue(WriteStream &writer, const T &value) {
|
||||
throw InternalException("DbpEncoder should only be used with integers");
|
||||
}
|
||||
|
||||
void FinishWrite(WriteStream &writer) {
|
||||
if (count + block_count != total_value_count) {
|
||||
throw InternalException("value count mismatch when writing DELTA_BINARY_PACKED");
|
||||
}
|
||||
if (block_count != 0) {
|
||||
WriteBlock(writer);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
void BeginWriteInternal(WriteStream &writer, const int64_t &first_value) {
|
||||
// <block size in values> <number of miniblocks in a block> <total value count> <first value>
|
||||
|
||||
// the block size is a multiple of 128; it is stored as a ULEB128 int
|
||||
ParquetDecodeUtils::VarintEncode(BLOCK_SIZE_IN_VALUES, writer);
|
||||
// the miniblock count per block is a divisor of the block size such that their quotient,
|
||||
// the number of values in a miniblock, is a multiple of 32
|
||||
static_assert(BLOCK_SIZE_IN_VALUES % NUMBER_OF_MINIBLOCKS_IN_A_BLOCK == 0 &&
|
||||
NUMBER_OF_VALUES_IN_A_MINIBLOCK % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE == 0,
|
||||
"invalid block sizes for DELTA_BINARY_PACKED");
|
||||
// it is stored as a ULEB128 int
|
||||
ParquetDecodeUtils::VarintEncode(NUMBER_OF_MINIBLOCKS_IN_A_BLOCK, writer);
|
||||
// the total value count is stored as a ULEB128 int
|
||||
ParquetDecodeUtils::VarintEncode(total_value_count, writer);
|
||||
// the first value is stored as a zigzag ULEB128 int
|
||||
ParquetDecodeUtils::VarintEncode(ParquetDecodeUtils::IntToZigzag(first_value), writer);
|
||||
|
||||
// initialize
|
||||
if (total_value_count != 0) {
|
||||
count++;
|
||||
}
|
||||
previous_value = first_value;
|
||||
|
||||
min_delta = NumericLimits<int64_t>::Maximum();
|
||||
block_count = 0;
|
||||
}
|
||||
|
||||
void WriteValueInternal(WriteStream &writer, const int64_t &value) {
|
||||
// 1. Compute the differences between consecutive elements. For the first element in the block,
|
||||
// use the last element in the previous block or, in the case of the first block,
|
||||
// use the first value of the whole sequence, stored in the header.
|
||||
|
||||
// Subtractions in steps 1) and 2) may incur signed arithmetic overflow,
|
||||
// and so will the corresponding additions when decoding.
|
||||
// Overflow should be allowed and handled as wrapping around in 2’s complement notation
|
||||
// so that the original values are correctly restituted.
|
||||
// This may require explicit care in some programming languages
|
||||
// (for example by doing all arithmetic in the unsigned domain).
|
||||
const auto delta = static_cast<int64_t>(static_cast<uint64_t>(value) - static_cast<uint64_t>(previous_value));
|
||||
previous_value = value;
|
||||
// Compute the frame of reference (the minimum of the deltas in the block).
|
||||
min_delta = MinValue(min_delta, delta);
|
||||
// append. if block is full, write it out
|
||||
data[block_count++] = delta;
|
||||
if (block_count == BLOCK_SIZE_IN_VALUES) {
|
||||
WriteBlock(writer);
|
||||
}
|
||||
}
|
||||
|
||||
void WriteBlock(WriteStream &writer) {
|
||||
D_ASSERT(count + block_count == total_value_count || block_count == BLOCK_SIZE_IN_VALUES);
|
||||
const auto number_of_miniblocks =
|
||||
(block_count + NUMBER_OF_VALUES_IN_A_MINIBLOCK - 1) / NUMBER_OF_VALUES_IN_A_MINIBLOCK;
|
||||
for (idx_t miniblock_idx = 0; miniblock_idx < number_of_miniblocks; miniblock_idx++) {
|
||||
for (idx_t i = 0; i < NUMBER_OF_VALUES_IN_A_MINIBLOCK; i++) {
|
||||
const idx_t index = miniblock_idx * NUMBER_OF_VALUES_IN_A_MINIBLOCK + i;
|
||||
auto &value = data[index];
|
||||
if (index < block_count) {
|
||||
// 2. Compute the frame of reference (the minimum of the deltas in the block).
|
||||
// Subtract this min delta from all deltas in the block.
|
||||
// This guarantees that all values are non-negative.
|
||||
D_ASSERT(min_delta <= value);
|
||||
value = static_cast<int64_t>(static_cast<uint64_t>(value) - static_cast<uint64_t>(min_delta));
|
||||
} else {
|
||||
// If there are not enough values to fill the last miniblock, we pad the miniblock
|
||||
// so that its length is always the number of values in a full miniblock multiplied by the bit
|
||||
// width. The values of the padding bits should be zero, but readers must accept paddings consisting
|
||||
// of arbitrary bits as well.
|
||||
value = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (idx_t miniblock_idx = 0; miniblock_idx < NUMBER_OF_MINIBLOCKS_IN_A_BLOCK; miniblock_idx++) {
|
||||
auto &width = list_of_bitwidths_of_miniblocks[miniblock_idx];
|
||||
if (miniblock_idx < number_of_miniblocks) {
|
||||
const auto src = &data[miniblock_idx * NUMBER_OF_VALUES_IN_A_MINIBLOCK];
|
||||
width = BitpackingPrimitives::MinimumBitWidth(reinterpret_cast<uint64_t *>(src),
|
||||
NUMBER_OF_VALUES_IN_A_MINIBLOCK);
|
||||
D_ASSERT(width <= sizeof(int64_t) * 8);
|
||||
} else {
|
||||
// If, in the last block, less than <number of miniblocks in a block> miniblocks are needed to store the
|
||||
// values, the bytes storing the bit widths of the unneeded miniblocks are still present, their value
|
||||
// should be zero, but readers must accept arbitrary values as well. There are no additional padding
|
||||
// bytes for the miniblock bodies though, as if their bit widths were 0 (regardless of the actual byte
|
||||
// values). The reader knows when to stop reading by keeping track of the number of values read.
|
||||
width = 0;
|
||||
}
|
||||
}
|
||||
|
||||
// 3. Encode the frame of reference (min delta) as a zigzag ULEB128 int
|
||||
// followed by the bit widths of the miniblocks
|
||||
// and the delta values (minus the min delta) bit-packed per miniblock.
|
||||
// <min delta> <list of bitwidths of miniblocks> <miniblocks>
|
||||
|
||||
// the min delta is a zigzag ULEB128 int (we compute a minimum as we need positive integers for bit packing)
|
||||
ParquetDecodeUtils::VarintEncode(ParquetDecodeUtils::IntToZigzag(min_delta), writer);
|
||||
// the bitwidth of each block is stored as a byte
|
||||
writer.WriteData(list_of_bitwidths_of_miniblocks, NUMBER_OF_MINIBLOCKS_IN_A_BLOCK);
|
||||
// each miniblock is a list of bit packed ints according to the bit width stored at the beginning of the block
|
||||
for (idx_t miniblock_idx = 0; miniblock_idx < number_of_miniblocks; miniblock_idx++) {
|
||||
const auto src = &data[miniblock_idx * NUMBER_OF_VALUES_IN_A_MINIBLOCK];
|
||||
const auto &width = list_of_bitwidths_of_miniblocks[miniblock_idx];
|
||||
memset(data_packed, 0, sizeof(data_packed));
|
||||
ParquetDecodeUtils::BitPackAligned(reinterpret_cast<uint64_t *>(src), data_packed,
|
||||
NUMBER_OF_VALUES_IN_A_MINIBLOCK, width);
|
||||
const auto write_size = NUMBER_OF_VALUES_IN_A_MINIBLOCK * width / 8;
|
||||
#ifdef DEBUG
|
||||
// immediately verify that unpacking yields the input data
|
||||
int64_t verification_data[NUMBER_OF_VALUES_IN_A_MINIBLOCK];
|
||||
ByteBuffer byte_buffer(data_ptr_cast(data_packed), write_size);
|
||||
bitpacking_width_t bitpack_pos = 0;
|
||||
ParquetDecodeUtils::BitUnpack(byte_buffer, bitpack_pos, reinterpret_cast<uint64_t *>(verification_data),
|
||||
NUMBER_OF_VALUES_IN_A_MINIBLOCK, width);
|
||||
for (idx_t i = 0; i < NUMBER_OF_VALUES_IN_A_MINIBLOCK; i++) {
|
||||
D_ASSERT(src[i] == verification_data[i]);
|
||||
}
|
||||
#endif
|
||||
writer.WriteData(data_packed, write_size);
|
||||
}
|
||||
|
||||
count += block_count;
|
||||
|
||||
min_delta = NumericLimits<int64_t>::Maximum();
|
||||
block_count = 0;
|
||||
}
|
||||
|
||||
private:
|
||||
//! Overall fields
|
||||
const idx_t total_value_count;
|
||||
idx_t count;
|
||||
int64_t previous_value;
|
||||
|
||||
//! Block-specific fields
|
||||
int64_t min_delta;
|
||||
int64_t data[BLOCK_SIZE_IN_VALUES];
|
||||
idx_t block_count;
|
||||
|
||||
//! Bitpacking fields
|
||||
bitpacking_width_t list_of_bitwidths_of_miniblocks[NUMBER_OF_MINIBLOCKS_IN_A_BLOCK];
|
||||
data_t data_packed[NUMBER_OF_VALUES_IN_A_MINIBLOCK * sizeof(int64_t)];
|
||||
};
|
||||
|
||||
template <>
|
||||
inline void DbpEncoder::BeginWrite(WriteStream &writer, const int32_t &first_value) {
|
||||
BeginWriteInternal(writer, first_value);
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void DbpEncoder::BeginWrite(WriteStream &writer, const int64_t &first_value) {
|
||||
BeginWriteInternal(writer, first_value);
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void DbpEncoder::BeginWrite(WriteStream &writer, const uint32_t &first_value) {
|
||||
BeginWriteInternal(writer, first_value);
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void DbpEncoder::BeginWrite(WriteStream &writer, const uint64_t &first_value) {
|
||||
BeginWriteInternal(writer, first_value);
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void DbpEncoder::WriteValue(WriteStream &writer, const int32_t &first_value) {
|
||||
WriteValueInternal(writer, first_value);
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void DbpEncoder::WriteValue(WriteStream &writer, const int64_t &first_value) {
|
||||
WriteValueInternal(writer, first_value);
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void DbpEncoder::WriteValue(WriteStream &writer, const uint32_t &first_value) {
|
||||
WriteValueInternal(writer, first_value);
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void DbpEncoder::WriteValue(WriteStream &writer, const uint64_t &first_value) {
|
||||
WriteValueInternal(writer, first_value);
|
||||
}
|
||||
|
||||
} // namespace duckdb
|
||||
55
external/duckdb/extension/parquet/include/parquet_decimal_utils.hpp
vendored
Normal file
55
external/duckdb/extension/parquet/include/parquet_decimal_utils.hpp
vendored
Normal file
@@ -0,0 +1,55 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_decimal_utils.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ParquetDecimalUtils {
|
||||
public:
|
||||
template <class PHYSICAL_TYPE>
|
||||
static PHYSICAL_TYPE ReadDecimalValue(const_data_ptr_t pointer, idx_t size, const ParquetColumnSchema &) {
|
||||
PHYSICAL_TYPE res = 0;
|
||||
|
||||
auto res_ptr = (uint8_t *)&res;
|
||||
bool positive = (*pointer & 0x80) == 0;
|
||||
|
||||
// numbers are stored as two's complement so some muckery is required
|
||||
for (idx_t i = 0; i < MinValue<idx_t>(size, sizeof(PHYSICAL_TYPE)); i++) {
|
||||
auto byte = *(pointer + (size - i - 1));
|
||||
res_ptr[i] = positive ? byte : byte ^ 0xFF;
|
||||
}
|
||||
// Verify that there are only 0s here
|
||||
if (size > sizeof(PHYSICAL_TYPE)) {
|
||||
for (idx_t i = sizeof(PHYSICAL_TYPE); i < size; i++) {
|
||||
auto byte = *(pointer + (size - i - 1));
|
||||
if (!positive) {
|
||||
byte ^= 0xFF;
|
||||
}
|
||||
if (byte != 0) {
|
||||
throw InvalidInputException("Invalid decimal encoding in Parquet file");
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!positive) {
|
||||
res += 1;
|
||||
return -res;
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
static unique_ptr<ColumnReader> CreateReader(ParquetReader &reader, const ParquetColumnSchema &schema);
|
||||
};
|
||||
|
||||
template <>
|
||||
double ParquetDecimalUtils::ReadDecimalValue(const_data_ptr_t pointer, idx_t size, const ParquetColumnSchema &schema);
|
||||
|
||||
} // namespace duckdb
|
||||
69
external/duckdb/extension/parquet/include/parquet_dlba_encoder.hpp
vendored
Normal file
69
external/duckdb/extension/parquet/include/parquet_dlba_encoder.hpp
vendored
Normal file
@@ -0,0 +1,69 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_dlba_encoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "parquet_dbp_encoder.hpp"
|
||||
#include "duckdb/common/serializer/memory_stream.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class DlbaEncoder {
|
||||
public:
|
||||
DlbaEncoder(const idx_t total_value_count_p, const idx_t total_string_size_p)
|
||||
: dbp_encoder(total_value_count_p), total_string_size(total_string_size_p) {
|
||||
}
|
||||
|
||||
public:
|
||||
template <class T>
|
||||
void BeginWrite(Allocator &, WriteStream &, const T &) {
|
||||
throw InternalException("DlbaEncoder should only be used with strings");
|
||||
}
|
||||
|
||||
template <class T>
|
||||
void WriteValue(WriteStream &, const T &) {
|
||||
throw InternalException("DlbaEncoder should only be used with strings");
|
||||
}
|
||||
|
||||
void FinishWrite(WriteStream &writer) {
|
||||
dbp_encoder.FinishWrite(writer);
|
||||
writer.WriteData(buffer.get(), stream->GetPosition());
|
||||
}
|
||||
|
||||
template <class SRC>
|
||||
static idx_t GetStringSize(const SRC &) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
private:
|
||||
DbpEncoder dbp_encoder;
|
||||
const idx_t total_string_size;
|
||||
AllocatedData buffer;
|
||||
unsafe_unique_ptr<MemoryStream> stream;
|
||||
};
|
||||
|
||||
template <>
|
||||
inline void DlbaEncoder::BeginWrite(Allocator &allocator, WriteStream &writer, const string_t &first_value) {
|
||||
buffer = allocator.Allocate(total_string_size + 1);
|
||||
stream = make_unsafe_uniq<MemoryStream>(buffer.get(), buffer.GetSize());
|
||||
dbp_encoder.BeginWrite(writer, UnsafeNumericCast<int64_t>(first_value.GetSize()));
|
||||
stream->WriteData(const_data_ptr_cast(first_value.GetData()), first_value.GetSize());
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void DlbaEncoder::WriteValue(WriteStream &writer, const string_t &value) {
|
||||
dbp_encoder.WriteValue(writer, UnsafeNumericCast<int64_t>(value.GetSize()));
|
||||
stream->WriteData(const_data_ptr_cast(value.GetData()), value.GetSize());
|
||||
}
|
||||
|
||||
template <>
|
||||
inline idx_t DlbaEncoder::GetStringSize(const string_t &src_value) {
|
||||
return src_value.GetSize();
|
||||
}
|
||||
|
||||
} // namespace duckdb
|
||||
22
external/duckdb/extension/parquet/include/parquet_extension.hpp
vendored
Normal file
22
external/duckdb/extension/parquet/include/parquet_extension.hpp
vendored
Normal file
@@ -0,0 +1,22 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_extension.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ParquetExtension : public Extension {
|
||||
public:
|
||||
void Load(ExtensionLoader &loader) override;
|
||||
std::string Name() override;
|
||||
std::string Version() const override;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
39
external/duckdb/extension/parquet/include/parquet_field_id.hpp
vendored
Normal file
39
external/duckdb/extension/parquet/include/parquet_field_id.hpp
vendored
Normal file
@@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
|
||||
#include "duckdb/common/serializer/buffered_file_writer.hpp"
|
||||
#include "duckdb/common/case_insensitive_map.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
struct FieldID;
|
||||
struct ChildFieldIDs {
|
||||
ChildFieldIDs();
|
||||
ChildFieldIDs Copy() const;
|
||||
unique_ptr<case_insensitive_map_t<FieldID>> ids;
|
||||
|
||||
void Serialize(Serializer &serializer) const;
|
||||
static ChildFieldIDs Deserialize(Deserializer &source);
|
||||
};
|
||||
|
||||
struct FieldID {
|
||||
public:
|
||||
static constexpr const auto DUCKDB_FIELD_ID = "__duckdb_field_id";
|
||||
FieldID();
|
||||
explicit FieldID(int32_t field_id);
|
||||
FieldID Copy() const;
|
||||
bool set;
|
||||
int32_t field_id;
|
||||
ChildFieldIDs child_field_ids;
|
||||
|
||||
void Serialize(Serializer &serializer) const;
|
||||
static FieldID Deserialize(Deserializer &source);
|
||||
|
||||
public:
|
||||
static void GenerateFieldIDs(ChildFieldIDs &field_ids, idx_t &field_id, const vector<string> &names,
|
||||
const vector<LogicalType> &sql_types);
|
||||
static void GetFieldIDs(const Value &field_ids_value, ChildFieldIDs &field_ids,
|
||||
unordered_set<uint32_t> &unique_field_ids,
|
||||
const case_insensitive_map_t<LogicalType> &name_to_type_map);
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
50
external/duckdb/extension/parquet/include/parquet_file_metadata_cache.hpp
vendored
Normal file
50
external/duckdb/extension/parquet/include/parquet_file_metadata_cache.hpp
vendored
Normal file
@@ -0,0 +1,50 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_file_metadata_cache.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "duckdb/storage/object_cache.hpp"
|
||||
#include "geo_parquet.hpp"
|
||||
#include "parquet_types.h"
|
||||
|
||||
namespace duckdb {
|
||||
struct CachingFileHandle;
|
||||
|
||||
enum class ParquetCacheValidity { VALID, INVALID, UNKNOWN };
|
||||
|
||||
class ParquetFileMetadataCache : public ObjectCacheEntry {
|
||||
public:
|
||||
ParquetFileMetadataCache(unique_ptr<duckdb_parquet::FileMetaData> file_metadata, CachingFileHandle &handle,
|
||||
unique_ptr<GeoParquetFileMetadata> geo_metadata, idx_t footer_size);
|
||||
~ParquetFileMetadataCache() override = default;
|
||||
|
||||
//! Parquet file metadata
|
||||
unique_ptr<const duckdb_parquet::FileMetaData> metadata;
|
||||
|
||||
//! GeoParquet metadata
|
||||
unique_ptr<GeoParquetFileMetadata> geo_metadata;
|
||||
|
||||
//! Parquet footer size
|
||||
idx_t footer_size;
|
||||
|
||||
public:
|
||||
static string ObjectType();
|
||||
string GetObjectType() override;
|
||||
|
||||
bool IsValid(CachingFileHandle &new_handle) const;
|
||||
//! Check if a cache entry is valid based ONLY on the OpenFileInfo (without doing any file system calls)
|
||||
//! If the OpenFileInfo does not have enough information this can return UNKNOWN
|
||||
ParquetCacheValidity IsValid(const OpenFileInfo &info) const;
|
||||
|
||||
private:
|
||||
bool validate;
|
||||
timestamp_t last_modified;
|
||||
string version_tag;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
17
external/duckdb/extension/parquet/include/parquet_float16.hpp
vendored
Normal file
17
external/duckdb/extension/parquet/include/parquet_float16.hpp
vendored
Normal file
@@ -0,0 +1,17 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_timestamp.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
float Float16ToFloat32(const uint16_t &float16_value);
|
||||
|
||||
} // namespace duckdb
|
||||
41
external/duckdb/extension/parquet/include/parquet_metadata.hpp
vendored
Normal file
41
external/duckdb/extension/parquet/include/parquet_metadata.hpp
vendored
Normal file
@@ -0,0 +1,41 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_metadata.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "parquet_reader.hpp"
|
||||
#include "duckdb/function/function_set.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ParquetMetaDataFunction : public TableFunction {
|
||||
public:
|
||||
ParquetMetaDataFunction();
|
||||
};
|
||||
|
||||
class ParquetSchemaFunction : public TableFunction {
|
||||
public:
|
||||
ParquetSchemaFunction();
|
||||
};
|
||||
|
||||
class ParquetKeyValueMetadataFunction : public TableFunction {
|
||||
public:
|
||||
ParquetKeyValueMetadataFunction();
|
||||
};
|
||||
|
||||
class ParquetFileMetadataFunction : public TableFunction {
|
||||
public:
|
||||
ParquetFileMetadataFunction();
|
||||
};
|
||||
|
||||
class ParquetBloomProbeFunction : public TableFunction {
|
||||
public:
|
||||
ParquetBloomProbeFunction();
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
66
external/duckdb/extension/parquet/include/parquet_multi_file_info.hpp
vendored
Normal file
66
external/duckdb/extension/parquet/include/parquet_multi_file_info.hpp
vendored
Normal file
@@ -0,0 +1,66 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_multi_file_info.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb/common/multi_file/multi_file_function.hpp"
|
||||
#include "parquet_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ParquetFileReaderOptions : public BaseFileReaderOptions {
|
||||
public:
|
||||
explicit ParquetFileReaderOptions(ParquetOptions options_p) : options(std::move(options_p)) {
|
||||
}
|
||||
explicit ParquetFileReaderOptions(ClientContext &context) : options(context) {
|
||||
}
|
||||
|
||||
ParquetOptions options;
|
||||
};
|
||||
|
||||
struct ParquetMultiFileInfo : MultiFileReaderInterface {
|
||||
static unique_ptr<MultiFileReaderInterface> CreateInterface(ClientContext &context);
|
||||
|
||||
unique_ptr<BaseFileReaderOptions> InitializeOptions(ClientContext &context,
|
||||
optional_ptr<TableFunctionInfo> info) override;
|
||||
bool ParseCopyOption(ClientContext &context, const string &key, const vector<Value> &values,
|
||||
BaseFileReaderOptions &options, vector<string> &expected_names,
|
||||
vector<LogicalType> &expected_types) override;
|
||||
bool ParseOption(ClientContext &context, const string &key, const Value &val, MultiFileOptions &file_options,
|
||||
BaseFileReaderOptions &options) override;
|
||||
void BindReader(ClientContext &context, vector<LogicalType> &return_types, vector<string> &names,
|
||||
MultiFileBindData &bind_data) override;
|
||||
unique_ptr<TableFunctionData> InitializeBindData(MultiFileBindData &multi_file_data,
|
||||
unique_ptr<BaseFileReaderOptions> options) override;
|
||||
void FinalizeBindData(MultiFileBindData &multi_file_data) override;
|
||||
void GetBindInfo(const TableFunctionData &bind_data, BindInfo &info) override;
|
||||
optional_idx MaxThreads(const MultiFileBindData &bind_data, const MultiFileGlobalState &global_state,
|
||||
FileExpandResult expand_result) override;
|
||||
unique_ptr<GlobalTableFunctionState> InitializeGlobalState(ClientContext &context, MultiFileBindData &bind_data,
|
||||
MultiFileGlobalState &global_state) override;
|
||||
unique_ptr<LocalTableFunctionState> InitializeLocalState(ExecutionContext &, GlobalTableFunctionState &) override;
|
||||
shared_ptr<BaseFileReader> CreateReader(ClientContext &context, GlobalTableFunctionState &gstate,
|
||||
BaseUnionData &union_data, const MultiFileBindData &bind_data_p) override;
|
||||
shared_ptr<BaseFileReader> CreateReader(ClientContext &context, GlobalTableFunctionState &gstate,
|
||||
const OpenFileInfo &file, idx_t file_idx,
|
||||
const MultiFileBindData &bind_data) override;
|
||||
shared_ptr<BaseFileReader> CreateReader(ClientContext &context, const OpenFileInfo &file,
|
||||
BaseFileReaderOptions &options,
|
||||
const MultiFileOptions &file_options) override;
|
||||
unique_ptr<NodeStatistics> GetCardinality(const MultiFileBindData &bind_data, idx_t file_count) override;
|
||||
void GetVirtualColumns(ClientContext &context, MultiFileBindData &bind_data, virtual_column_map_t &result) override;
|
||||
unique_ptr<MultiFileReaderInterface> Copy() override;
|
||||
FileGlobInput GetGlobInput() override;
|
||||
};
|
||||
|
||||
class ParquetScanFunction {
|
||||
public:
|
||||
static TableFunctionSet GetFunctionSet();
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
239
external/duckdb/extension/parquet/include/parquet_reader.hpp
vendored
Normal file
239
external/duckdb/extension/parquet/include/parquet_reader.hpp
vendored
Normal file
@@ -0,0 +1,239 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "duckdb/storage/caching_file_system.hpp"
|
||||
#include "duckdb/common/common.hpp"
|
||||
#include "duckdb/common/encryption_state.hpp"
|
||||
#include "duckdb/common/exception.hpp"
|
||||
#include "duckdb/common/multi_file/base_file_reader.hpp"
|
||||
#include "duckdb/common/multi_file/multi_file_options.hpp"
|
||||
#include "duckdb/common/string_util.hpp"
|
||||
#include "duckdb/common/types/data_chunk.hpp"
|
||||
#include "column_reader.hpp"
|
||||
#include "parquet_file_metadata_cache.hpp"
|
||||
#include "parquet_rle_bp_decoder.hpp"
|
||||
#include "parquet_types.h"
|
||||
#include "resizable_buffer.hpp"
|
||||
#include "duckdb/execution/adaptive_filter.hpp"
|
||||
|
||||
#include <exception>
|
||||
|
||||
namespace duckdb_parquet {
|
||||
namespace format {
|
||||
class FileMetaData;
|
||||
}
|
||||
} // namespace duckdb_parquet
|
||||
|
||||
namespace duckdb {
|
||||
class Allocator;
|
||||
class ClientContext;
|
||||
class BaseStatistics;
|
||||
class TableFilterSet;
|
||||
class ParquetEncryptionConfig;
|
||||
class ParquetReader;
|
||||
|
||||
struct ParquetReaderPrefetchConfig {
|
||||
// Percentage of data in a row group span that should be scanned for enabling whole group prefetch
|
||||
static constexpr double WHOLE_GROUP_PREFETCH_MINIMUM_SCAN = 0.95;
|
||||
};
|
||||
|
||||
struct ParquetScanFilter {
|
||||
ParquetScanFilter(ClientContext &context, idx_t filter_idx, TableFilter &filter);
|
||||
~ParquetScanFilter();
|
||||
ParquetScanFilter(ParquetScanFilter &&) = default;
|
||||
|
||||
idx_t filter_idx;
|
||||
TableFilter &filter;
|
||||
unique_ptr<TableFilterState> filter_state;
|
||||
};
|
||||
|
||||
struct ParquetReaderScanState {
|
||||
vector<idx_t> group_idx_list;
|
||||
int64_t current_group;
|
||||
idx_t offset_in_group;
|
||||
idx_t group_offset;
|
||||
unique_ptr<CachingFileHandle> file_handle;
|
||||
unique_ptr<ColumnReader> root_reader;
|
||||
duckdb_base_std::unique_ptr<duckdb_apache::thrift::protocol::TProtocol> thrift_file_proto;
|
||||
|
||||
bool finished;
|
||||
SelectionVector sel;
|
||||
|
||||
ResizeableBuffer define_buf;
|
||||
ResizeableBuffer repeat_buf;
|
||||
|
||||
bool prefetch_mode = false;
|
||||
bool current_group_prefetched = false;
|
||||
|
||||
//! Adaptive filter
|
||||
unique_ptr<AdaptiveFilter> adaptive_filter;
|
||||
//! Table filter list
|
||||
vector<ParquetScanFilter> scan_filters;
|
||||
|
||||
//! (optional) pointer to the PhysicalOperator for logging
|
||||
optional_ptr<const PhysicalOperator> op;
|
||||
};
|
||||
|
||||
struct ParquetColumnDefinition {
|
||||
public:
|
||||
static ParquetColumnDefinition FromSchemaValue(ClientContext &context, const Value &column_value);
|
||||
|
||||
public:
|
||||
// DEPRECATED, use 'identifier' instead
|
||||
int32_t field_id;
|
||||
string name;
|
||||
LogicalType type;
|
||||
Value default_value;
|
||||
Value identifier;
|
||||
|
||||
public:
|
||||
void Serialize(Serializer &serializer) const;
|
||||
static ParquetColumnDefinition Deserialize(Deserializer &deserializer);
|
||||
};
|
||||
|
||||
struct ParquetOptions {
|
||||
explicit ParquetOptions() {
|
||||
}
|
||||
explicit ParquetOptions(ClientContext &context);
|
||||
|
||||
bool binary_as_string = false;
|
||||
bool variant_legacy_encoding = false;
|
||||
bool file_row_number = false;
|
||||
shared_ptr<ParquetEncryptionConfig> encryption_config;
|
||||
bool debug_use_openssl = true;
|
||||
|
||||
vector<ParquetColumnDefinition> schema;
|
||||
idx_t explicit_cardinality = 0;
|
||||
bool can_have_nan = false; // if floats or doubles can contain NaN values
|
||||
};
|
||||
|
||||
struct ParquetOptionsSerialization {
|
||||
ParquetOptionsSerialization() = default;
|
||||
ParquetOptionsSerialization(ParquetOptions parquet_options_p, MultiFileOptions file_options_p)
|
||||
: parquet_options(std::move(parquet_options_p)), file_options(std::move(file_options_p)) {
|
||||
}
|
||||
|
||||
ParquetOptions parquet_options;
|
||||
MultiFileOptions file_options;
|
||||
|
||||
public:
|
||||
void Serialize(Serializer &serializer) const;
|
||||
static ParquetOptionsSerialization Deserialize(Deserializer &deserializer);
|
||||
};
|
||||
|
||||
struct ParquetUnionData : public BaseUnionData {
|
||||
explicit ParquetUnionData(OpenFileInfo file_p) : BaseUnionData(std::move(file_p)) {
|
||||
}
|
||||
~ParquetUnionData() override;
|
||||
|
||||
unique_ptr<BaseStatistics> GetStatistics(ClientContext &context, const string &name) override;
|
||||
|
||||
ParquetOptions options;
|
||||
shared_ptr<ParquetFileMetadataCache> metadata;
|
||||
unique_ptr<ParquetColumnSchema> root_schema;
|
||||
};
|
||||
|
||||
class ParquetReader : public BaseFileReader {
|
||||
public:
|
||||
ParquetReader(ClientContext &context, OpenFileInfo file, ParquetOptions parquet_options,
|
||||
shared_ptr<ParquetFileMetadataCache> metadata = nullptr);
|
||||
~ParquetReader() override;
|
||||
|
||||
CachingFileSystem fs;
|
||||
Allocator &allocator;
|
||||
shared_ptr<ParquetFileMetadataCache> metadata;
|
||||
ParquetOptions parquet_options;
|
||||
unique_ptr<ParquetColumnSchema> root_schema;
|
||||
shared_ptr<EncryptionUtil> encryption_util;
|
||||
//! How many rows have been read from this file
|
||||
atomic<idx_t> rows_read;
|
||||
|
||||
public:
|
||||
string GetReaderType() const override {
|
||||
return "Parquet";
|
||||
}
|
||||
|
||||
shared_ptr<BaseUnionData> GetUnionData(idx_t file_idx) override;
|
||||
unique_ptr<BaseStatistics> GetStatistics(ClientContext &context, const string &name) override;
|
||||
|
||||
bool TryInitializeScan(ClientContext &context, GlobalTableFunctionState &gstate,
|
||||
LocalTableFunctionState &lstate) override;
|
||||
void Scan(ClientContext &context, GlobalTableFunctionState &global_state, LocalTableFunctionState &local_state,
|
||||
DataChunk &chunk) override;
|
||||
void FinishFile(ClientContext &context, GlobalTableFunctionState &gstate_p) override;
|
||||
double GetProgressInFile(ClientContext &context) override;
|
||||
|
||||
public:
|
||||
void InitializeScan(ClientContext &context, ParquetReaderScanState &state, vector<idx_t> groups_to_read);
|
||||
void Scan(ClientContext &context, ParquetReaderScanState &state, DataChunk &output);
|
||||
|
||||
idx_t NumRows() const;
|
||||
idx_t NumRowGroups() const;
|
||||
|
||||
const duckdb_parquet::FileMetaData *GetFileMetadata() const;
|
||||
|
||||
uint32_t Read(duckdb_apache::thrift::TBase &object, TProtocol &iprot);
|
||||
uint32_t ReadData(duckdb_apache::thrift::protocol::TProtocol &iprot, const data_ptr_t buffer,
|
||||
const uint32_t buffer_size);
|
||||
|
||||
unique_ptr<BaseStatistics> ReadStatistics(const string &name);
|
||||
|
||||
CachingFileHandle &GetHandle() {
|
||||
return *file_handle;
|
||||
}
|
||||
|
||||
static unique_ptr<BaseStatistics> ReadStatistics(ClientContext &context, ParquetOptions parquet_options,
|
||||
shared_ptr<ParquetFileMetadataCache> metadata, const string &name);
|
||||
static unique_ptr<BaseStatistics> ReadStatistics(const ParquetUnionData &union_data, const string &name);
|
||||
|
||||
LogicalType DeriveLogicalType(const SchemaElement &s_ele, ParquetColumnSchema &schema) const;
|
||||
|
||||
void AddVirtualColumn(column_t virtual_column_id) override;
|
||||
|
||||
void GetPartitionStats(vector<PartitionStatistics> &result);
|
||||
static void GetPartitionStats(const duckdb_parquet::FileMetaData &metadata, vector<PartitionStatistics> &result);
|
||||
static bool MetadataCacheEnabled(ClientContext &context);
|
||||
static shared_ptr<ParquetFileMetadataCache> GetMetadataCacheEntry(ClientContext &context, const OpenFileInfo &file);
|
||||
|
||||
private:
|
||||
//! Construct a parquet reader but **do not** open a file, used in ReadStatistics only
|
||||
ParquetReader(ClientContext &context, ParquetOptions parquet_options,
|
||||
shared_ptr<ParquetFileMetadataCache> metadata);
|
||||
|
||||
void InitializeSchema(ClientContext &context);
|
||||
bool ScanInternal(ClientContext &context, ParquetReaderScanState &state, DataChunk &output);
|
||||
//! Parse the schema of the file
|
||||
unique_ptr<ParquetColumnSchema> ParseSchema(ClientContext &context);
|
||||
ParquetColumnSchema ParseSchemaRecursive(idx_t depth, idx_t max_define, idx_t max_repeat, idx_t &next_schema_idx,
|
||||
idx_t &next_file_idx, ClientContext &context);
|
||||
|
||||
unique_ptr<ColumnReader> CreateReader(ClientContext &context);
|
||||
|
||||
unique_ptr<ColumnReader> CreateReaderRecursive(ClientContext &context, const vector<ColumnIndex> &indexes,
|
||||
const ParquetColumnSchema &schema);
|
||||
const duckdb_parquet::RowGroup &GetGroup(ParquetReaderScanState &state);
|
||||
uint64_t GetGroupCompressedSize(ParquetReaderScanState &state);
|
||||
idx_t GetGroupOffset(ParquetReaderScanState &state);
|
||||
// Group span is the distance between the min page offset and the max page offset plus the max page compressed size
|
||||
uint64_t GetGroupSpan(ParquetReaderScanState &state);
|
||||
void PrepareRowGroupBuffer(ParquetReaderScanState &state, idx_t out_col_idx);
|
||||
ParquetColumnSchema ParseColumnSchema(const SchemaElement &s_ele, idx_t max_define, idx_t max_repeat,
|
||||
idx_t schema_index, idx_t column_index,
|
||||
ParquetColumnSchemaType type = ParquetColumnSchemaType::COLUMN);
|
||||
|
||||
MultiFileColumnDefinition ParseColumnDefinition(const duckdb_parquet::FileMetaData &file_meta_data,
|
||||
ParquetColumnSchema &element);
|
||||
|
||||
private:
|
||||
unique_ptr<CachingFileHandle> file_handle;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
158
external/duckdb/extension/parquet/include/parquet_rle_bp_decoder.hpp
vendored
Normal file
158
external/duckdb/extension/parquet/include/parquet_rle_bp_decoder.hpp
vendored
Normal file
@@ -0,0 +1,158 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_rle_bp_decoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
#include "decode_utils.hpp"
|
||||
#include "parquet_types.h"
|
||||
#include "resizable_buffer.hpp"
|
||||
#include "thrift_tools.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class RleBpDecoder {
|
||||
public:
|
||||
/// Create a decoder object. buffer/buffer_len is the decoded data.
|
||||
/// bit_width is the width of each value (before encoding).
|
||||
RleBpDecoder(data_ptr_t buffer, uint32_t buffer_len, uint32_t bit_width)
|
||||
: buffer_(buffer, buffer_len), bit_width_(bit_width), current_value_(0), repeat_count_(0), literal_count_(0) {
|
||||
if (bit_width >= 64) {
|
||||
throw std::runtime_error("Decode bit width too large");
|
||||
}
|
||||
byte_encoded_len = ((bit_width_ + 7) / 8);
|
||||
max_val = (uint64_t(1) << bit_width_) - 1;
|
||||
}
|
||||
|
||||
template <class T>
|
||||
bool HasRepeatedBatch(const uint32_t batch_size, const T value) {
|
||||
if (repeat_count_ == 0 && literal_count_ == 0) {
|
||||
NextCounts();
|
||||
}
|
||||
return repeat_count_ >= batch_size && current_value_ == static_cast<uint64_t>(value);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void GetRepeatedBatch(const uint32_t batch_size, const T value) {
|
||||
D_ASSERT(repeat_count_ >= batch_size && current_value_ == static_cast<uint64_t>(value));
|
||||
repeat_count_ -= batch_size;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void GetBatch(data_ptr_t values_target_ptr, const uint32_t batch_size) {
|
||||
auto values = reinterpret_cast<T *>(values_target_ptr);
|
||||
uint32_t values_read = 0;
|
||||
|
||||
while (values_read < batch_size) {
|
||||
if (repeat_count_ > 0) {
|
||||
auto repeat_batch = MinValue<uint32_t>(batch_size - values_read, repeat_count_);
|
||||
std::fill_n(values + values_read, repeat_batch, static_cast<T>(current_value_));
|
||||
repeat_count_ -= repeat_batch;
|
||||
values_read += repeat_batch;
|
||||
} else if (literal_count_ > 0) {
|
||||
auto literal_batch = MinValue<uint32_t>(batch_size - values_read, literal_count_);
|
||||
ParquetDecodeUtils::BitUnpack<T>(buffer_, bitpack_pos, values + values_read, literal_batch, bit_width_);
|
||||
literal_count_ -= literal_batch;
|
||||
values_read += literal_batch;
|
||||
} else {
|
||||
NextCounts();
|
||||
}
|
||||
}
|
||||
D_ASSERT(values_read == batch_size);
|
||||
}
|
||||
|
||||
void Skip(uint32_t batch_size) {
|
||||
uint32_t values_skipped = 0;
|
||||
|
||||
while (values_skipped < batch_size) {
|
||||
if (repeat_count_ > 0) {
|
||||
auto repeat_batch = MinValue<uint32_t>(batch_size - values_skipped, repeat_count_);
|
||||
repeat_count_ -= repeat_batch;
|
||||
values_skipped += repeat_batch;
|
||||
} else if (literal_count_ > 0) {
|
||||
auto literal_batch = MinValue<uint32_t>(batch_size - values_skipped, literal_count_);
|
||||
ParquetDecodeUtils::Skip(buffer_, bitpack_pos, literal_batch, bit_width_);
|
||||
literal_count_ -= literal_batch;
|
||||
values_skipped += literal_batch;
|
||||
} else {
|
||||
NextCounts();
|
||||
}
|
||||
}
|
||||
D_ASSERT(values_skipped == batch_size);
|
||||
}
|
||||
|
||||
static uint8_t ComputeBitWidth(idx_t val) {
|
||||
if (val == 0) {
|
||||
return 0;
|
||||
}
|
||||
uint8_t ret = 1;
|
||||
while ((((idx_t)1u << (idx_t)ret) - 1) < val) {
|
||||
ret++;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
private:
|
||||
ByteBuffer buffer_;
|
||||
|
||||
/// Number of bits needed to encode the value. Must be between 0 and 64.
|
||||
uint32_t bit_width_;
|
||||
uint64_t current_value_;
|
||||
uint32_t repeat_count_;
|
||||
uint32_t literal_count_;
|
||||
uint8_t byte_encoded_len;
|
||||
uint64_t max_val;
|
||||
|
||||
uint8_t bitpack_pos = 0;
|
||||
|
||||
/// Fills literal_count_ and repeat_count_ with next values. Returns false if there
|
||||
/// are no more.
|
||||
template <bool CHECKED>
|
||||
void NextCountsTemplated() {
|
||||
// Read the next run's indicator int, it could be a literal or repeated run.
|
||||
// The int is encoded as a vlq-encoded value.
|
||||
if (bitpack_pos != 0) {
|
||||
if (CHECKED) {
|
||||
buffer_.inc(1);
|
||||
} else {
|
||||
buffer_.unsafe_inc(1);
|
||||
}
|
||||
bitpack_pos = 0;
|
||||
}
|
||||
auto indicator_value = ParquetDecodeUtils::VarintDecode<uint32_t, CHECKED>(buffer_);
|
||||
|
||||
// lsb indicates if it is a literal run or repeated run
|
||||
bool is_literal = indicator_value & 1;
|
||||
if (is_literal) {
|
||||
literal_count_ = (indicator_value >> 1) * 8;
|
||||
} else {
|
||||
repeat_count_ = indicator_value >> 1;
|
||||
// (ARROW-4018) this is not big-endian compatible, lol
|
||||
current_value_ = 0;
|
||||
if (CHECKED) {
|
||||
buffer_.available(byte_encoded_len);
|
||||
}
|
||||
for (auto i = 0; i < byte_encoded_len; i++) {
|
||||
auto next_byte = Load<uint8_t>(buffer_.ptr + i);
|
||||
current_value_ |= (next_byte << (i * 8));
|
||||
}
|
||||
buffer_.unsafe_inc(byte_encoded_len);
|
||||
// sanity check
|
||||
if (repeat_count_ > 0 && current_value_ > max_val) {
|
||||
throw std::runtime_error("Payload value bigger than allowed. Corrupted file?");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void NextCounts() {
|
||||
if (buffer_.check_available(byte_encoded_len + sizeof(uint32_t) + 2)) {
|
||||
NextCountsTemplated<false>();
|
||||
} else {
|
||||
NextCountsTemplated<true>();
|
||||
}
|
||||
}
|
||||
};
|
||||
} // namespace duckdb
|
||||
155
external/duckdb/extension/parquet/include/parquet_rle_bp_encoder.hpp
vendored
Normal file
155
external/duckdb/extension/parquet/include/parquet_rle_bp_encoder.hpp
vendored
Normal file
@@ -0,0 +1,155 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_rle_bp_encoder.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "decode_utils.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class RleBpEncoder {
|
||||
public:
|
||||
explicit RleBpEncoder(uint32_t bit_width_p) : bit_width(bit_width_p), byte_width((bit_width + 7) / 8) {
|
||||
}
|
||||
|
||||
public:
|
||||
void BeginWrite() {
|
||||
rle_count = 0;
|
||||
bp_block_count = 0;
|
||||
}
|
||||
|
||||
void WriteValue(WriteStream &writer, const uint32_t &value) {
|
||||
if (bp_block_count != 0) {
|
||||
// We already committed to a BP run
|
||||
D_ASSERT(rle_count == 0);
|
||||
bp_block[bp_block_count++] = value;
|
||||
if (bp_block_count == BP_BLOCK_SIZE) {
|
||||
WriteRun(writer);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
if (rle_count == 0) {
|
||||
// Starting fresh, try for an RLE run first
|
||||
rle_value = value;
|
||||
rle_count = 1;
|
||||
return;
|
||||
}
|
||||
|
||||
// We're trying for an RLE run
|
||||
if (rle_value == value) {
|
||||
// Same as current RLE value
|
||||
rle_count++;
|
||||
return;
|
||||
}
|
||||
|
||||
// Value differs from current RLE value
|
||||
if (rle_count >= MINIMUM_RLE_COUNT) {
|
||||
// We have enough values for an RLE run
|
||||
WriteRun(writer);
|
||||
rle_value = value;
|
||||
rle_count = 1;
|
||||
return;
|
||||
}
|
||||
|
||||
// Not enough values, convert and commit to a BP run
|
||||
D_ASSERT(bp_block_count == 0);
|
||||
for (idx_t i = 0; i < rle_count; i++) {
|
||||
bp_block[bp_block_count++] = rle_value;
|
||||
}
|
||||
bp_block[bp_block_count++] = value;
|
||||
rle_count = 0;
|
||||
}
|
||||
|
||||
void WriteMany(WriteStream &writer, uint32_t value, idx_t count) {
|
||||
if (rle_count != 0) {
|
||||
// If an RLE run is going on, write a single value to either finish it or convert to BP
|
||||
WriteValue(writer, value);
|
||||
count--;
|
||||
}
|
||||
|
||||
if (bp_block_count != 0) {
|
||||
// If a BP run is going on, finish it
|
||||
while (bp_block_count != 0 && count > 0) {
|
||||
WriteValue(writer, value);
|
||||
count--;
|
||||
}
|
||||
}
|
||||
|
||||
// Set remaining as current RLE run
|
||||
rle_value = value;
|
||||
rle_count += count;
|
||||
}
|
||||
|
||||
void FinishWrite(WriteStream &writer) {
|
||||
WriteRun(writer);
|
||||
}
|
||||
|
||||
private:
|
||||
//! Meta information
|
||||
uint32_t bit_width;
|
||||
uint32_t byte_width;
|
||||
|
||||
//! RLE stuff
|
||||
static constexpr idx_t MINIMUM_RLE_COUNT = 4;
|
||||
uint32_t rle_value;
|
||||
idx_t rle_count;
|
||||
|
||||
//! BP stuff
|
||||
static constexpr idx_t BP_BLOCK_SIZE = 256;
|
||||
static_assert(BP_BLOCK_SIZE % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE == 0,
|
||||
"BP_BLOCK_SIZE must be divisible by BITPACKING_ALGORITHM_GROUP_SIZE");
|
||||
uint32_t bp_block[BP_BLOCK_SIZE] = {0};
|
||||
uint32_t bp_block_packed[BP_BLOCK_SIZE] = {0};
|
||||
idx_t bp_block_count;
|
||||
|
||||
private:
|
||||
void WriteRun(WriteStream &writer) {
|
||||
if (rle_count != 0) {
|
||||
WriteCurrentBlockRLE(writer);
|
||||
} else {
|
||||
WriteCurrentBlockBP(writer);
|
||||
}
|
||||
}
|
||||
|
||||
void WriteCurrentBlockRLE(WriteStream &writer) {
|
||||
ParquetDecodeUtils::VarintEncode(rle_count << 1 | 0, writer); // (... | 0) signals RLE run
|
||||
D_ASSERT(rle_value >> (byte_width * 8) == 0);
|
||||
switch (byte_width) {
|
||||
case 1:
|
||||
writer.Write<uint8_t>(rle_value);
|
||||
break;
|
||||
case 2:
|
||||
writer.Write<uint16_t>(rle_value);
|
||||
break;
|
||||
case 3:
|
||||
writer.Write<uint8_t>(rle_value & 0xFF);
|
||||
writer.Write<uint8_t>((rle_value >> 8) & 0xFF);
|
||||
writer.Write<uint8_t>((rle_value >> 16) & 0xFF);
|
||||
break;
|
||||
case 4:
|
||||
writer.Write<uint32_t>(rle_value);
|
||||
break;
|
||||
default:
|
||||
throw InternalException("unsupported byte width for RLE encoding");
|
||||
}
|
||||
rle_count = 0;
|
||||
}
|
||||
|
||||
void WriteCurrentBlockBP(WriteStream &writer) {
|
||||
if (bp_block_count == 0) {
|
||||
return;
|
||||
}
|
||||
ParquetDecodeUtils::VarintEncode(BP_BLOCK_SIZE / 8 << 1 | 1, writer); // (... | 1) signals BP run
|
||||
ParquetDecodeUtils::BitPackAligned(bp_block, data_ptr_cast(bp_block_packed), BP_BLOCK_SIZE, bit_width);
|
||||
writer.WriteData(data_ptr_cast(bp_block_packed), BP_BLOCK_SIZE * bit_width / 8);
|
||||
bp_block_count = 0;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
49
external/duckdb/extension/parquet/include/parquet_shredding.hpp
vendored
Normal file
49
external/duckdb/extension/parquet/include/parquet_shredding.hpp
vendored
Normal file
@@ -0,0 +1,49 @@
|
||||
#pragma once
|
||||
|
||||
#include "duckdb/common/serializer/buffered_file_writer.hpp"
|
||||
#include "duckdb/common/case_insensitive_map.hpp"
|
||||
#include "duckdb/common/types/variant.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
struct ShreddingType;
|
||||
|
||||
struct ChildShreddingTypes {
|
||||
public:
|
||||
ChildShreddingTypes();
|
||||
|
||||
public:
|
||||
ChildShreddingTypes Copy() const;
|
||||
|
||||
public:
|
||||
void Serialize(Serializer &serializer) const;
|
||||
static ChildShreddingTypes Deserialize(Deserializer &source);
|
||||
|
||||
public:
|
||||
unique_ptr<case_insensitive_map_t<ShreddingType>> types;
|
||||
};
|
||||
|
||||
struct ShreddingType {
|
||||
public:
|
||||
ShreddingType();
|
||||
explicit ShreddingType(const LogicalType &type);
|
||||
|
||||
public:
|
||||
ShreddingType Copy() const;
|
||||
|
||||
public:
|
||||
void Serialize(Serializer &serializer) const;
|
||||
static ShreddingType Deserialize(Deserializer &source);
|
||||
|
||||
public:
|
||||
static ShreddingType GetShreddingTypes(const Value &val);
|
||||
void AddChild(const string &name, ShreddingType &&child);
|
||||
optional_ptr<const ShreddingType> GetChild(const string &name) const;
|
||||
|
||||
public:
|
||||
bool set = false;
|
||||
LogicalType type;
|
||||
ChildShreddingTypes children;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
111
external/duckdb/extension/parquet/include/parquet_statistics.hpp
vendored
Normal file
111
external/duckdb/extension/parquet/include/parquet_statistics.hpp
vendored
Normal file
@@ -0,0 +1,111 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_statistics.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "duckdb/storage/statistics/base_statistics.hpp"
|
||||
#include "parquet_types.h"
|
||||
#include "resizable_buffer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
using duckdb_parquet::ColumnChunk;
|
||||
using duckdb_parquet::SchemaElement;
|
||||
|
||||
struct LogicalType;
|
||||
struct ParquetColumnSchema;
|
||||
class ResizeableBuffer;
|
||||
|
||||
struct ParquetStatisticsUtils {
|
||||
|
||||
static unique_ptr<BaseStatistics> TransformColumnStatistics(const ParquetColumnSchema &reader,
|
||||
const vector<ColumnChunk> &columns, bool can_have_nan);
|
||||
|
||||
static Value ConvertValue(const LogicalType &type, const ParquetColumnSchema &schema_ele, const std::string &stats);
|
||||
|
||||
static bool BloomFilterSupported(const LogicalTypeId &type_id);
|
||||
|
||||
static bool BloomFilterExcludes(const TableFilter &filter, const duckdb_parquet::ColumnMetaData &column_meta_data,
|
||||
duckdb_apache::thrift::protocol::TProtocol &file_proto, Allocator &allocator);
|
||||
|
||||
static unique_ptr<BaseStatistics> CreateNumericStats(const LogicalType &type, const ParquetColumnSchema &schema_ele,
|
||||
const duckdb_parquet::Statistics &parquet_stats);
|
||||
|
||||
private:
|
||||
static Value ConvertValueInternal(const LogicalType &type, const ParquetColumnSchema &schema_ele,
|
||||
const std::string &stats);
|
||||
};
|
||||
|
||||
class ParquetBloomFilter {
|
||||
static constexpr const idx_t DEFAULT_BLOCK_COUNT = 32; // 4k filter
|
||||
|
||||
public:
|
||||
ParquetBloomFilter(idx_t num_entries, double bloom_filter_false_positive_ratio);
|
||||
ParquetBloomFilter(unique_ptr<ResizeableBuffer> data_p);
|
||||
void FilterInsert(uint64_t x);
|
||||
bool FilterCheck(uint64_t x);
|
||||
void Shrink(idx_t new_block_count);
|
||||
double OneRatio();
|
||||
ResizeableBuffer *Get();
|
||||
|
||||
private:
|
||||
unique_ptr<ResizeableBuffer> data;
|
||||
idx_t block_count;
|
||||
};
|
||||
|
||||
// see https://github.com/apache/parquet-format/blob/master/BloomFilter.md
|
||||
|
||||
struct ParquetBloomBlock {
|
||||
struct ParquetBloomMaskResult {
|
||||
uint8_t bit_set[8] = {0};
|
||||
};
|
||||
|
||||
uint32_t block[8] = {0};
|
||||
|
||||
static bool check_bit(uint32_t &x, const uint8_t i) {
|
||||
D_ASSERT(i < 32);
|
||||
return (x >> i) & (uint32_t)1;
|
||||
}
|
||||
|
||||
static void set_bit(uint32_t &x, const uint8_t i) {
|
||||
D_ASSERT(i < 32);
|
||||
x |= (uint32_t)1 << i;
|
||||
D_ASSERT(check_bit(x, i));
|
||||
}
|
||||
|
||||
static ParquetBloomMaskResult Mask(uint32_t x) {
|
||||
static const uint32_t parquet_bloom_salt[8] = {0x47b6137bU, 0x44974d91U, 0x8824ad5bU, 0xa2b7289dU,
|
||||
0x705495c7U, 0x2df1424bU, 0x9efc4947U, 0x5c6bfb31U};
|
||||
ParquetBloomMaskResult result;
|
||||
for (idx_t i = 0; i < 8; i++) {
|
||||
result.bit_set[i] = (x * parquet_bloom_salt[i]) >> 27;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
static void BlockInsert(ParquetBloomBlock &b, uint32_t x) {
|
||||
auto masked = Mask(x);
|
||||
for (idx_t i = 0; i < 8; i++) {
|
||||
set_bit(b.block[i], masked.bit_set[i]);
|
||||
D_ASSERT(check_bit(b.block[i], masked.bit_set[i]));
|
||||
}
|
||||
}
|
||||
|
||||
static bool BlockCheck(ParquetBloomBlock &b, uint32_t x) {
|
||||
auto masked = Mask(x);
|
||||
for (idx_t i = 0; i < 8; i++) {
|
||||
if (!check_bit(b.block[i], masked.bit_set[i])) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
621
external/duckdb/extension/parquet/include/parquet_support.hpp
vendored
Normal file
621
external/duckdb/extension/parquet/include/parquet_support.hpp
vendored
Normal file
@@ -0,0 +1,621 @@
|
||||
#pragma once
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class StripeStreams {
|
||||
public:
|
||||
virtual ~StripeStreams() = default;
|
||||
|
||||
/**
|
||||
* get column selector for current stripe reading session
|
||||
* @return column selector will hold column projection info
|
||||
*/
|
||||
virtual const dwio::common::ColumnSelector &getColumnSelector() const = 0;
|
||||
|
||||
// Get row reader options
|
||||
virtual const dwio::common::RowReaderOptclass StripeStreams {
|
||||
public:
|
||||
virtual ~StripeStreams() = default;
|
||||
|
||||
/**
|
||||
* get column selector for current stripe reading session
|
||||
* @return column selector will hold column projection info
|
||||
*/
|
||||
virtual const dwio::common::ColumnSelector &getColumnSelector() const = 0;
|
||||
|
||||
// Get row reader options
|
||||
virtual const dwio::common::RowReaderOptions &getRowReaderOptions() const = 0;
|
||||
|
||||
/**
|
||||
* Get the encoding for the given column for this stripe.
|
||||
*/
|
||||
virtual const proto::ColumnEncoding &getEncoding(const EncodingKey &) const = 0;
|
||||
|
||||
/**
|
||||
* Get the stream for the given column/kind in this stripe.
|
||||
* @param streamId stream identifier object
|
||||
* @param throwIfNotFound fail if a stream is required and not found
|
||||
* @return the new stream
|
||||
*/
|
||||
virtual unique_ptr<SeekableInputStream> getStream(const StreamIdentifier &si, bool throwIfNotFound) const = 0;
|
||||
|
||||
/**
|
||||
* visit all streams of given node and execute visitor logic
|
||||
* return number of streams visited
|
||||
*/
|
||||
virtual uint32_t visitStreamsOfNode(uint32_t node, std::function<void(const StreamInformation &)> visitor)
|
||||
const = 0;
|
||||
|
||||
/**
|
||||
* Get the value of useVInts for the given column in this stripe.
|
||||
* Defaults to true.
|
||||
* @param streamId stream identifier
|
||||
*/
|
||||
virtual bool getUseVInts(const StreamIdentifier &streamId) const = 0;
|
||||
|
||||
/**
|
||||
* Get the memory pool for this reader.
|
||||
*/
|
||||
virtual memory::MemoryPool &getMemoryPool() const = 0;
|
||||
|
||||
/**
|
||||
* Get the RowGroupIndex.
|
||||
* @return a vector of RowIndex belonging to the stripe
|
||||
*/
|
||||
virtual unique_ptr<proto::RowIndex> getRowGroupIndex(const StreamIdentifier &si) const = 0;
|
||||
|
||||
/**
|
||||
* Get stride index provider which is used by string dictionary reader to
|
||||
* get the row index stride index where next() happens
|
||||
*/
|
||||
virtual const StrideIndexProvider &getStrideIndexProvider() const = 0;
|
||||
}
|
||||
ions &getRowReaderOptions() const = 0;
|
||||
|
||||
/**
|
||||
* Get the encoding for the given column for this stripe.
|
||||
*/
|
||||
virtual const proto::ColumnEncoding &getEncoding(const EncodingKey &) const = 0;
|
||||
|
||||
/**
|
||||
* Get the stream for the given column/kind in this stripe.
|
||||
* @param streamId stream identifier object
|
||||
* @param throwIfNotFound fail if a stream is required and not found
|
||||
* @return the new stream
|
||||
*/
|
||||
virtual unique_ptr<SeekableInputStream> getStream(const StreamIdentifier &si, bool throwIfNotFound) const = 0;
|
||||
|
||||
/**
|
||||
* visit all streams of given node and execute visitor logic
|
||||
* return number of streams visited
|
||||
*/
|
||||
virtual uint32_t visitStreamsOfNode(uint32_t node,
|
||||
std::function<void(const StreamInformation &)> visitor) const = 0;
|
||||
|
||||
/**
|
||||
* Get the value of useVInts for the given column in this stripe.
|
||||
* Defaults to true.
|
||||
* @param streamId stream identifier
|
||||
*/
|
||||
virtual bool getUseVInts(const StreamIdentifier &streamId) const = 0;
|
||||
|
||||
/**
|
||||
* Get the memory pool for this reader.
|
||||
*/
|
||||
virtual memory::MemoryPool &getMemoryPool() const = 0;
|
||||
|
||||
/**
|
||||
* Get the RowGroupIndex.
|
||||
* @return a vector of RowIndex belonging to the stripe
|
||||
*/
|
||||
virtual unique_ptr<proto::RowIndex> getRowGroupIndex(const StreamIdentifier &si) const = 0;
|
||||
|
||||
/**
|
||||
* Get stride index provider which is used by string dictionary reader to
|
||||
* get the row index stride index where next() happens
|
||||
*/
|
||||
virtual const StrideIndexProvider &getStrideIndexProvider() const = 0;
|
||||
};
|
||||
|
||||
class ColumnReader {
|
||||
|
||||
public:
|
||||
ColumnReader(const EncodingKey &ek, StripeStreams &stripe);
|
||||
|
||||
virtual ~ColumnReader() = default;
|
||||
|
||||
/**
|
||||
* Skip number of specified rows.
|
||||
* @param numValues the number of values to skip
|
||||
* @return the number of non-null values skipped
|
||||
*/
|
||||
virtual uint64_t skip(uint64_t numValues);
|
||||
|
||||
/**
|
||||
* Read the next group of values into a RowVector.
|
||||
* @param numValues the number of values to read
|
||||
* @param vector to read into
|
||||
*/
|
||||
virtual void next(uint64_t numValues, VectorPtr &result, const uint64_t *nulls = nullptr) = 0;
|
||||
};
|
||||
|
||||
class SelectiveColumnReader : public ColumnReader {
|
||||
public:
|
||||
static constexpr uint64_t kStringBufferSize = 16 * 1024;
|
||||
|
||||
SelectiveColumnReader(const EncodingKey &ek, StripeStreams &stripe, common::ScanSpec *scanSpec);
|
||||
|
||||
/**
|
||||
* Read the next group of values into a RowVector.
|
||||
* @param numValues the number of values to read
|
||||
* @param vector to read into
|
||||
*/
|
||||
void next(uint64_t /*numValues*/, VectorPtr & /*result*/, const uint64_t * /*incomingNulls*/) override {
|
||||
DATALIB_CHECK(false) << "next() is only defined in SelectiveStructColumnReader";
|
||||
}
|
||||
|
||||
// Creates a reader for the given stripe.
|
||||
static unique_ptr<SelectiveColumnReader> build(const std::shared_ptr<const dwio::common::TypeWithId> &requestedType,
|
||||
const std::shared_ptr<const dwio::common::TypeWithId> &dataType,
|
||||
StripeStreams &stripe, common::ScanSpec *scanSpec,
|
||||
uint32_t sequence = 0);
|
||||
|
||||
// Seeks to offset and reads the rows in 'rows' and applies
|
||||
// filters and value processing as given by 'scanSpec supplied at
|
||||
// construction. 'offset' is relative to start of stripe. 'rows' are
|
||||
// relative to 'offset', so that row 0 is the 'offset'th row from
|
||||
// start of stripe. 'rows' is expected to stay constant
|
||||
// between this and the next call to read.
|
||||
virtual void read(vector_size_t offset, RowSet rows, const uint64_t *incomingNulls) = 0;
|
||||
|
||||
// Extracts the values at 'rows' into '*result'. May rewrite or
|
||||
// reallocate '*result'. 'rows' must be the same set or a subset of
|
||||
// 'rows' passed to the last 'read().
|
||||
virtual void getValues(RowSet rows, VectorPtr *result) = 0;
|
||||
|
||||
// Returns the rows that were selected/visited by the last
|
||||
// read(). If 'this' has no filter, returns 'rows' passed to last
|
||||
// read().
|
||||
const RowSet outputRows() const {
|
||||
if (scanSpec_->hasFilter()) {
|
||||
return outputRows_;
|
||||
}
|
||||
return inputRows_;
|
||||
}
|
||||
|
||||
// Advances to 'offset', so that the next item to be read is the
|
||||
// offset-th from the start of stripe.
|
||||
void seekTo(vector_size_t offset, bool readsNullsOnly);
|
||||
|
||||
// The below functions are called from ColumnVisitor to fill the result set.
|
||||
inline void addOutputRow(vector_size_t row) {
|
||||
outputRows_.push_back(row);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline void addNull() {
|
||||
DATALIB_DCHECK(rawResultNulls_ && rawValues_ && (numValues_ + 1) * sizeof(T) < rawSize_);
|
||||
|
||||
anyNulls_ = true;
|
||||
bits::setBit(rawResultNulls_, numValues_);
|
||||
reinterpret_cast<T *>(rawValues_)[numValues_] = T();
|
||||
numValues_++;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline void addValue(const T value) {
|
||||
// @lint-ignore-every HOWTOEVEN ConstantArgumentPassByValue
|
||||
static_assert(std::is_pod<T>::value, "General case of addValue is only for primitive types");
|
||||
DATALIB_DCHECK(rawValues_ && (numValues _ + 1) * sizeof(T) < rawSize_);
|
||||
reinterpret_cast<T *>(rawValues_)[numValues_] = value;
|
||||
numValues_++;
|
||||
}
|
||||
|
||||
void dropResults(vector_size_t count) {
|
||||
outputRows_.resize(outputRows_.size() - count);
|
||||
numValues_ -= count;
|
||||
}
|
||||
|
||||
common::ScanSpec *scanSpec() const {
|
||||
return scanSpec_;
|
||||
}
|
||||
|
||||
auto readOffset() const {
|
||||
return readOffset_;
|
||||
}
|
||||
|
||||
void setReadOffset(vector_size_t readOffset) {
|
||||
readOffset_ = readOffset;
|
||||
}
|
||||
|
||||
protected:
|
||||
static constexpr int8_t kNoValueSize = -1;
|
||||
|
||||
template <typename T>
|
||||
void ensureValuesCapacity(vector_size_t numRows);
|
||||
|
||||
void prepareNulls(vector_size_t numRows, bool needNulls);
|
||||
|
||||
template <typename T>
|
||||
void filterNulls(RowSet rows, bool isNull, bool extractValues);
|
||||
|
||||
template <typename T>
|
||||
void prepareRead(vector_size_t offset, RowSet rows, const uint64_t *incomingNulls);
|
||||
|
||||
void setOutputRows(RowSet rows) {
|
||||
outputRows_.resize(rows.size());
|
||||
if (!rows.size()) {
|
||||
return;
|
||||
}
|
||||
memcpy(outputRows_.data(), &rows[0], rows.size() * sizeof(vector_size_t));
|
||||
}
|
||||
template <typename T, typename TVector>
|
||||
void getFlatValues(RowSet rows, VectorPtr *result);
|
||||
|
||||
template <typename T, typename TVector>
|
||||
void compactScalarValues(RowSet rows);
|
||||
|
||||
void addStringValue(folly::StringPiece value);
|
||||
|
||||
// Specification of filters, value extraction, pruning etc. The
|
||||
// spec is assigned at construction and the contents may change at
|
||||
// run time based on adaptation. Owned by caller.
|
||||
common::ScanSpec *const scanSpec_;
|
||||
// Row number after last read row, relative to stripe start.
|
||||
vector_size_t readOffset_ = 0;
|
||||
// The rows to process in read(). References memory supplied by
|
||||
// caller. The values must remain live until the next call to read().
|
||||
RowSet inputRows_;
|
||||
// Rows passing the filter in readWithVisitor. Must stay
|
||||
// constant between consecutive calls to read().
|
||||
vector<vector_size_t> outputRows_;
|
||||
// The row number corresponding to each element in 'values_'
|
||||
vector<vector_size_t> valueRows_;
|
||||
// The set of all nulls in the range of read(). Created when first
|
||||
// needed and then reused. Not returned to callers.
|
||||
BufferPtr nullsInReadRange_;
|
||||
// Nulls buffer for readWithVisitor. Not set if no nulls. 'numValues'
|
||||
// is the index of the first non-set bit.
|
||||
BufferPtr resultNulls_;
|
||||
uint64_t *rawResultNulls_ = nullptr;
|
||||
// Buffer for gathering scalar values in readWithVisitor.
|
||||
BufferPtr values_;
|
||||
// Writable content in 'values'
|
||||
void *rawValues_ = nullptr;
|
||||
vector_size_t numValues_ = 0;
|
||||
// Size of fixed width value in 'rawValues'. For integers, values
|
||||
// are read at 64 bit width and can be compacted or extracted at a
|
||||
// different width.
|
||||
int8_t valueSize_ = kNoValueSize;
|
||||
// Buffers backing the StringViews in 'values' when reading strings.
|
||||
vector<BufferPtr> stringBuffers_;
|
||||
// Writable contents of 'stringBuffers_.back()'.
|
||||
char *rawStringBuffer_ = nullptr;
|
||||
// Total writable bytes in 'rawStringBuffer_'.
|
||||
int32_t rawStringSize_ = 0;
|
||||
// Number of written bytes in 'rawStringBuffer_'.
|
||||
uint32_t rawStringUsed_ = 0;
|
||||
|
||||
// True if last read() added any nulls.
|
||||
bool anyNulls_ = false;
|
||||
// True if all values in scope for last read() are null.
|
||||
bool allNull_ = false;
|
||||
};
|
||||
|
||||
struct ExtractValues {
|
||||
static constexpr bool kSkipNulls = false;
|
||||
|
||||
bool acceptsNulls() const {
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename V>
|
||||
void addValue(vector_size_t /*rowIndex*/, V /*value*/) {
|
||||
}
|
||||
void addNull(vector_size_t /*rowIndex*/) {
|
||||
}
|
||||
};
|
||||
|
||||
class Filter {
|
||||
protected:
|
||||
Filter(bool deterministic, bool nullAllowed, FilterKind kind)
|
||||
: nullAllowed_(nullAllowed), deterministic_(deterministic), kind_(kind) {
|
||||
}
|
||||
|
||||
public:
|
||||
virtual ~Filter() = default;
|
||||
|
||||
// Templates parametrized on filter need to know determinism at compile
|
||||
// time. If this is false, deterministic() will be consulted at
|
||||
// runtime.
|
||||
static constexpr bool deterministic = true;
|
||||
|
||||
FilterKind kind() const {
|
||||
return kind_;
|
||||
}
|
||||
|
||||
virtual unique_ptr<Filter> clone() const = 0;
|
||||
|
||||
/**
|
||||
* A filter becomes non-deterministic when applies to nested column,
|
||||
* e.g. a[1] > 10 is non-deterministic because > 10 filter applies only to
|
||||
* some positions, e.g. first entry in a set of entries that correspond to a
|
||||
* single top-level position.
|
||||
*/
|
||||
virtual bool isDeterministic() const {
|
||||
return deterministic_;
|
||||
}
|
||||
|
||||
/**
|
||||
* When a filter applied to a nested column fails, the whole top-level
|
||||
* position should fail. To enable this functionality, the filter keeps track
|
||||
* of the boundaries of top-level positions and allows the caller to find out
|
||||
* where the current top-level position started and how far it continues.
|
||||
* @return number of positions from the start of the current top-level
|
||||
* position up to the current position (excluding current position)
|
||||
*/
|
||||
virtual int getPrecedingPositionsToFail() const {
|
||||
return 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return number of positions remaining until the end of the current
|
||||
* top-level position
|
||||
*/
|
||||
virtual int getSucceedingPositionsToFail() const {
|
||||
return 0;
|
||||
}
|
||||
|
||||
virtual bool testNull() const {
|
||||
return nullAllowed_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to apply is [not] null filters to complex types, e.g.
|
||||
* a[1] is null AND a[3] is not null, where a is an array(array(T)).
|
||||
*
|
||||
* In these case, the exact values are not known, but it is known whether they
|
||||
* are null or not. Furthermore, for some positions only nulls are allowed
|
||||
* (a[1] is null), for others only non-nulls (a[3] is not null), and for the
|
||||
* rest both are allowed (a[2] and a[N], where N > 3).
|
||||
*/
|
||||
virtual bool testNonNull() const {
|
||||
DWIO_RAISE("not supported");
|
||||
}
|
||||
|
||||
virtual bool testInt64(int64_t /* unused */) const {
|
||||
DWIO_RAISE("not supported");
|
||||
}
|
||||
|
||||
virtual bool testDouble(double /* unused */) const {
|
||||
DWIO_RAISE("not supported");
|
||||
}
|
||||
|
||||
virtual bool testFloat(float /* unused */) const {
|
||||
DWIO_RAISE("not supported");
|
||||
}
|
||||
|
||||
virtual bool testBool(bool /* unused */) const {
|
||||
DWIO_RAISE("not supported");
|
||||
}
|
||||
|
||||
virtual bool testBytes(const char * /* unused */, int32_t /* unused */) const {
|
||||
DWIO_RAISE("not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Filters like string equality and IN, as well as conditions on cardinality
|
||||
* of lists and maps can be at least partly decided by looking at lengths
|
||||
* alone. If this is false, then no further checks are needed. If true,
|
||||
* eventual filters on the data itself need to be evaluated.
|
||||
*/
|
||||
virtual bool testLength(int32_t /* unused */) const {
|
||||
DWIO_RAISE("not supported");
|
||||
}
|
||||
|
||||
protected:
|
||||
const bool nullAllowed_;
|
||||
|
||||
private:
|
||||
const bool deterministic_;
|
||||
const FilterKind kind_;
|
||||
};
|
||||
|
||||
// Template parameter for controlling filtering and action on a set of rows.
|
||||
template <typename T, typename TFilter, typename ExtractValues, bool isDense>
|
||||
class ColumnVisitor {
|
||||
public:
|
||||
using FilterType = TFilter;
|
||||
static constexpr bool dense = isDense;
|
||||
ColumnVisitor(TFilter &filter, SelectiveColumnReader *reader, const RowSet &rows, ExtractValues values)
|
||||
: filter_(filter), reader_(reader), allowNulls_(!TFilter::deterministic || filter.testNull()), rows_(&rows[0]),
|
||||
numRows_(rows.size()), rowIndex_(0), values_(values) {
|
||||
}
|
||||
|
||||
bool allowNulls() {
|
||||
if (ExtractValues::kSkipNulls && TFilter::deterministic) {
|
||||
return false;
|
||||
}
|
||||
return allowNulls_ && values_.acceptsNulls();
|
||||
}
|
||||
|
||||
vector_size_t start() {
|
||||
return isDense ? 0 : rowAt(0);
|
||||
}
|
||||
|
||||
// Tests for a null value and processes it. If the value is not
|
||||
// null, returns 0 and has no effect. If the value is null, advances
|
||||
// to the next non-null value in 'rows_'. Returns the number of
|
||||
// values (not including nulls) to skip to get to the next non-null.
|
||||
// If there is no next non-null in 'rows_', sets 'atEnd'. If 'atEnd'
|
||||
// is set and a non-zero skip is returned, the caller must perform
|
||||
// the skip before returning.
|
||||
FOLLY_ALWAYS_INLINE vector_size_t checkAndSkipNulls(const uint64_t *nulls, vector_size_t ¤t, bool &atEnd) {
|
||||
auto testRow = currentRow();
|
||||
// Check that the caller and the visitor are in sync about current row.
|
||||
DATALIB_DCHECK(current == testRow);
|
||||
uint32_t nullIndex = testRow >> 6;
|
||||
uint64_t nullWord = nulls[nullIndex];
|
||||
if (!nullWord) {
|
||||
return 0;
|
||||
}
|
||||
uint8_t nullBit = testRow & 63;
|
||||
if ((nullWord & (1UL << nullBit)) == 0) {
|
||||
return 0;
|
||||
}
|
||||
// We have a null. We find the next non-null.
|
||||
if (++rowIndex_ >= numRows_) {
|
||||
atEnd = true;
|
||||
return 0;
|
||||
}
|
||||
auto rowOfNullWord = testRow - nullBit;
|
||||
if (isDense) {
|
||||
if (nullBit == 63) {
|
||||
nullBit = 0;
|
||||
rowOfNullWord += 64;
|
||||
nullWord = nulls[++nullIndex];
|
||||
} else {
|
||||
++nullBit;
|
||||
// set all the bits below the row to null.
|
||||
nullWord |= f4d::bits::lowMask(nullBit);
|
||||
}
|
||||
for (;;) {
|
||||
auto nextNonNull = count_trailing_zeros(~nullWord);
|
||||
if (rowOfNullWord + nextNonNull >= numRows_) {
|
||||
// Nulls all the way to the end.
|
||||
atEnd = true;
|
||||
return 0;
|
||||
}
|
||||
if (nextNonNull < 64) {
|
||||
DATALIB_CHECK(rowIndex_ <= rowOfNullWord + nextNonNull);
|
||||
rowIndex_ = rowOfNullWord + nextNonNull;
|
||||
current = currentRow();
|
||||
return 0;
|
||||
}
|
||||
rowOfNullWord += 64;
|
||||
nullWord = nulls[++nullIndex];
|
||||
}
|
||||
} else {
|
||||
// Sparse row numbers. We find the first non-null and count
|
||||
// how many non-nulls on rows not in 'rows_' we skipped.
|
||||
int32_t toSkip = 0;
|
||||
nullWord |= f4d::bits::lowMask(nullBit);
|
||||
for (;;) {
|
||||
testRow = currentRow();
|
||||
while (testRow >= rowOfNullWord + 64) {
|
||||
toSkip += __builtin_popcountll(~nullWord);
|
||||
nullWord = nulls[++nullIndex];
|
||||
rowOfNullWord += 64;
|
||||
}
|
||||
// testRow is inside nullWord. See if non-null.
|
||||
nullBit = testRow & 63;
|
||||
if ((nullWord & (1UL << nullBit)) == 0) {
|
||||
toSkip += __builtin_popcountll(~nullWord & f4d::bits::lowMask(nullBit));
|
||||
current = testRow;
|
||||
return toSkip;
|
||||
}
|
||||
if (++rowIndex_ >= numRows_) {
|
||||
// We end with a null. Add the non-nulls below the final null.
|
||||
toSkip += __builtin_popcountll(~nullWord & f4d::bits::lowMask(testRow - rowOfNullWord));
|
||||
atEnd = true;
|
||||
return toSkip;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
vector_size_t processNull(bool &atEnd) {
|
||||
vector_size_t previous = currentRow();
|
||||
if (filter_.testNull()) {
|
||||
filterPassedForNull();
|
||||
} else {
|
||||
filterFailed();
|
||||
}
|
||||
if (++rowIndex_ >= numRows_) {
|
||||
atEnd = true;
|
||||
return rows_[numRows_ - 1] - previous;
|
||||
}
|
||||
if (TFilter::deterministic && isDense) {
|
||||
return 0;
|
||||
}
|
||||
return currentRow() - previous - 1;
|
||||
}
|
||||
|
||||
FOLLY_ALWAYS_INLINE vector_size_t process(T value, bool &atEnd) {
|
||||
if (!TFilter::deterministic) {
|
||||
auto previous = currentRow();
|
||||
if (common::applyFilter(filter_, value)) {
|
||||
filterPassed(value);
|
||||
} else {
|
||||
filterFailed();
|
||||
}
|
||||
if (++rowIndex_ >= numRows_) {
|
||||
atEnd = true;
|
||||
return rows_[numRows_ - 1] - previous;
|
||||
}
|
||||
return currentRow() - previous - 1;
|
||||
}
|
||||
// The filter passes or fails and we go to the next row if any.
|
||||
if (common::applyFilter(filter_, value)) {
|
||||
filterPassed(value);
|
||||
} else {
|
||||
filterFailed();
|
||||
}
|
||||
if (++rowIndex_ >= numRows_) {
|
||||
atEnd = true;
|
||||
return 0;
|
||||
}
|
||||
if (isDense) {
|
||||
return 0;
|
||||
}
|
||||
return currentRow() - rows_[rowIndex_ - 1] - 1;
|
||||
}
|
||||
|
||||
inline vector_size_t rowAt(vector_size_t index) {
|
||||
if (isDense) {
|
||||
return index;
|
||||
}
|
||||
return rows_[index];
|
||||
}
|
||||
|
||||
vector_size_t currentRow() {
|
||||
if (isDense) {
|
||||
return rowIndex_;
|
||||
}
|
||||
return rows_[rowIndex_];
|
||||
}
|
||||
|
||||
vector_size_t numRows() {
|
||||
return numRows_;
|
||||
}
|
||||
|
||||
void filterPassed(T value) {
|
||||
addResult(value);
|
||||
if (!std::is_same<TFilter, common::AlwaysTrue>::value) {
|
||||
addOutputRow(currentRow());
|
||||
}
|
||||
}
|
||||
|
||||
inline void filterPassedForNull() {
|
||||
addNull();
|
||||
if (!std::is_same<TFilter, common::AlwaysTrue>::value) {
|
||||
addOutputRow(currentRow());
|
||||
}
|
||||
}
|
||||
|
||||
FOLLY_ALWAYS_INLINE void filterFailed();
|
||||
inline void addResult(T value);
|
||||
inline void addNull();
|
||||
inline void addOutputRow(vector_size_t row);
|
||||
|
||||
protected:
|
||||
TFilter &filter_;
|
||||
SelectiveColumnReader *reader_;
|
||||
const bool allowNulls_;
|
||||
const vector_size_t *rows_;
|
||||
vector_size_t numRows_;
|
||||
vector_size_t rowIndex_;
|
||||
ExtractValues values_;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
44
external/duckdb/extension/parquet/include/parquet_timestamp.hpp
vendored
Normal file
44
external/duckdb/extension/parquet/include/parquet_timestamp.hpp
vendored
Normal file
@@ -0,0 +1,44 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_timestamp.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
struct Int96 {
|
||||
uint32_t value[3];
|
||||
};
|
||||
|
||||
timestamp_t ImpalaTimestampToTimestamp(const Int96 &raw_ts);
|
||||
timestamp_ns_t ImpalaTimestampToTimestampNS(const Int96 &raw_ts);
|
||||
Int96 TimestampToImpalaTimestamp(timestamp_t &ts);
|
||||
|
||||
timestamp_t ParquetTimestampMicrosToTimestamp(const int64_t &raw_ts);
|
||||
timestamp_t ParquetTimestampMsToTimestamp(const int64_t &raw_ts);
|
||||
timestamp_t ParquetTimestampNsToTimestamp(const int64_t &raw_ts);
|
||||
|
||||
timestamp_ns_t ParquetTimestampMsToTimestampNs(const int64_t &raw_ms);
|
||||
timestamp_ns_t ParquetTimestampUsToTimestampNs(const int64_t &raw_us);
|
||||
timestamp_ns_t ParquetTimestampNsToTimestampNs(const int64_t &raw_ns);
|
||||
|
||||
date_t ParquetIntToDate(const int32_t &raw_date);
|
||||
dtime_t ParquetMsIntToTime(const int32_t &raw_millis);
|
||||
dtime_t ParquetIntToTime(const int64_t &raw_micros);
|
||||
dtime_t ParquetNsIntToTime(const int64_t &raw_nanos);
|
||||
|
||||
dtime_ns_t ParquetMsIntToTimeNs(const int32_t &raw_millis);
|
||||
dtime_ns_t ParquetUsIntToTimeNs(const int64_t &raw_micros);
|
||||
dtime_ns_t ParquetIntToTimeNs(const int64_t &raw_nanos);
|
||||
|
||||
dtime_tz_t ParquetIntToTimeMsTZ(const int32_t &raw_millis);
|
||||
dtime_tz_t ParquetIntToTimeTZ(const int64_t &raw_micros);
|
||||
dtime_tz_t ParquetIntToTimeNsTZ(const int64_t &raw_nanos);
|
||||
|
||||
} // namespace duckdb
|
||||
182
external/duckdb/extension/parquet/include/parquet_writer.hpp
vendored
Normal file
182
external/duckdb/extension/parquet/include/parquet_writer.hpp
vendored
Normal file
@@ -0,0 +1,182 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// parquet_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "duckdb/common/common.hpp"
|
||||
#include "duckdb/common/optional_idx.hpp"
|
||||
#include "duckdb/common/encryption_state.hpp"
|
||||
#include "duckdb/common/exception.hpp"
|
||||
#include "duckdb/common/mutex.hpp"
|
||||
#include "duckdb/common/atomic.hpp"
|
||||
#include "duckdb/common/serializer/buffered_file_writer.hpp"
|
||||
#include "duckdb/common/types/column/column_data_collection.hpp"
|
||||
#include "duckdb/function/copy_function.hpp"
|
||||
|
||||
#include "parquet_statistics.hpp"
|
||||
#include "column_writer.hpp"
|
||||
#include "parquet_field_id.hpp"
|
||||
#include "parquet_shredding.hpp"
|
||||
#include "parquet_types.h"
|
||||
#include "geo_parquet.hpp"
|
||||
#include "writer/parquet_write_stats.hpp"
|
||||
#include "thrift/protocol/TCompactProtocol.h"
|
||||
|
||||
namespace duckdb {
|
||||
class FileSystem;
|
||||
class FileOpener;
|
||||
class ParquetEncryptionConfig;
|
||||
class ParquetStatsAccumulator;
|
||||
|
||||
class Serializer;
|
||||
class Deserializer;
|
||||
|
||||
class ColumnWriterStatistics;
|
||||
struct CopyFunctionFileStatistics;
|
||||
|
||||
struct PreparedRowGroup {
|
||||
duckdb_parquet::RowGroup row_group;
|
||||
vector<unique_ptr<ColumnWriterState>> states;
|
||||
};
|
||||
|
||||
struct ParquetBloomFilterEntry {
|
||||
unique_ptr<ParquetBloomFilter> bloom_filter;
|
||||
idx_t row_group_idx;
|
||||
idx_t column_idx;
|
||||
};
|
||||
|
||||
enum class ParquetVersion : uint8_t {
|
||||
V1 = 1, //! Excludes DELTA_BINARY_PACKED, DELTA_LENGTH_BYTE_ARRAY, BYTE_STREAM_SPLIT
|
||||
V2 = 2, //! Includes the encodings above
|
||||
};
|
||||
|
||||
class ParquetWriter {
|
||||
public:
|
||||
ParquetWriter(ClientContext &context, FileSystem &fs, string file_name, vector<LogicalType> types,
|
||||
vector<string> names, duckdb_parquet::CompressionCodec::type codec, ChildFieldIDs field_ids,
|
||||
ShreddingType shredding_types, const vector<pair<string, string>> &kv_metadata,
|
||||
shared_ptr<ParquetEncryptionConfig> encryption_config, optional_idx dictionary_size_limit,
|
||||
idx_t string_dictionary_page_size_limit, bool enable_bloom_filters,
|
||||
double bloom_filter_false_positive_ratio, int64_t compression_level, bool debug_use_openssl,
|
||||
ParquetVersion parquet_version, GeoParquetVersion geoparquet_version);
|
||||
~ParquetWriter();
|
||||
|
||||
public:
|
||||
void PrepareRowGroup(ColumnDataCollection &buffer, PreparedRowGroup &result);
|
||||
void FlushRowGroup(PreparedRowGroup &row_group);
|
||||
void Flush(ColumnDataCollection &buffer);
|
||||
void Finalize();
|
||||
|
||||
static duckdb_parquet::Type::type DuckDBTypeToParquetType(const LogicalType &duckdb_type);
|
||||
static void SetSchemaProperties(const LogicalType &duckdb_type, duckdb_parquet::SchemaElement &schema_ele,
|
||||
bool allow_geometry);
|
||||
|
||||
ClientContext &GetContext() {
|
||||
return context;
|
||||
}
|
||||
duckdb_apache::thrift::protocol::TProtocol *GetProtocol() {
|
||||
return protocol.get();
|
||||
}
|
||||
duckdb_parquet::CompressionCodec::type GetCodec() {
|
||||
return codec;
|
||||
}
|
||||
duckdb_parquet::Type::type GetType(idx_t schema_idx) {
|
||||
return file_meta_data.schema[schema_idx].type;
|
||||
}
|
||||
LogicalType GetSQLType(idx_t schema_idx) const {
|
||||
return sql_types[schema_idx];
|
||||
}
|
||||
BufferedFileWriter &GetWriter() {
|
||||
return *writer;
|
||||
}
|
||||
idx_t FileSize() {
|
||||
return total_written;
|
||||
}
|
||||
optional_idx DictionarySizeLimit() const {
|
||||
return dictionary_size_limit;
|
||||
}
|
||||
idx_t StringDictionaryPageSizeLimit() const {
|
||||
return string_dictionary_page_size_limit;
|
||||
}
|
||||
double EnableBloomFilters() const {
|
||||
return enable_bloom_filters;
|
||||
}
|
||||
double BloomFilterFalsePositiveRatio() const {
|
||||
return bloom_filter_false_positive_ratio;
|
||||
}
|
||||
int64_t CompressionLevel() const {
|
||||
return compression_level;
|
||||
}
|
||||
idx_t NumberOfRowGroups() {
|
||||
return num_row_groups;
|
||||
}
|
||||
ParquetVersion GetParquetVersion() const {
|
||||
return parquet_version;
|
||||
}
|
||||
GeoParquetVersion GetGeoParquetVersion() const {
|
||||
return geoparquet_version;
|
||||
}
|
||||
const string &GetFileName() const {
|
||||
return file_name;
|
||||
}
|
||||
|
||||
uint32_t Write(const duckdb_apache::thrift::TBase &object);
|
||||
uint32_t WriteData(const const_data_ptr_t buffer, const uint32_t buffer_size);
|
||||
|
||||
GeoParquetFileMetadata &GetGeoParquetData();
|
||||
|
||||
static bool TryGetParquetType(const LogicalType &duckdb_type,
|
||||
optional_ptr<duckdb_parquet::Type::type> type = nullptr);
|
||||
|
||||
void BufferBloomFilter(idx_t col_idx, unique_ptr<ParquetBloomFilter> bloom_filter);
|
||||
void SetWrittenStatistics(CopyFunctionFileStatistics &written_stats);
|
||||
void FlushColumnStats(idx_t col_idx, duckdb_parquet::ColumnChunk &chunk,
|
||||
optional_ptr<ColumnWriterStatistics> writer_stats);
|
||||
|
||||
private:
|
||||
void GatherWrittenStatistics();
|
||||
|
||||
private:
|
||||
ClientContext &context;
|
||||
string file_name;
|
||||
vector<LogicalType> sql_types;
|
||||
vector<string> column_names;
|
||||
duckdb_parquet::CompressionCodec::type codec;
|
||||
ChildFieldIDs field_ids;
|
||||
ShreddingType shredding_types;
|
||||
shared_ptr<ParquetEncryptionConfig> encryption_config;
|
||||
optional_idx dictionary_size_limit;
|
||||
idx_t string_dictionary_page_size_limit;
|
||||
bool enable_bloom_filters;
|
||||
double bloom_filter_false_positive_ratio;
|
||||
int64_t compression_level;
|
||||
bool debug_use_openssl;
|
||||
shared_ptr<EncryptionUtil> encryption_util;
|
||||
ParquetVersion parquet_version;
|
||||
GeoParquetVersion geoparquet_version;
|
||||
vector<ParquetColumnSchema> column_schemas;
|
||||
|
||||
unique_ptr<BufferedFileWriter> writer;
|
||||
//! Atomics to reduce contention when rotating writes to multiple Parquet files
|
||||
atomic<idx_t> total_written;
|
||||
atomic<idx_t> num_row_groups;
|
||||
std::shared_ptr<duckdb_apache::thrift::protocol::TProtocol> protocol;
|
||||
duckdb_parquet::FileMetaData file_meta_data;
|
||||
std::mutex lock;
|
||||
|
||||
vector<unique_ptr<ColumnWriter>> column_writers;
|
||||
|
||||
unique_ptr<GeoParquetFileMetadata> geoparquet_data;
|
||||
vector<ParquetBloomFilterEntry> bloom_filters;
|
||||
|
||||
optional_ptr<CopyFunctionFileStatistics> written_stats;
|
||||
unique_ptr<ParquetStatsAccumulator> stats_accumulator;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
70
external/duckdb/extension/parquet/include/reader/boolean_column_reader.hpp
vendored
Normal file
70
external/duckdb/extension/parquet/include/reader/boolean_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,70 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/boolean_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
struct BooleanParquetValueConversion;
|
||||
|
||||
class BooleanColumnReader : public TemplatedColumnReader<bool, BooleanParquetValueConversion> {
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::BOOL;
|
||||
|
||||
public:
|
||||
BooleanColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema)
|
||||
: TemplatedColumnReader<bool, BooleanParquetValueConversion>(reader, schema), byte_pos(0) {
|
||||
}
|
||||
|
||||
uint8_t byte_pos;
|
||||
|
||||
void InitializeRead(idx_t row_group_idx_p, const vector<ColumnChunk> &columns, TProtocol &protocol_p) override {
|
||||
byte_pos = 0;
|
||||
TemplatedColumnReader<bool, BooleanParquetValueConversion>::InitializeRead(row_group_idx_p, columns,
|
||||
protocol_p);
|
||||
}
|
||||
|
||||
void ResetPage() override {
|
||||
byte_pos = 0;
|
||||
}
|
||||
};
|
||||
|
||||
struct BooleanParquetValueConversion {
|
||||
template <bool CHECKED>
|
||||
static bool PlainRead(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
auto &byte_pos = reader.Cast<BooleanColumnReader>().byte_pos;
|
||||
bool ret = (*plain_data.ptr >> byte_pos) & 1;
|
||||
if (++byte_pos == 8) {
|
||||
byte_pos = 0;
|
||||
if (CHECKED) {
|
||||
plain_data.inc(1);
|
||||
} else {
|
||||
plain_data.unsafe_inc(1);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <bool CHECKED>
|
||||
static void PlainSkip(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
PlainRead<CHECKED>(plain_data, reader);
|
||||
}
|
||||
|
||||
static bool PlainAvailable(const ByteBuffer &plain_data, const idx_t count) {
|
||||
return plain_data.check_available((count + 7) / 8);
|
||||
}
|
||||
|
||||
static idx_t PlainConstantSize() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
46
external/duckdb/extension/parquet/include/reader/callback_column_reader.hpp
vendored
Normal file
46
external/duckdb/extension/parquet/include/reader/callback_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,46 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/callback_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
#include "parquet_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
template <class PARQUET_PHYSICAL_TYPE, class DUCKDB_PHYSICAL_TYPE,
|
||||
DUCKDB_PHYSICAL_TYPE (*FUNC)(const PARQUET_PHYSICAL_TYPE &input)>
|
||||
class CallbackColumnReader
|
||||
: public TemplatedColumnReader<DUCKDB_PHYSICAL_TYPE,
|
||||
CallbackParquetValueConversion<PARQUET_PHYSICAL_TYPE, DUCKDB_PHYSICAL_TYPE, FUNC>> {
|
||||
using BaseType =
|
||||
TemplatedColumnReader<DUCKDB_PHYSICAL_TYPE,
|
||||
CallbackParquetValueConversion<PARQUET_PHYSICAL_TYPE, DUCKDB_PHYSICAL_TYPE, FUNC>>;
|
||||
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::INVALID;
|
||||
|
||||
public:
|
||||
CallbackColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema)
|
||||
: TemplatedColumnReader<DUCKDB_PHYSICAL_TYPE,
|
||||
CallbackParquetValueConversion<PARQUET_PHYSICAL_TYPE, DUCKDB_PHYSICAL_TYPE, FUNC>>(
|
||||
reader, schema) {
|
||||
}
|
||||
|
||||
protected:
|
||||
void Dictionary(shared_ptr<ResizeableBuffer> dictionary_data, idx_t num_entries) {
|
||||
BaseType::AllocateDict(num_entries * sizeof(DUCKDB_PHYSICAL_TYPE));
|
||||
auto dict_ptr = (DUCKDB_PHYSICAL_TYPE *)this->dict->ptr;
|
||||
for (idx_t i = 0; i < num_entries; i++) {
|
||||
dict_ptr[i] = FUNC(dictionary_data->read<PARQUET_PHYSICAL_TYPE>());
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
65
external/duckdb/extension/parquet/include/reader/decimal_column_reader.hpp
vendored
Normal file
65
external/duckdb/extension/parquet/include/reader/decimal_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,65 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/decimal_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
#include "parquet_reader.hpp"
|
||||
#include "parquet_decimal_utils.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
template <class DUCKDB_PHYSICAL_TYPE, bool FIXED_LENGTH>
|
||||
struct DecimalParquetValueConversion {
|
||||
template <bool CHECKED>
|
||||
static DUCKDB_PHYSICAL_TYPE PlainRead(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
idx_t byte_len;
|
||||
if (FIXED_LENGTH) {
|
||||
byte_len = reader.Schema().type_length;
|
||||
} else {
|
||||
byte_len = plain_data.read<uint32_t>();
|
||||
}
|
||||
plain_data.available(byte_len);
|
||||
auto res = ParquetDecimalUtils::ReadDecimalValue<DUCKDB_PHYSICAL_TYPE>(const_data_ptr_cast(plain_data.ptr),
|
||||
byte_len, reader.Schema());
|
||||
|
||||
plain_data.inc(byte_len);
|
||||
return res;
|
||||
}
|
||||
|
||||
template <bool CHECKED>
|
||||
static void PlainSkip(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
uint32_t decimal_len = FIXED_LENGTH ? reader.Schema().type_length : plain_data.read<uint32_t>();
|
||||
plain_data.inc(decimal_len);
|
||||
}
|
||||
|
||||
static bool PlainAvailable(const ByteBuffer &plain_data, const idx_t count) {
|
||||
return true;
|
||||
}
|
||||
|
||||
static idx_t PlainConstantSize() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
template <class DUCKDB_PHYSICAL_TYPE, bool FIXED_LENGTH>
|
||||
class DecimalColumnReader
|
||||
: public TemplatedColumnReader<DUCKDB_PHYSICAL_TYPE,
|
||||
DecimalParquetValueConversion<DUCKDB_PHYSICAL_TYPE, FIXED_LENGTH>> {
|
||||
using BaseType =
|
||||
TemplatedColumnReader<DUCKDB_PHYSICAL_TYPE, DecimalParquetValueConversion<DUCKDB_PHYSICAL_TYPE, FIXED_LENGTH>>;
|
||||
|
||||
public:
|
||||
DecimalColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema)
|
||||
: TemplatedColumnReader<DUCKDB_PHYSICAL_TYPE,
|
||||
DecimalParquetValueConversion<DUCKDB_PHYSICAL_TYPE, FIXED_LENGTH>>(reader, schema) {
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
56
external/duckdb/extension/parquet/include/reader/expression_column_reader.hpp
vendored
Normal file
56
external/duckdb/extension/parquet/include/reader/expression_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,56 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/expression_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "duckdb/execution/expression_executor.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
//! A column reader that executes an expression over a child reader
|
||||
class ExpressionColumnReader : public ColumnReader {
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::INVALID;
|
||||
|
||||
public:
|
||||
ExpressionColumnReader(ClientContext &context, unique_ptr<ColumnReader> child_reader, unique_ptr<Expression> expr,
|
||||
const ParquetColumnSchema &schema);
|
||||
ExpressionColumnReader(ClientContext &context, unique_ptr<ColumnReader> child_reader, unique_ptr<Expression> expr,
|
||||
unique_ptr<ParquetColumnSchema> owned_schema);
|
||||
|
||||
unique_ptr<ColumnReader> child_reader;
|
||||
DataChunk intermediate_chunk;
|
||||
unique_ptr<Expression> expr;
|
||||
ExpressionExecutor executor;
|
||||
|
||||
// If this reader was created on top of a child reader, after-the-fact, the schema needs to live somewhere
|
||||
unique_ptr<ParquetColumnSchema> owned_schema;
|
||||
|
||||
public:
|
||||
void InitializeRead(idx_t row_group_idx_p, const vector<ColumnChunk> &columns, TProtocol &protocol_p) override;
|
||||
|
||||
idx_t Read(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result) override;
|
||||
|
||||
void Skip(idx_t num_values) override;
|
||||
idx_t GroupRowsAvailable() override;
|
||||
|
||||
uint64_t TotalCompressedSize() override {
|
||||
return child_reader->TotalCompressedSize();
|
||||
}
|
||||
|
||||
idx_t FileOffset() const override {
|
||||
return child_reader->FileOffset();
|
||||
}
|
||||
|
||||
void RegisterPrefetch(ThriftFileTransport &transport, bool allow_merge) override {
|
||||
child_reader->RegisterPrefetch(transport, allow_merge);
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
67
external/duckdb/extension/parquet/include/reader/interval_column_reader.hpp
vendored
Normal file
67
external/duckdb/extension/parquet/include/reader/interval_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,67 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/interval_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
#include "parquet_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
//===--------------------------------------------------------------------===//
|
||||
// Interval Column Reader
|
||||
//===--------------------------------------------------------------------===//
|
||||
struct IntervalValueConversion {
|
||||
static constexpr const idx_t PARQUET_INTERVAL_SIZE = 12;
|
||||
|
||||
static interval_t ReadParquetInterval(const_data_ptr_t input) {
|
||||
interval_t result;
|
||||
result.months = Load<int32_t>(input);
|
||||
result.days = Load<int32_t>(input + sizeof(uint32_t));
|
||||
result.micros = int64_t(Load<uint32_t>(input + sizeof(uint32_t) * 2)) * 1000;
|
||||
return result;
|
||||
}
|
||||
|
||||
template <bool CHECKED>
|
||||
static interval_t PlainRead(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
if (CHECKED) {
|
||||
plain_data.available(PARQUET_INTERVAL_SIZE);
|
||||
}
|
||||
auto res = ReadParquetInterval(const_data_ptr_cast(plain_data.ptr));
|
||||
plain_data.unsafe_inc(PARQUET_INTERVAL_SIZE);
|
||||
return res;
|
||||
}
|
||||
|
||||
template <bool CHECKED>
|
||||
static void PlainSkip(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
if (CHECKED) {
|
||||
plain_data.inc(PARQUET_INTERVAL_SIZE);
|
||||
} else {
|
||||
plain_data.unsafe_inc(PARQUET_INTERVAL_SIZE);
|
||||
}
|
||||
}
|
||||
|
||||
static bool PlainAvailable(const ByteBuffer &plain_data, const idx_t count) {
|
||||
return plain_data.check_available(count * PARQUET_INTERVAL_SIZE);
|
||||
}
|
||||
|
||||
static idx_t PlainConstantSize() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
class IntervalColumnReader : public TemplatedColumnReader<interval_t, IntervalValueConversion> {
|
||||
|
||||
public:
|
||||
IntervalColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema)
|
||||
: TemplatedColumnReader<interval_t, IntervalValueConversion>(reader, schema) {
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
62
external/duckdb/extension/parquet/include/reader/list_column_reader.hpp
vendored
Normal file
62
external/duckdb/extension/parquet/include/reader/list_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,62 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/list_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ListColumnReader : public ColumnReader {
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::LIST;
|
||||
|
||||
public:
|
||||
ListColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema,
|
||||
unique_ptr<ColumnReader> child_column_reader_p);
|
||||
|
||||
idx_t Read(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result_out) override;
|
||||
|
||||
void ApplyPendingSkips(data_ptr_t define_out, data_ptr_t repeat_out) override;
|
||||
|
||||
void InitializeRead(idx_t row_group_idx_p, const vector<ColumnChunk> &columns, TProtocol &protocol_p) override {
|
||||
child_column_reader->InitializeRead(row_group_idx_p, columns, protocol_p);
|
||||
}
|
||||
|
||||
idx_t GroupRowsAvailable() override {
|
||||
return child_column_reader->GroupRowsAvailable() + overflow_child_count;
|
||||
}
|
||||
|
||||
uint64_t TotalCompressedSize() override {
|
||||
return child_column_reader->TotalCompressedSize();
|
||||
}
|
||||
|
||||
void RegisterPrefetch(ThriftFileTransport &transport, bool allow_merge) override {
|
||||
child_column_reader->RegisterPrefetch(transport, allow_merge);
|
||||
}
|
||||
|
||||
protected:
|
||||
template <class OP>
|
||||
idx_t ReadInternal(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out,
|
||||
optional_ptr<Vector> result_out);
|
||||
|
||||
private:
|
||||
unique_ptr<ColumnReader> child_column_reader;
|
||||
ResizeableBuffer child_defines;
|
||||
ResizeableBuffer child_repeats;
|
||||
uint8_t *child_defines_ptr;
|
||||
uint8_t *child_repeats_ptr;
|
||||
|
||||
VectorCache read_cache;
|
||||
Vector read_vector;
|
||||
|
||||
idx_t overflow_child_count;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
38
external/duckdb/extension/parquet/include/reader/null_column_reader.hpp
vendored
Normal file
38
external/duckdb/extension/parquet/include/reader/null_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,38 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/null_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "duckdb/common/helper.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class NullColumnReader : public ColumnReader {
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::INVALID;
|
||||
|
||||
public:
|
||||
NullColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema) : ColumnReader(reader, schema) {};
|
||||
|
||||
shared_ptr<ResizeableBuffer> dict;
|
||||
|
||||
public:
|
||||
void Plain(ByteBuffer &plain_data, uint8_t *defines, uint64_t num_values, idx_t result_offset,
|
||||
Vector &result) override {
|
||||
(void)defines;
|
||||
(void)plain_data;
|
||||
|
||||
auto &result_mask = FlatVector::Validity(result);
|
||||
for (idx_t row_idx = 0; row_idx < num_values; row_idx++) {
|
||||
result_mask.SetInvalid(row_idx + result_offset);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
52
external/duckdb/extension/parquet/include/reader/row_number_column_reader.hpp
vendored
Normal file
52
external/duckdb/extension/parquet/include/reader/row_number_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,52 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/row_number_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb/common/limits.hpp"
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
//! Reads a file-absolute row number as a virtual column that's not actually stored in the file
|
||||
class RowNumberColumnReader : public ColumnReader {
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::INT64;
|
||||
|
||||
public:
|
||||
RowNumberColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema);
|
||||
|
||||
public:
|
||||
idx_t Read(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result) override;
|
||||
void Filter(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result_out,
|
||||
const TableFilter &filter, TableFilterState &filter_state, SelectionVector &sel,
|
||||
idx_t &approved_tuple_count, bool is_first_filter) override;
|
||||
|
||||
void InitializeRead(idx_t row_group_idx_p, const vector<ColumnChunk> &columns, TProtocol &protocol_p) override;
|
||||
|
||||
void Skip(idx_t num_values) override {
|
||||
row_group_offset += num_values;
|
||||
}
|
||||
idx_t GroupRowsAvailable() override {
|
||||
return NumericLimits<idx_t>::Maximum();
|
||||
};
|
||||
uint64_t TotalCompressedSize() override {
|
||||
return 0;
|
||||
}
|
||||
idx_t FileOffset() const override {
|
||||
return 0;
|
||||
}
|
||||
void RegisterPrefetch(ThriftFileTransport &transport, bool allow_merge) override {
|
||||
}
|
||||
|
||||
private:
|
||||
idx_t row_group_offset;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
91
external/duckdb/extension/parquet/include/reader/string_column_reader.hpp
vendored
Normal file
91
external/duckdb/extension/parquet/include/reader/string_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,91 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/string_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class StringColumnReader : public ColumnReader {
|
||||
enum class StringColumnType : uint8_t { VARCHAR, JSON, OTHER };
|
||||
|
||||
static StringColumnType GetStringColumnType(const LogicalType &type) {
|
||||
if (type.IsJSONType()) {
|
||||
return StringColumnType::JSON;
|
||||
}
|
||||
if (type.id() == LogicalTypeId::VARCHAR) {
|
||||
return StringColumnType::VARCHAR;
|
||||
}
|
||||
return StringColumnType::OTHER;
|
||||
}
|
||||
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::VARCHAR;
|
||||
|
||||
public:
|
||||
StringColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema);
|
||||
idx_t fixed_width_string_length;
|
||||
const StringColumnType string_column_type;
|
||||
|
||||
public:
|
||||
static void VerifyString(const char *str_data, uint32_t str_len, const bool isVarchar);
|
||||
void VerifyString(const char *str_data, uint32_t str_len);
|
||||
|
||||
static void ReferenceBlock(Vector &result, shared_ptr<ResizeableBuffer> &block);
|
||||
|
||||
protected:
|
||||
void Plain(ByteBuffer &plain_data, uint8_t *defines, idx_t num_values, idx_t result_offset,
|
||||
Vector &result) override {
|
||||
throw NotImplementedException("StringColumnReader can only read plain data from a shared buffer");
|
||||
}
|
||||
void Plain(shared_ptr<ResizeableBuffer> &plain_data, uint8_t *defines, idx_t num_values, idx_t result_offset,
|
||||
Vector &result) override;
|
||||
void PlainSkip(ByteBuffer &plain_data, uint8_t *defines, idx_t num_values) override;
|
||||
void PlainSelect(shared_ptr<ResizeableBuffer> &plain_data, uint8_t *defines, idx_t num_values, Vector &result,
|
||||
const SelectionVector &sel, idx_t count) override;
|
||||
|
||||
bool SupportsDirectFilter() const override {
|
||||
return true;
|
||||
}
|
||||
bool SupportsDirectSelect() const override {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
struct StringParquetValueConversion {
|
||||
template <bool CHECKED>
|
||||
static string_t PlainRead(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
auto &scr = reader.Cast<StringColumnReader>();
|
||||
uint32_t str_len =
|
||||
scr.fixed_width_string_length == 0 ? plain_data.read<uint32_t>() : scr.fixed_width_string_length;
|
||||
plain_data.available(str_len);
|
||||
auto plain_str = char_ptr_cast(plain_data.ptr);
|
||||
scr.VerifyString(plain_str, str_len);
|
||||
auto ret_str = string_t(plain_str, str_len);
|
||||
plain_data.inc(str_len);
|
||||
return ret_str;
|
||||
}
|
||||
template <bool CHECKED>
|
||||
static void PlainSkip(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
auto &scr = reader.Cast<StringColumnReader>();
|
||||
uint32_t str_len =
|
||||
scr.fixed_width_string_length == 0 ? plain_data.read<uint32_t>() : scr.fixed_width_string_length;
|
||||
plain_data.inc(str_len);
|
||||
}
|
||||
static bool PlainAvailable(const ByteBuffer &plain_data, const idx_t count) {
|
||||
return false;
|
||||
}
|
||||
|
||||
static idx_t PlainConstantSize() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
39
external/duckdb/extension/parquet/include/reader/struct_column_reader.hpp
vendored
Normal file
39
external/duckdb/extension/parquet/include/reader/struct_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,39 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/struct_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class StructColumnReader : public ColumnReader {
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::STRUCT;
|
||||
|
||||
public:
|
||||
StructColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema,
|
||||
vector<unique_ptr<ColumnReader>> child_readers_p);
|
||||
|
||||
vector<unique_ptr<ColumnReader>> child_readers;
|
||||
|
||||
public:
|
||||
ColumnReader &GetChildReader(idx_t child_idx);
|
||||
|
||||
void InitializeRead(idx_t row_group_idx_p, const vector<ColumnChunk> &columns, TProtocol &protocol_p) override;
|
||||
|
||||
idx_t Read(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result) override;
|
||||
|
||||
void Skip(idx_t num_values) override;
|
||||
idx_t GroupRowsAvailable() override;
|
||||
uint64_t TotalCompressedSize() override;
|
||||
void RegisterPrefetch(ThriftFileTransport &transport, bool allow_merge) override;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
110
external/duckdb/extension/parquet/include/reader/templated_column_reader.hpp
vendored
Normal file
110
external/duckdb/extension/parquet/include/reader/templated_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,110 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/templated_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "duckdb/common/helper.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
template <class VALUE_TYPE>
|
||||
struct TemplatedParquetValueConversion {
|
||||
template <bool CHECKED>
|
||||
static VALUE_TYPE PlainRead(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
if (CHECKED) {
|
||||
return plain_data.read<VALUE_TYPE>();
|
||||
} else {
|
||||
return plain_data.unsafe_read<VALUE_TYPE>();
|
||||
}
|
||||
}
|
||||
|
||||
template <bool CHECKED>
|
||||
static void PlainSkip(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
if (CHECKED) {
|
||||
plain_data.inc(sizeof(VALUE_TYPE));
|
||||
} else {
|
||||
plain_data.unsafe_inc(sizeof(VALUE_TYPE));
|
||||
}
|
||||
}
|
||||
|
||||
static bool PlainAvailable(const ByteBuffer &plain_data, const idx_t count) {
|
||||
return plain_data.check_available(count * sizeof(VALUE_TYPE));
|
||||
}
|
||||
|
||||
static idx_t PlainConstantSize() {
|
||||
return sizeof(VALUE_TYPE);
|
||||
}
|
||||
};
|
||||
|
||||
template <class VALUE_TYPE, class VALUE_CONVERSION>
|
||||
class TemplatedColumnReader : public ColumnReader {
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::INVALID;
|
||||
|
||||
public:
|
||||
TemplatedColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema) : ColumnReader(reader, schema) {
|
||||
}
|
||||
|
||||
shared_ptr<ResizeableBuffer> dict;
|
||||
|
||||
public:
|
||||
void AllocateDict(idx_t size) {
|
||||
if (!dict) {
|
||||
dict = make_shared_ptr<ResizeableBuffer>(GetAllocator(), size);
|
||||
} else {
|
||||
dict->resize(GetAllocator(), size);
|
||||
}
|
||||
}
|
||||
|
||||
void Plain(ByteBuffer &plain_data, uint8_t *defines, uint64_t num_values, idx_t result_offset,
|
||||
Vector &result) override {
|
||||
PlainTemplated<VALUE_TYPE, VALUE_CONVERSION>(plain_data, defines, num_values, result_offset, result);
|
||||
}
|
||||
|
||||
void PlainSkip(ByteBuffer &plain_data, uint8_t *defines, idx_t num_values) override {
|
||||
PlainSkipTemplated<VALUE_CONVERSION>(plain_data, defines, num_values);
|
||||
}
|
||||
|
||||
bool SupportsDirectFilter() const override {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
template <class PARQUET_PHYSICAL_TYPE, class DUCKDB_PHYSICAL_TYPE,
|
||||
DUCKDB_PHYSICAL_TYPE (*FUNC)(const PARQUET_PHYSICAL_TYPE &input)>
|
||||
struct CallbackParquetValueConversion {
|
||||
|
||||
template <bool CHECKED>
|
||||
static DUCKDB_PHYSICAL_TYPE PlainRead(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
if (CHECKED) {
|
||||
return FUNC(plain_data.read<PARQUET_PHYSICAL_TYPE>());
|
||||
} else {
|
||||
return FUNC(plain_data.unsafe_read<PARQUET_PHYSICAL_TYPE>());
|
||||
}
|
||||
}
|
||||
|
||||
template <bool CHECKED>
|
||||
static void PlainSkip(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
if (CHECKED) {
|
||||
plain_data.inc(sizeof(PARQUET_PHYSICAL_TYPE));
|
||||
} else {
|
||||
plain_data.unsafe_inc(sizeof(PARQUET_PHYSICAL_TYPE));
|
||||
}
|
||||
}
|
||||
|
||||
static bool PlainAvailable(const ByteBuffer &plain_data, const idx_t count) {
|
||||
return plain_data.check_available(count * sizeof(PARQUET_PHYSICAL_TYPE));
|
||||
}
|
||||
|
||||
static idx_t PlainConstantSize() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
60
external/duckdb/extension/parquet/include/reader/uuid_column_reader.hpp
vendored
Normal file
60
external/duckdb/extension/parquet/include/reader/uuid_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,60 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/uuid_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "templated_column_reader.hpp"
|
||||
#include "parquet_reader.hpp"
|
||||
#include "duckdb/common/types/uuid.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
struct UUIDValueConversion {
|
||||
static hugeint_t ReadParquetUUID(const_data_ptr_t input) {
|
||||
// Use the utility function from BaseUUID
|
||||
return BaseUUID::FromBlob(input);
|
||||
}
|
||||
|
||||
template <bool CHECKED>
|
||||
static hugeint_t PlainRead(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
if (CHECKED) {
|
||||
plain_data.available(sizeof(hugeint_t));
|
||||
}
|
||||
auto res = ReadParquetUUID(const_data_ptr_cast(plain_data.ptr));
|
||||
plain_data.unsafe_inc(sizeof(hugeint_t));
|
||||
return res;
|
||||
}
|
||||
|
||||
template <bool CHECKED>
|
||||
static void PlainSkip(ByteBuffer &plain_data, ColumnReader &reader) {
|
||||
if (CHECKED) {
|
||||
plain_data.inc(sizeof(hugeint_t));
|
||||
} else {
|
||||
plain_data.unsafe_inc(sizeof(hugeint_t));
|
||||
}
|
||||
}
|
||||
|
||||
static bool PlainAvailable(const ByteBuffer &plain_data, const idx_t count) {
|
||||
return plain_data.check_available(count * sizeof(hugeint_t));
|
||||
}
|
||||
|
||||
static idx_t PlainConstantSize() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
class UUIDColumnReader : public TemplatedColumnReader<hugeint_t, UUIDValueConversion> {
|
||||
|
||||
public:
|
||||
UUIDColumnReader(ParquetReader &reader, const ParquetColumnSchema &schema)
|
||||
: TemplatedColumnReader<hugeint_t, UUIDValueConversion>(reader, schema) {
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
148
external/duckdb/extension/parquet/include/reader/variant/variant_binary_decoder.hpp
vendored
Normal file
148
external/duckdb/extension/parquet/include/reader/variant/variant_binary_decoder.hpp
vendored
Normal file
@@ -0,0 +1,148 @@
|
||||
#pragma once
|
||||
|
||||
#include "duckdb/common/types/string_type.hpp"
|
||||
#include "duckdb/common/types/value.hpp"
|
||||
#include "reader/variant/variant_value.hpp"
|
||||
|
||||
using namespace duckdb_yyjson;
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
//! ------------ Metadata ------------
|
||||
|
||||
struct VariantMetadataHeader {
|
||||
public:
|
||||
static VariantMetadataHeader FromHeaderByte(uint8_t byte);
|
||||
|
||||
public:
|
||||
//! The version of the protocol used (only '1' supported for now)
|
||||
uint8_t version;
|
||||
//! Number of bytes per dictionary size and offset field
|
||||
uint8_t offset_size;
|
||||
//! Whether dictionary strings are sorted and unique
|
||||
bool sorted_strings = false;
|
||||
};
|
||||
|
||||
struct VariantMetadata {
|
||||
public:
|
||||
explicit VariantMetadata(const string_t &metadata);
|
||||
|
||||
public:
|
||||
const string_t &metadata;
|
||||
|
||||
public:
|
||||
VariantMetadataHeader header;
|
||||
const_data_ptr_t offsets;
|
||||
const_data_ptr_t bytes;
|
||||
|
||||
//! The json object keys have to be null-terminated
|
||||
//! But we don't receive them null-terminated
|
||||
vector<string> strings;
|
||||
};
|
||||
|
||||
//! ------------ Value ------------
|
||||
|
||||
enum class VariantBasicType : uint8_t { PRIMITIVE = 0, SHORT_STRING = 1, OBJECT = 2, ARRAY = 3, INVALID };
|
||||
|
||||
enum class VariantPrimitiveType : uint8_t {
|
||||
NULL_TYPE = 0,
|
||||
BOOLEAN_TRUE = 1,
|
||||
BOOLEAN_FALSE = 2,
|
||||
INT8 = 3,
|
||||
INT16 = 4,
|
||||
INT32 = 5,
|
||||
INT64 = 6,
|
||||
DOUBLE = 7,
|
||||
DECIMAL4 = 8,
|
||||
DECIMAL8 = 9,
|
||||
DECIMAL16 = 10,
|
||||
DATE = 11,
|
||||
TIMESTAMP_MICROS = 12,
|
||||
TIMESTAMP_NTZ_MICROS = 13,
|
||||
FLOAT = 14,
|
||||
BINARY = 15,
|
||||
STRING = 16,
|
||||
TIME_NTZ_MICROS = 17,
|
||||
TIMESTAMP_NANOS = 18,
|
||||
TIMESTAMP_NTZ_NANOS = 19,
|
||||
UUID = 20,
|
||||
INVALID
|
||||
};
|
||||
|
||||
struct VariantValueMetadata {
|
||||
public:
|
||||
VariantValueMetadata() {
|
||||
}
|
||||
|
||||
public:
|
||||
static VariantValueMetadata FromHeaderByte(uint8_t byte);
|
||||
static VariantBasicType VariantBasicTypeFromByte(uint8_t byte) {
|
||||
if (byte >= static_cast<uint8_t>(VariantBasicType::INVALID)) {
|
||||
throw NotImplementedException("Variant BasicType (%d) is not supported", byte);
|
||||
}
|
||||
return static_cast<VariantBasicType>(byte);
|
||||
}
|
||||
|
||||
static VariantPrimitiveType VariantPrimitiveTypeFromByte(uint8_t byte) {
|
||||
if (byte >= static_cast<uint8_t>(VariantPrimitiveType::INVALID)) {
|
||||
throw NotImplementedException("Variant PrimitiveType (%d) is not supported", byte);
|
||||
}
|
||||
return static_cast<VariantPrimitiveType>(byte);
|
||||
}
|
||||
|
||||
public:
|
||||
VariantBasicType basic_type;
|
||||
|
||||
public:
|
||||
//! Primitive Type header
|
||||
VariantPrimitiveType primitive_type;
|
||||
|
||||
public:
|
||||
//! Short String header
|
||||
uint8_t string_size;
|
||||
|
||||
public:
|
||||
//! Object header | Array header
|
||||
|
||||
//! Size in bytes for each 'field_offset' entry
|
||||
uint32_t field_offset_size;
|
||||
//! Size in bytes for each 'field_id' entry
|
||||
uint32_t field_id_size;
|
||||
//! Whether the number of elements is encoded in 1 byte (false) or 4 bytes (true)
|
||||
bool is_large;
|
||||
};
|
||||
|
||||
struct VariantDecodeResult {
|
||||
public:
|
||||
VariantDecodeResult() = default;
|
||||
~VariantDecodeResult() {
|
||||
if (doc) {
|
||||
yyjson_mut_doc_free(doc);
|
||||
}
|
||||
if (data) {
|
||||
free(data);
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
yyjson_mut_doc *doc = nullptr;
|
||||
char *data = nullptr;
|
||||
};
|
||||
|
||||
class VariantBinaryDecoder {
|
||||
public:
|
||||
VariantBinaryDecoder() = delete;
|
||||
|
||||
public:
|
||||
static VariantValue Decode(const VariantMetadata &metadata, const_data_ptr_t data);
|
||||
|
||||
public:
|
||||
static VariantValue PrimitiveTypeDecode(const VariantValueMetadata &value_metadata, const_data_ptr_t data);
|
||||
static VariantValue ShortStringDecode(const VariantValueMetadata &value_metadata, const_data_ptr_t data);
|
||||
static VariantValue ObjectDecode(const VariantMetadata &metadata, const VariantValueMetadata &value_metadata,
|
||||
const_data_ptr_t data);
|
||||
static VariantValue ArrayDecode(const VariantMetadata &metadata, const VariantValueMetadata &value_metadata,
|
||||
const_data_ptr_t data);
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
24
external/duckdb/extension/parquet/include/reader/variant/variant_shredded_conversion.hpp
vendored
Normal file
24
external/duckdb/extension/parquet/include/reader/variant/variant_shredded_conversion.hpp
vendored
Normal file
@@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include "reader/variant/variant_value.hpp"
|
||||
#include "reader/variant/variant_binary_decoder.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class VariantShreddedConversion {
|
||||
public:
|
||||
VariantShreddedConversion() = delete;
|
||||
|
||||
public:
|
||||
static vector<VariantValue> Convert(Vector &metadata, Vector &group, idx_t offset, idx_t length, idx_t total_size,
|
||||
bool is_field);
|
||||
static vector<VariantValue> ConvertShreddedLeaf(Vector &metadata, Vector &value, Vector &typed_value, idx_t offset,
|
||||
idx_t length, idx_t total_size, const bool is_field);
|
||||
static vector<VariantValue> ConvertShreddedArray(Vector &metadata, Vector &value, Vector &typed_value, idx_t offset,
|
||||
idx_t length, idx_t total_size, const bool is_field);
|
||||
static vector<VariantValue> ConvertShreddedObject(Vector &metadata, Vector &value, Vector &typed_value,
|
||||
idx_t offset, idx_t length, idx_t total_size,
|
||||
const bool is_field);
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
54
external/duckdb/extension/parquet/include/reader/variant/variant_value.hpp
vendored
Normal file
54
external/duckdb/extension/parquet/include/reader/variant/variant_value.hpp
vendored
Normal file
@@ -0,0 +1,54 @@
|
||||
#pragma once
|
||||
|
||||
#include "duckdb/common/map.hpp"
|
||||
#include "duckdb/common/vector.hpp"
|
||||
#include "duckdb/common/types/value.hpp"
|
||||
|
||||
#include "yyjson.hpp"
|
||||
|
||||
using namespace duckdb_yyjson;
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
enum class VariantValueType : uint8_t { PRIMITIVE, OBJECT, ARRAY, MISSING };
|
||||
|
||||
struct VariantValue {
|
||||
public:
|
||||
VariantValue() : value_type(VariantValueType::MISSING) {
|
||||
}
|
||||
explicit VariantValue(VariantValueType type) : value_type(type) {
|
||||
}
|
||||
explicit VariantValue(Value &&val) : value_type(VariantValueType::PRIMITIVE), primitive_value(std::move(val)) {
|
||||
}
|
||||
// Delete copy constructor and copy assignment operator
|
||||
VariantValue(const VariantValue &) = delete;
|
||||
VariantValue &operator=(const VariantValue &) = delete;
|
||||
|
||||
// Default move constructor and move assignment operator
|
||||
VariantValue(VariantValue &&) noexcept = default;
|
||||
VariantValue &operator=(VariantValue &&) noexcept = default;
|
||||
|
||||
public:
|
||||
bool IsNull() const {
|
||||
return value_type == VariantValueType::PRIMITIVE && primitive_value.IsNull();
|
||||
}
|
||||
bool IsMissing() const {
|
||||
return value_type == VariantValueType::MISSING;
|
||||
}
|
||||
|
||||
public:
|
||||
void AddChild(const string &key, VariantValue &&val);
|
||||
void AddItem(VariantValue &&val);
|
||||
|
||||
public:
|
||||
yyjson_mut_val *ToJSON(ClientContext &context, yyjson_mut_doc *doc) const;
|
||||
|
||||
public:
|
||||
VariantValueType value_type;
|
||||
//! FIXME: how can we get a deterministic child order for a partially shredded object?
|
||||
map<string, VariantValue> object_children;
|
||||
vector<VariantValue> array_items;
|
||||
Value primitive_value;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
44
external/duckdb/extension/parquet/include/reader/variant_column_reader.hpp
vendored
Normal file
44
external/duckdb/extension/parquet/include/reader/variant_column_reader.hpp
vendored
Normal file
@@ -0,0 +1,44 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// reader/variant_column_reader.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_reader.hpp"
|
||||
#include "reader/templated_column_reader.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class VariantColumnReader : public ColumnReader {
|
||||
public:
|
||||
static constexpr const PhysicalType TYPE = PhysicalType::VARCHAR;
|
||||
|
||||
public:
|
||||
VariantColumnReader(ClientContext &context, ParquetReader &reader, const ParquetColumnSchema &schema,
|
||||
vector<unique_ptr<ColumnReader>> child_readers_p);
|
||||
|
||||
ClientContext &context;
|
||||
vector<unique_ptr<ColumnReader>> child_readers;
|
||||
|
||||
public:
|
||||
ColumnReader &GetChildReader(idx_t child_idx);
|
||||
|
||||
void InitializeRead(idx_t row_group_idx_p, const vector<ColumnChunk> &columns, TProtocol &protocol_p) override;
|
||||
|
||||
idx_t Read(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result) override;
|
||||
|
||||
void Skip(idx_t num_values) override;
|
||||
idx_t GroupRowsAvailable() override;
|
||||
uint64_t TotalCompressedSize() override;
|
||||
void RegisterPrefetch(ThriftFileTransport &transport, bool allow_merge) override;
|
||||
|
||||
protected:
|
||||
idx_t metadata_reader_idx;
|
||||
idx_t value_reader_idx;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
114
external/duckdb/extension/parquet/include/resizable_buffer.hpp
vendored
Normal file
114
external/duckdb/extension/parquet/include/resizable_buffer.hpp
vendored
Normal file
@@ -0,0 +1,114 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// resizable_buffer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "duckdb/common/allocator.hpp"
|
||||
|
||||
#include <exception>
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ByteBuffer { // on to the 10 thousandth impl
|
||||
public:
|
||||
ByteBuffer() {};
|
||||
ByteBuffer(data_ptr_t ptr, uint64_t len) : ptr(ptr), len(len) {};
|
||||
|
||||
data_ptr_t ptr = nullptr;
|
||||
uint64_t len = 0;
|
||||
|
||||
public:
|
||||
void inc(const uint64_t increment) {
|
||||
available(increment);
|
||||
unsafe_inc(increment);
|
||||
}
|
||||
|
||||
void unsafe_inc(const uint64_t increment) {
|
||||
len -= increment;
|
||||
ptr += increment;
|
||||
}
|
||||
|
||||
template <class T>
|
||||
T read() {
|
||||
available(sizeof(T));
|
||||
return unsafe_read<T>();
|
||||
}
|
||||
|
||||
template <class T>
|
||||
T unsafe_read() {
|
||||
T val = unsafe_get<T>();
|
||||
unsafe_inc(sizeof(T));
|
||||
return val;
|
||||
}
|
||||
|
||||
template <class T>
|
||||
T get() {
|
||||
available(sizeof(T));
|
||||
return unsafe_get<T>();
|
||||
}
|
||||
|
||||
template <class T>
|
||||
T unsafe_get() {
|
||||
return Load<T>(ptr);
|
||||
}
|
||||
|
||||
void copy_to(char *dest, const uint64_t len) const {
|
||||
available(len);
|
||||
unsafe_copy_to(dest, len);
|
||||
}
|
||||
|
||||
void unsafe_copy_to(char *dest, const uint64_t len) const {
|
||||
std::memcpy(dest, ptr, len);
|
||||
}
|
||||
|
||||
void zero() const {
|
||||
std::memset(ptr, 0, len);
|
||||
}
|
||||
|
||||
void available(const uint64_t req_len) const {
|
||||
if (!check_available(req_len)) {
|
||||
throw std::runtime_error("Out of buffer");
|
||||
}
|
||||
}
|
||||
|
||||
bool check_available(const uint64_t req_len) const {
|
||||
return req_len <= len;
|
||||
}
|
||||
};
|
||||
|
||||
class ResizeableBuffer : public ByteBuffer {
|
||||
public:
|
||||
ResizeableBuffer() {
|
||||
}
|
||||
ResizeableBuffer(Allocator &allocator, const uint64_t new_size) {
|
||||
resize(allocator, new_size);
|
||||
}
|
||||
void resize(Allocator &allocator, const uint64_t new_size) {
|
||||
len = new_size;
|
||||
if (new_size == 0) {
|
||||
return;
|
||||
}
|
||||
if (new_size > alloc_len) {
|
||||
alloc_len = NextPowerOfTwo(new_size);
|
||||
allocated_data.Reset(); // Have to reset before allocating new buffer (otherwise we use ~2x the memory)
|
||||
allocated_data = allocator.Allocate(alloc_len);
|
||||
ptr = allocated_data.get();
|
||||
}
|
||||
}
|
||||
void reset() {
|
||||
ptr = allocated_data.get();
|
||||
len = alloc_len;
|
||||
}
|
||||
|
||||
private:
|
||||
AllocatedData allocated_data;
|
||||
idx_t alloc_len = 0;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
228
external/duckdb/extension/parquet/include/thrift_tools.hpp
vendored
Normal file
228
external/duckdb/extension/parquet/include/thrift_tools.hpp
vendored
Normal file
@@ -0,0 +1,228 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// thrift_tools.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===/
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <list>
|
||||
#include "thrift/protocol/TCompactProtocol.h"
|
||||
#include "thrift/transport/TBufferTransports.h"
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "duckdb/storage/caching_file_system.hpp"
|
||||
#include "duckdb/common/file_system.hpp"
|
||||
#include "duckdb/common/allocator.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
// A ReadHead for prefetching data in a specific range
|
||||
struct ReadHead {
|
||||
ReadHead(idx_t location, uint64_t size) : location(location), size(size) {};
|
||||
// Hint info
|
||||
idx_t location;
|
||||
uint64_t size;
|
||||
|
||||
// Current info
|
||||
BufferHandle buffer_handle;
|
||||
data_ptr_t buffer_ptr;
|
||||
bool data_isset = false;
|
||||
|
||||
idx_t GetEnd() const {
|
||||
return size + location;
|
||||
}
|
||||
};
|
||||
|
||||
// Comparator for ReadHeads that are either overlapping, adjacent, or within ALLOW_GAP bytes from each other
|
||||
struct ReadHeadComparator {
|
||||
static constexpr uint64_t ALLOW_GAP = 1 << 14; // 16 KiB
|
||||
bool operator()(const ReadHead *a, const ReadHead *b) const {
|
||||
auto a_start = a->location;
|
||||
auto a_end = a->location + a->size;
|
||||
auto b_start = b->location;
|
||||
|
||||
if (a_end <= NumericLimits<idx_t>::Maximum() - ALLOW_GAP) {
|
||||
a_end += ALLOW_GAP;
|
||||
}
|
||||
|
||||
return a_start < b_start && a_end < b_start;
|
||||
}
|
||||
};
|
||||
|
||||
// Two-step read ahead buffer
|
||||
// 1: register all ranges that will be read, merging ranges that are consecutive
|
||||
// 2: prefetch all registered ranges
|
||||
struct ReadAheadBuffer {
|
||||
explicit ReadAheadBuffer(CachingFileHandle &file_handle_p) : file_handle(file_handle_p) {
|
||||
}
|
||||
|
||||
// The list of read heads
|
||||
std::list<ReadHead> read_heads;
|
||||
// Set for merging consecutive ranges
|
||||
std::set<ReadHead *, ReadHeadComparator> merge_set;
|
||||
|
||||
CachingFileHandle &file_handle;
|
||||
|
||||
idx_t total_size = 0;
|
||||
|
||||
// Add a read head to the prefetching list
|
||||
void AddReadHead(idx_t pos, uint64_t len, bool merge_buffers = true) {
|
||||
// Attempt to merge with existing
|
||||
if (merge_buffers) {
|
||||
ReadHead new_read_head {pos, len};
|
||||
auto lookup_set = merge_set.find(&new_read_head);
|
||||
if (lookup_set != merge_set.end()) {
|
||||
auto existing_head = *lookup_set;
|
||||
auto new_start = MinValue<idx_t>(existing_head->location, new_read_head.location);
|
||||
auto new_length = MaxValue<idx_t>(existing_head->GetEnd(), new_read_head.GetEnd()) - new_start;
|
||||
existing_head->location = new_start;
|
||||
existing_head->size = new_length;
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
read_heads.emplace_front(ReadHead(pos, len));
|
||||
total_size += len;
|
||||
auto &read_head = read_heads.front();
|
||||
|
||||
if (merge_buffers) {
|
||||
merge_set.insert(&read_head);
|
||||
}
|
||||
|
||||
if (read_head.GetEnd() > file_handle.GetFileSize()) {
|
||||
throw std::runtime_error("Prefetch registered for bytes outside file: " + file_handle.GetPath() +
|
||||
", attempted range: [" + std::to_string(pos) + ", " +
|
||||
std::to_string(read_head.GetEnd()) +
|
||||
"), file size: " + std::to_string(file_handle.GetFileSize()));
|
||||
}
|
||||
}
|
||||
|
||||
// Returns the relevant read head
|
||||
ReadHead *GetReadHead(idx_t pos) {
|
||||
for (auto &read_head : read_heads) {
|
||||
if (pos >= read_head.location && pos < read_head.GetEnd()) {
|
||||
return &read_head;
|
||||
}
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
// Prefetch all read heads
|
||||
void Prefetch() {
|
||||
for (auto &read_head : read_heads) {
|
||||
if (read_head.GetEnd() > file_handle.GetFileSize()) {
|
||||
throw std::runtime_error("Prefetch registered requested for bytes outside file");
|
||||
}
|
||||
read_head.buffer_handle = file_handle.Read(read_head.buffer_ptr, read_head.size, read_head.location);
|
||||
D_ASSERT(read_head.buffer_handle.IsValid());
|
||||
read_head.data_isset = true;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
class ThriftFileTransport : public duckdb_apache::thrift::transport::TVirtualTransport<ThriftFileTransport> {
|
||||
public:
|
||||
static constexpr uint64_t PREFETCH_FALLBACK_BUFFERSIZE = 1000000;
|
||||
|
||||
ThriftFileTransport(CachingFileHandle &file_handle_p, bool prefetch_mode_p)
|
||||
: file_handle(file_handle_p), location(0), size(file_handle.GetFileSize()),
|
||||
ra_buffer(ReadAheadBuffer(file_handle)), prefetch_mode(prefetch_mode_p) {
|
||||
}
|
||||
|
||||
uint32_t read(uint8_t *buf, uint32_t len) {
|
||||
auto prefetch_buffer = ra_buffer.GetReadHead(location);
|
||||
if (prefetch_buffer != nullptr && location - prefetch_buffer->location + len <= prefetch_buffer->size) {
|
||||
D_ASSERT(location - prefetch_buffer->location + len <= prefetch_buffer->size);
|
||||
|
||||
if (!prefetch_buffer->data_isset) {
|
||||
prefetch_buffer->buffer_handle =
|
||||
file_handle.Read(prefetch_buffer->buffer_ptr, prefetch_buffer->size, prefetch_buffer->location);
|
||||
D_ASSERT(prefetch_buffer->buffer_handle.IsValid());
|
||||
prefetch_buffer->data_isset = true;
|
||||
}
|
||||
D_ASSERT(prefetch_buffer->buffer_handle.IsValid());
|
||||
memcpy(buf, prefetch_buffer->buffer_ptr + location - prefetch_buffer->location, len);
|
||||
} else if (prefetch_mode && len < PREFETCH_FALLBACK_BUFFERSIZE && len > 0) {
|
||||
Prefetch(location, MinValue<uint64_t>(PREFETCH_FALLBACK_BUFFERSIZE, file_handle.GetFileSize() - location));
|
||||
auto prefetch_buffer_fallback = ra_buffer.GetReadHead(location);
|
||||
D_ASSERT(location - prefetch_buffer_fallback->location + len <= prefetch_buffer_fallback->size);
|
||||
memcpy(buf, prefetch_buffer_fallback->buffer_ptr + location - prefetch_buffer_fallback->location, len);
|
||||
} else {
|
||||
// No prefetch, do a regular (non-caching) read
|
||||
file_handle.GetFileHandle().Read(context, buf, len, location);
|
||||
}
|
||||
|
||||
location += len;
|
||||
return len;
|
||||
}
|
||||
|
||||
// Prefetch a single buffer
|
||||
void Prefetch(idx_t pos, uint64_t len) {
|
||||
RegisterPrefetch(pos, len, false);
|
||||
FinalizeRegistration();
|
||||
PrefetchRegistered();
|
||||
}
|
||||
|
||||
// Register a buffer for prefixing
|
||||
void RegisterPrefetch(idx_t pos, uint64_t len, bool can_merge = true) {
|
||||
ra_buffer.AddReadHead(pos, len, can_merge);
|
||||
}
|
||||
|
||||
// Prevents any further merges, should be called before PrefetchRegistered
|
||||
void FinalizeRegistration() {
|
||||
ra_buffer.merge_set.clear();
|
||||
}
|
||||
|
||||
// Prefetch all previously registered ranges
|
||||
void PrefetchRegistered() {
|
||||
ra_buffer.Prefetch();
|
||||
}
|
||||
|
||||
void ClearPrefetch() {
|
||||
ra_buffer.read_heads.clear();
|
||||
ra_buffer.merge_set.clear();
|
||||
}
|
||||
|
||||
void Skip(idx_t skip_count) {
|
||||
location += skip_count;
|
||||
}
|
||||
|
||||
bool HasPrefetch() const {
|
||||
return !ra_buffer.read_heads.empty() || !ra_buffer.merge_set.empty();
|
||||
}
|
||||
|
||||
void SetLocation(idx_t location_p) {
|
||||
location = location_p;
|
||||
}
|
||||
|
||||
idx_t GetLocation() const {
|
||||
return location;
|
||||
}
|
||||
|
||||
optional_ptr<ReadHead> GetReadHead(idx_t pos) {
|
||||
return ra_buffer.GetReadHead(pos);
|
||||
}
|
||||
|
||||
idx_t GetSize() const {
|
||||
return size;
|
||||
}
|
||||
|
||||
private:
|
||||
QueryContext context;
|
||||
|
||||
CachingFileHandle &file_handle;
|
||||
idx_t location;
|
||||
idx_t size;
|
||||
|
||||
// Multi-buffer prefetch
|
||||
ReadAheadBuffer ra_buffer;
|
||||
|
||||
// Whether the prefetch mode is enabled. In this mode the DirectIO flag of the handle will be set and the parquet
|
||||
// reader will manage the read buffering.
|
||||
bool prefetch_mode;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
34
external/duckdb/extension/parquet/include/writer/array_column_writer.hpp
vendored
Normal file
34
external/duckdb/extension/parquet/include/writer/array_column_writer.hpp
vendored
Normal file
@@ -0,0 +1,34 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/array_column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "writer/list_column_writer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ArrayColumnWriter : public ListColumnWriter {
|
||||
public:
|
||||
ArrayColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema, vector<string> schema_path_p,
|
||||
unique_ptr<ColumnWriter> child_writer_p, bool can_have_nulls)
|
||||
: ListColumnWriter(writer, column_schema, std::move(schema_path_p), std::move(child_writer_p), can_have_nulls) {
|
||||
}
|
||||
~ArrayColumnWriter() override = default;
|
||||
|
||||
public:
|
||||
void Analyze(ColumnWriterState &state, ColumnWriterState *parent, Vector &vector, idx_t count) override;
|
||||
void Prepare(ColumnWriterState &state, ColumnWriterState *parent, Vector &vector, idx_t count,
|
||||
bool vector_can_span_multiple_pages) override;
|
||||
void Write(ColumnWriterState &state, Vector &vector, idx_t count) override;
|
||||
|
||||
protected:
|
||||
void WriteArrayState(ListColumnWriterState &state, idx_t array_size, uint16_t first_repeat_level,
|
||||
idx_t define_value, const bool is_empty = false);
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
33
external/duckdb/extension/parquet/include/writer/boolean_column_writer.hpp
vendored
Normal file
33
external/duckdb/extension/parquet/include/writer/boolean_column_writer.hpp
vendored
Normal file
@@ -0,0 +1,33 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/boolean_column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "writer/primitive_column_writer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class BooleanColumnWriter : public PrimitiveColumnWriter {
|
||||
public:
|
||||
BooleanColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema, vector<string> schema_path_p,
|
||||
bool can_have_nulls);
|
||||
~BooleanColumnWriter() override = default;
|
||||
|
||||
public:
|
||||
unique_ptr<ColumnWriterStatistics> InitializeStatsState() override;
|
||||
|
||||
void WriteVector(WriteStream &temp_writer, ColumnWriterStatistics *stats_p, ColumnWriterPageState *state_p,
|
||||
Vector &input_column, idx_t chunk_start, idx_t chunk_end) override;
|
||||
|
||||
unique_ptr<ColumnWriterPageState> InitializePageState(PrimitiveColumnWriterState &state, idx_t page_idx) override;
|
||||
void FlushPageState(WriteStream &temp_writer, ColumnWriterPageState *state_p) override;
|
||||
|
||||
idx_t GetRowSize(const Vector &vector, const idx_t index, const PrimitiveColumnWriterState &state) const override;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
30
external/duckdb/extension/parquet/include/writer/decimal_column_writer.hpp
vendored
Normal file
30
external/duckdb/extension/parquet/include/writer/decimal_column_writer.hpp
vendored
Normal file
@@ -0,0 +1,30 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/decimal_column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "writer/primitive_column_writer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class FixedDecimalColumnWriter : public PrimitiveColumnWriter {
|
||||
public:
|
||||
FixedDecimalColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema,
|
||||
vector<string> schema_path_p, bool can_have_nulls);
|
||||
~FixedDecimalColumnWriter() override = default;
|
||||
|
||||
public:
|
||||
unique_ptr<ColumnWriterStatistics> InitializeStatsState() override;
|
||||
|
||||
void WriteVector(WriteStream &temp_writer, ColumnWriterStatistics *stats_p, ColumnWriterPageState *page_state,
|
||||
Vector &input_column, idx_t chunk_start, idx_t chunk_end) override;
|
||||
|
||||
idx_t GetRowSize(const Vector &vector, const idx_t index, const PrimitiveColumnWriterState &state) const override;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
50
external/duckdb/extension/parquet/include/writer/enum_column_writer.hpp
vendored
Normal file
50
external/duckdb/extension/parquet/include/writer/enum_column_writer.hpp
vendored
Normal file
@@ -0,0 +1,50 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/enum_column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "writer/primitive_column_writer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
class EnumWriterPageState;
|
||||
|
||||
class EnumColumnWriter : public PrimitiveColumnWriter {
|
||||
public:
|
||||
EnumColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema, vector<string> schema_path_p,
|
||||
bool can_have_nulls);
|
||||
~EnumColumnWriter() override = default;
|
||||
|
||||
uint32_t bit_width;
|
||||
|
||||
public:
|
||||
unique_ptr<ColumnWriterStatistics> InitializeStatsState() override;
|
||||
|
||||
void WriteVector(WriteStream &temp_writer, ColumnWriterStatistics *stats_p, ColumnWriterPageState *page_state_p,
|
||||
Vector &input_column, idx_t chunk_start, idx_t chunk_end) override;
|
||||
|
||||
unique_ptr<ColumnWriterPageState> InitializePageState(PrimitiveColumnWriterState &state, idx_t page_idx) override;
|
||||
|
||||
void FlushPageState(WriteStream &temp_writer, ColumnWriterPageState *state_p) override;
|
||||
|
||||
duckdb_parquet::Encoding::type GetEncoding(PrimitiveColumnWriterState &state) override;
|
||||
|
||||
bool HasDictionary(PrimitiveColumnWriterState &state) override;
|
||||
|
||||
idx_t DictionarySize(PrimitiveColumnWriterState &state_p) override;
|
||||
|
||||
void FlushDictionary(PrimitiveColumnWriterState &state, ColumnWriterStatistics *stats_p) override;
|
||||
|
||||
idx_t GetRowSize(const Vector &vector, const idx_t index, const PrimitiveColumnWriterState &state) const override;
|
||||
|
||||
private:
|
||||
template <class T>
|
||||
void WriteEnumInternal(WriteStream &temp_writer, Vector &input_column, idx_t chunk_start, idx_t chunk_end,
|
||||
EnumWriterPageState &page_state);
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
52
external/duckdb/extension/parquet/include/writer/list_column_writer.hpp
vendored
Normal file
52
external/duckdb/extension/parquet/include/writer/list_column_writer.hpp
vendored
Normal file
@@ -0,0 +1,52 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/list_column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_writer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ListColumnWriterState : public ColumnWriterState {
|
||||
public:
|
||||
ListColumnWriterState(duckdb_parquet::RowGroup &row_group, idx_t col_idx) : row_group(row_group), col_idx(col_idx) {
|
||||
}
|
||||
~ListColumnWriterState() override = default;
|
||||
|
||||
duckdb_parquet::RowGroup &row_group;
|
||||
idx_t col_idx;
|
||||
unique_ptr<ColumnWriterState> child_state;
|
||||
idx_t parent_index = 0;
|
||||
};
|
||||
|
||||
class ListColumnWriter : public ColumnWriter {
|
||||
public:
|
||||
ListColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema, vector<string> schema_path_p,
|
||||
unique_ptr<ColumnWriter> child_writer_p, bool can_have_nulls)
|
||||
: ColumnWriter(writer, column_schema, std::move(schema_path_p), can_have_nulls) {
|
||||
child_writers.push_back(std::move(child_writer_p));
|
||||
}
|
||||
~ListColumnWriter() override = default;
|
||||
|
||||
public:
|
||||
unique_ptr<ColumnWriterState> InitializeWriteState(duckdb_parquet::RowGroup &row_group) override;
|
||||
bool HasAnalyze() override;
|
||||
void Analyze(ColumnWriterState &state, ColumnWriterState *parent, Vector &vector, idx_t count) override;
|
||||
void FinalizeAnalyze(ColumnWriterState &state) override;
|
||||
void Prepare(ColumnWriterState &state, ColumnWriterState *parent, Vector &vector, idx_t count,
|
||||
bool vector_can_span_multiple_pages) override;
|
||||
|
||||
void BeginWrite(ColumnWriterState &state) override;
|
||||
void Write(ColumnWriterState &state, Vector &vector, idx_t count) override;
|
||||
void FinalizeWrite(ColumnWriterState &state) override;
|
||||
|
||||
protected:
|
||||
ColumnWriter &GetChildWriter();
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
326
external/duckdb/extension/parquet/include/writer/parquet_write_operators.hpp
vendored
Normal file
326
external/duckdb/extension/parquet/include/writer/parquet_write_operators.hpp
vendored
Normal file
@@ -0,0 +1,326 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/parquet_write_operators.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "writer/parquet_write_stats.hpp"
|
||||
#include "zstd/common/xxhash.hpp"
|
||||
#include "duckdb/common/types/uhugeint.hpp"
|
||||
#include "duckdb/common/types/uuid.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
struct BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static void WriteToStream(const TGT &input, WriteStream &ser) {
|
||||
ser.WriteData(const_data_ptr_cast(&input), sizeof(TGT));
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static constexpr idx_t WriteSize(const TGT &input) {
|
||||
return sizeof(TGT);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static uint64_t XXHash64(const TGT &target_value) {
|
||||
return duckdb_zstd::XXH64(&target_value, sizeof(target_value), 0);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static unique_ptr<ColumnWriterStatistics> InitializeStats() {
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void HandleStats(ColumnWriterStatistics *stats, TGT target_value) {
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static idx_t GetRowSize(const Vector &, idx_t) {
|
||||
return sizeof(TGT);
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetCastOperator : public BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
return TGT(input);
|
||||
}
|
||||
template <class SRC, class TGT>
|
||||
static unique_ptr<ColumnWriterStatistics> InitializeStats() {
|
||||
return make_uniq<NumericStatisticsState<SRC, TGT, BaseParquetOperator>>();
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void HandleStats(ColumnWriterStatistics *stats, TGT target_value) {
|
||||
auto &numeric_stats = stats->Cast<NumericStatisticsState<SRC, TGT, BaseParquetOperator>>();
|
||||
if (LessThan::Operation(target_value, numeric_stats.min)) {
|
||||
numeric_stats.min = target_value;
|
||||
}
|
||||
if (GreaterThan::Operation(target_value, numeric_stats.max)) {
|
||||
numeric_stats.max = target_value;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
struct FloatingPointOperator : public BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
return TGT(input);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static unique_ptr<ColumnWriterStatistics> InitializeStats() {
|
||||
return make_uniq<FloatingPointStatisticsState<SRC, TGT, BaseParquetOperator>>();
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void HandleStats(ColumnWriterStatistics *stats, TGT target_value) {
|
||||
auto &numeric_stats = stats->Cast<FloatingPointStatisticsState<SRC, TGT, BaseParquetOperator>>();
|
||||
if (Value::IsNan(target_value)) {
|
||||
numeric_stats.has_nan = true;
|
||||
} else {
|
||||
if (LessThan::Operation(target_value, numeric_stats.min)) {
|
||||
numeric_stats.min = target_value;
|
||||
}
|
||||
if (GreaterThan::Operation(target_value, numeric_stats.max)) {
|
||||
numeric_stats.max = target_value;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetTimestampNSOperator : public ParquetCastOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
return TGT(input);
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetTimestampSOperator : public ParquetCastOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
return Timestamp::FromEpochSecondsPossiblyInfinite(input).value;
|
||||
}
|
||||
};
|
||||
|
||||
// We will need a different operator for GEOGRAPHY later, so we define a base geo operator
|
||||
struct ParquetBaseGeoOperator : public BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
return input;
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void HandleStats(ColumnWriterStatistics *stats, TGT target_value) {
|
||||
auto &geo_stats = stats->Cast<GeoStatisticsState>();
|
||||
geo_stats.Update(target_value);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void WriteToStream(const TGT &target_value, WriteStream &ser) {
|
||||
ser.Write<uint32_t>(target_value.GetSize());
|
||||
ser.WriteData(const_data_ptr_cast(target_value.GetData()), target_value.GetSize());
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static idx_t WriteSize(const TGT &target_value) {
|
||||
return sizeof(uint32_t) + target_value.GetSize();
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static uint64_t XXHash64(const TGT &target_value) {
|
||||
return duckdb_zstd::XXH64(target_value.GetData(), target_value.GetSize(), 0);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static idx_t GetRowSize(const Vector &vector, idx_t index) {
|
||||
// This needs to add the 4 bytes (just like WriteSize) otherwise we underestimate and we have to realloc
|
||||
// This seriously harms performance, mostly by making it very inconsistent (see internal issue #4990)
|
||||
return sizeof(uint32_t) + FlatVector::GetData<string_t>(vector)[index].GetSize();
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetGeometryOperator : public ParquetBaseGeoOperator {
|
||||
template <class SRC, class TGT>
|
||||
static unique_ptr<ColumnWriterStatistics> InitializeStats() {
|
||||
return make_uniq<GeoStatisticsState>();
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetBaseStringOperator : public BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
return input;
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void HandleStats(ColumnWriterStatistics *stats, TGT target_value) {
|
||||
auto &string_stats = stats->Cast<StringStatisticsState>();
|
||||
string_stats.Update(target_value);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void WriteToStream(const TGT &target_value, WriteStream &ser) {
|
||||
ser.Write<uint32_t>(target_value.GetSize());
|
||||
ser.WriteData(const_data_ptr_cast(target_value.GetData()), target_value.GetSize());
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static idx_t WriteSize(const TGT &target_value) {
|
||||
return sizeof(uint32_t) + target_value.GetSize();
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static uint64_t XXHash64(const TGT &target_value) {
|
||||
return duckdb_zstd::XXH64(target_value.GetData(), target_value.GetSize(), 0);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static idx_t GetRowSize(const Vector &vector, idx_t index) {
|
||||
// This needs to add the 4 bytes (just like WriteSize) otherwise we underestimate and we have to realloc
|
||||
// This seriously harms performance, mostly by making it very inconsistent (see internal issue #4990)
|
||||
return sizeof(uint32_t) + FlatVector::GetData<string_t>(vector)[index].GetSize();
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetBlobOperator : public ParquetBaseStringOperator {
|
||||
template <class SRC, class TGT>
|
||||
static unique_ptr<ColumnWriterStatistics> InitializeStats() {
|
||||
return make_uniq<StringStatisticsState>(LogicalTypeId::BLOB);
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetStringOperator : public ParquetBaseStringOperator {
|
||||
template <class SRC, class TGT>
|
||||
static unique_ptr<ColumnWriterStatistics> InitializeStats() {
|
||||
return make_uniq<StringStatisticsState>();
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetIntervalTargetType {
|
||||
static constexpr const idx_t PARQUET_INTERVAL_SIZE = 12;
|
||||
data_t bytes[PARQUET_INTERVAL_SIZE];
|
||||
};
|
||||
|
||||
struct ParquetIntervalOperator : public BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
if (input.days < 0 || input.months < 0 || input.micros < 0) {
|
||||
throw IOException("Parquet files do not support negative intervals");
|
||||
}
|
||||
TGT result;
|
||||
Store<uint32_t>(input.months, result.bytes);
|
||||
Store<uint32_t>(input.days, result.bytes + sizeof(uint32_t));
|
||||
Store<uint32_t>(input.micros / 1000, result.bytes + sizeof(uint32_t) * 2);
|
||||
return result;
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void WriteToStream(const TGT &target_value, WriteStream &ser) {
|
||||
ser.WriteData(target_value.bytes, ParquetIntervalTargetType::PARQUET_INTERVAL_SIZE);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static constexpr idx_t WriteSize(const TGT &target_value) {
|
||||
return ParquetIntervalTargetType::PARQUET_INTERVAL_SIZE;
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static uint64_t XXHash64(const TGT &target_value) {
|
||||
return duckdb_zstd::XXH64(target_value.bytes, ParquetIntervalTargetType::PARQUET_INTERVAL_SIZE, 0);
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetUUIDTargetType {
|
||||
static constexpr const idx_t PARQUET_UUID_SIZE = 16;
|
||||
data_t bytes[PARQUET_UUID_SIZE];
|
||||
};
|
||||
|
||||
struct ParquetUUIDOperator : public BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
TGT result;
|
||||
// Use the utility function from BaseUUID
|
||||
BaseUUID::ToBlob(input, result.bytes);
|
||||
return result;
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void WriteToStream(const TGT &target_value, WriteStream &ser) {
|
||||
ser.WriteData(target_value.bytes, ParquetUUIDTargetType::PARQUET_UUID_SIZE);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static constexpr idx_t WriteSize(const TGT &target_value) {
|
||||
return ParquetUUIDTargetType::PARQUET_UUID_SIZE;
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static uint64_t XXHash64(const TGT &target_value) {
|
||||
return duckdb_zstd::XXH64(target_value.bytes, ParquetUUIDTargetType::PARQUET_UUID_SIZE, 0);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static unique_ptr<ColumnWriterStatistics> InitializeStats() {
|
||||
return make_uniq<UUIDStatisticsState>();
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void HandleStats(ColumnWriterStatistics *stats_p, TGT target_value) {
|
||||
auto &stats = stats_p->Cast<UUIDStatisticsState>();
|
||||
if (!stats.has_stats || memcmp(target_value.bytes, stats.min, ParquetUUIDTargetType::PARQUET_UUID_SIZE) < 0) {
|
||||
memcpy(stats.min, target_value.bytes, ParquetUUIDTargetType::PARQUET_UUID_SIZE);
|
||||
}
|
||||
if (!stats.has_stats || memcmp(target_value.bytes, stats.max, ParquetUUIDTargetType::PARQUET_UUID_SIZE) > 0) {
|
||||
memcpy(stats.max, target_value.bytes, ParquetUUIDTargetType::PARQUET_UUID_SIZE);
|
||||
}
|
||||
stats.has_stats = true;
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetTimeTZOperator : public BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
return input.time().micros;
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetHugeintOperator : public BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
return Hugeint::Cast<double>(input);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static unique_ptr<ColumnWriterStatistics> InitializeStats() {
|
||||
return make_uniq<ColumnWriterStatistics>();
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void HandleStats(ColumnWriterStatistics *stats, TGT target_value) {
|
||||
}
|
||||
};
|
||||
|
||||
struct ParquetUhugeintOperator : public BaseParquetOperator {
|
||||
template <class SRC, class TGT>
|
||||
static TGT Operation(SRC input) {
|
||||
return Uhugeint::Cast<double>(input);
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static unique_ptr<ColumnWriterStatistics> InitializeStats() {
|
||||
return make_uniq<ColumnWriterStatistics>();
|
||||
}
|
||||
|
||||
template <class SRC, class TGT>
|
||||
static void HandleStats(ColumnWriterStatistics *stats, TGT target_value) {
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
305
external/duckdb/extension/parquet/include/writer/parquet_write_stats.hpp
vendored
Normal file
305
external/duckdb/extension/parquet/include/writer/parquet_write_stats.hpp
vendored
Normal file
@@ -0,0 +1,305 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/parquet_write_stats.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_writer.hpp"
|
||||
#include "geo_parquet.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ColumnWriterStatistics {
|
||||
public:
|
||||
virtual ~ColumnWriterStatistics();
|
||||
|
||||
virtual bool HasStats();
|
||||
virtual string GetMin();
|
||||
virtual string GetMax();
|
||||
virtual string GetMinValue();
|
||||
virtual string GetMaxValue();
|
||||
virtual bool CanHaveNaN();
|
||||
virtual bool HasNaN();
|
||||
virtual bool MinIsExact();
|
||||
virtual bool MaxIsExact();
|
||||
|
||||
virtual bool HasGeoStats();
|
||||
virtual optional_ptr<GeometryStatsData> GetGeoStats();
|
||||
virtual void WriteGeoStats(duckdb_parquet::GeospatialStatistics &stats);
|
||||
|
||||
public:
|
||||
template <class TARGET>
|
||||
TARGET &Cast() {
|
||||
DynamicCastCheck<TARGET>(this);
|
||||
return reinterpret_cast<TARGET &>(*this);
|
||||
}
|
||||
template <class TARGET>
|
||||
const TARGET &Cast() const {
|
||||
D_ASSERT(dynamic_cast<const TARGET *>(this));
|
||||
return reinterpret_cast<const TARGET &>(*this);
|
||||
}
|
||||
};
|
||||
|
||||
template <class SRC, class T, class OP>
|
||||
class NumericStatisticsState : public ColumnWriterStatistics {
|
||||
public:
|
||||
NumericStatisticsState() : min(NumericLimits<T>::Maximum()), max(NumericLimits<T>::Minimum()) {
|
||||
}
|
||||
|
||||
T min;
|
||||
T max;
|
||||
|
||||
public:
|
||||
bool HasStats() override {
|
||||
return min <= max;
|
||||
}
|
||||
|
||||
string GetMin() override {
|
||||
return NumericLimits<SRC>::IsSigned() ? GetMinValue() : string();
|
||||
}
|
||||
string GetMax() override {
|
||||
return NumericLimits<SRC>::IsSigned() ? GetMaxValue() : string();
|
||||
}
|
||||
string GetMinValue() override {
|
||||
return HasStats() ? string(char_ptr_cast(&min), sizeof(T)) : string();
|
||||
}
|
||||
string GetMaxValue() override {
|
||||
return HasStats() ? string(char_ptr_cast(&max), sizeof(T)) : string();
|
||||
}
|
||||
};
|
||||
|
||||
template <class SRC, class T, class OP>
|
||||
class FloatingPointStatisticsState : public NumericStatisticsState<SRC, T, OP> {
|
||||
public:
|
||||
bool has_nan = false;
|
||||
|
||||
public:
|
||||
bool CanHaveNaN() override {
|
||||
return true;
|
||||
}
|
||||
bool HasNaN() override {
|
||||
return has_nan;
|
||||
}
|
||||
};
|
||||
|
||||
class StringStatisticsState : public ColumnWriterStatistics {
|
||||
static constexpr const idx_t MAX_STRING_STATISTICS_SIZE = 256;
|
||||
|
||||
public:
|
||||
explicit StringStatisticsState(LogicalTypeId type = LogicalTypeId::VARCHAR)
|
||||
: type(type), has_stats(false), min_truncated(false), max_truncated(false), min(), max() {
|
||||
}
|
||||
|
||||
LogicalTypeId type;
|
||||
bool has_stats;
|
||||
bool min_truncated;
|
||||
bool max_truncated;
|
||||
bool failed_truncate = false;
|
||||
string min;
|
||||
string max;
|
||||
|
||||
public:
|
||||
bool HasStats() override {
|
||||
return has_stats;
|
||||
}
|
||||
|
||||
void Update(const string_t &val) {
|
||||
if (failed_truncate) {
|
||||
return;
|
||||
}
|
||||
if (!has_stats || LessThan::Operation(val, string_t(min))) {
|
||||
if (val.GetSize() > MAX_STRING_STATISTICS_SIZE) {
|
||||
// string value exceeds our max string stats size - truncate
|
||||
min = TruncateMin(val, MAX_STRING_STATISTICS_SIZE);
|
||||
min_truncated = true;
|
||||
} else {
|
||||
min = val.GetString();
|
||||
min_truncated = false;
|
||||
}
|
||||
}
|
||||
if (!has_stats || GreaterThan::Operation(val, string_t(max))) {
|
||||
if (val.GetSize() > MAX_STRING_STATISTICS_SIZE) {
|
||||
// string value exceeds our max string stats size - truncate
|
||||
if (!TryTruncateMax(val, MAX_STRING_STATISTICS_SIZE, max)) {
|
||||
// we failed to truncate - this can happen in some edge cases
|
||||
// skip stats for this column
|
||||
failed_truncate = true;
|
||||
has_stats = false;
|
||||
min = string();
|
||||
max = string();
|
||||
return;
|
||||
}
|
||||
max_truncated = true;
|
||||
} else {
|
||||
max = val.GetString();
|
||||
max_truncated = false;
|
||||
}
|
||||
}
|
||||
has_stats = true;
|
||||
}
|
||||
|
||||
static inline bool IsCharacter(char c) {
|
||||
return (c & 0xc0) != 0x80;
|
||||
}
|
||||
|
||||
string TruncateMin(string_t str, idx_t max_size) {
|
||||
// truncate a string for the min value
|
||||
// since 'AAA' < 'AAAA', we can just truncate the string
|
||||
D_ASSERT(str.GetSize() > max_size);
|
||||
if (type == LogicalTypeId::BLOB) {
|
||||
// for blobs - just truncate directly
|
||||
return string(str.GetData(), max_size);
|
||||
}
|
||||
D_ASSERT(type == LogicalTypeId::VARCHAR);
|
||||
// for varchar we need to truncate to a valid UTF8 string - so we need to truncate to the last valid UTF8 byte
|
||||
auto str_data = str.GetData();
|
||||
for (; max_size > 0; max_size--) {
|
||||
if (IsCharacter(str_data[max_size])) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
return string(str_data, max_size);
|
||||
}
|
||||
|
||||
bool TryTruncateMax(string_t str, idx_t max_size, string &result, data_t max_byte) {
|
||||
auto data = const_data_ptr_cast(str.GetData());
|
||||
|
||||
// find the last position in the string which we can increment for the truncation
|
||||
// if ALL characters are above the max byte we cannot truncate
|
||||
idx_t increment_pos;
|
||||
for (increment_pos = max_size; increment_pos > 0; increment_pos--) {
|
||||
idx_t str_idx = increment_pos - 1;
|
||||
if (data[str_idx] < max_byte) {
|
||||
// found the increment position
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (increment_pos == 0) {
|
||||
// all characters are above the max byte - we cannot truncate - return false
|
||||
return false;
|
||||
}
|
||||
// set up the result string - we don't care about anything after the increment pos
|
||||
result = string(str.GetData(), increment_pos);
|
||||
// actually increment
|
||||
result[increment_pos - 1]++;
|
||||
return true;
|
||||
}
|
||||
|
||||
bool TryTruncateMax(string_t str, idx_t max_size, string &result) {
|
||||
// truncate a string for the max value
|
||||
// since 'XXX' < 'XXXX', we need to "increment" a byte to get a correct max value
|
||||
// i.e. we need to generate 'XXY' as a string
|
||||
// note that this is not necessarily always possible
|
||||
D_ASSERT(str.GetSize() > max_size);
|
||||
if (type == LogicalTypeId::BLOB) {
|
||||
// for blobs we can always increment bytes - we just can't increment past the max of a single byte (2^8)
|
||||
return TryTruncateMax(str, max_size, result, static_cast<data_t>(0xFF));
|
||||
}
|
||||
D_ASSERT(type == LogicalTypeId::VARCHAR);
|
||||
// for varchar the situation is more complex - we need to truncate to a valid UTF8 string and increment
|
||||
// for now we only increment ASCII characters (characters below 0x7F)
|
||||
return TryTruncateMax(str, max_size, result, static_cast<data_t>(0x7F));
|
||||
}
|
||||
|
||||
string GetMin() override {
|
||||
return GetMinValue();
|
||||
}
|
||||
string GetMax() override {
|
||||
return GetMaxValue();
|
||||
}
|
||||
string GetMinValue() override {
|
||||
return HasStats() ? min : string();
|
||||
}
|
||||
string GetMaxValue() override {
|
||||
return HasStats() ? max : string();
|
||||
}
|
||||
|
||||
bool MinIsExact() override {
|
||||
return !min_truncated;
|
||||
}
|
||||
|
||||
bool MaxIsExact() override {
|
||||
return !max_truncated;
|
||||
}
|
||||
};
|
||||
|
||||
class UUIDStatisticsState : public ColumnWriterStatistics {
|
||||
public:
|
||||
bool has_stats = false;
|
||||
data_t min[16] = {0};
|
||||
data_t max[16] = {0};
|
||||
|
||||
public:
|
||||
bool HasStats() override {
|
||||
return has_stats;
|
||||
}
|
||||
|
||||
string GetMin() override {
|
||||
return GetMinValue();
|
||||
}
|
||||
string GetMax() override {
|
||||
return GetMaxValue();
|
||||
}
|
||||
string GetMinValue() override {
|
||||
return HasStats() ? string(char_ptr_cast(min), 16) : string();
|
||||
}
|
||||
string GetMaxValue() override {
|
||||
return HasStats() ? string(char_ptr_cast(max), 16) : string();
|
||||
}
|
||||
};
|
||||
|
||||
class GeoStatisticsState final : public ColumnWriterStatistics {
|
||||
public:
|
||||
explicit GeoStatisticsState() : has_stats(false) {
|
||||
geo_stats.SetEmpty();
|
||||
}
|
||||
|
||||
bool has_stats;
|
||||
GeometryStatsData geo_stats;
|
||||
|
||||
public:
|
||||
void Update(const string_t &val) {
|
||||
geo_stats.Update(val);
|
||||
has_stats = true;
|
||||
}
|
||||
bool HasGeoStats() override {
|
||||
return has_stats;
|
||||
}
|
||||
optional_ptr<GeometryStatsData> GetGeoStats() override {
|
||||
return geo_stats;
|
||||
}
|
||||
void WriteGeoStats(duckdb_parquet::GeospatialStatistics &stats) override {
|
||||
const auto &types = geo_stats.types;
|
||||
const auto &bbox = geo_stats.extent;
|
||||
|
||||
if (bbox.HasXY()) {
|
||||
stats.__isset.bbox = true;
|
||||
stats.bbox.xmin = bbox.x_min;
|
||||
stats.bbox.xmax = bbox.x_max;
|
||||
stats.bbox.ymin = bbox.y_min;
|
||||
stats.bbox.ymax = bbox.y_max;
|
||||
|
||||
if (bbox.HasZ()) {
|
||||
stats.bbox.__isset.zmin = true;
|
||||
stats.bbox.__isset.zmax = true;
|
||||
stats.bbox.zmin = bbox.z_min;
|
||||
stats.bbox.zmax = bbox.z_max;
|
||||
}
|
||||
if (bbox.HasM()) {
|
||||
stats.bbox.__isset.mmin = true;
|
||||
stats.bbox.__isset.mmax = true;
|
||||
stats.bbox.mmin = bbox.m_min;
|
||||
stats.bbox.mmax = bbox.m_max;
|
||||
}
|
||||
}
|
||||
|
||||
stats.__isset.geospatial_types = true;
|
||||
stats.geospatial_types = types.ToWKBList();
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
115
external/duckdb/extension/parquet/include/writer/primitive_column_writer.hpp
vendored
Normal file
115
external/duckdb/extension/parquet/include/writer/primitive_column_writer.hpp
vendored
Normal file
@@ -0,0 +1,115 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/primitive_column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_writer.hpp"
|
||||
#include "writer/parquet_write_stats.hpp"
|
||||
#include "duckdb/common/serializer/memory_stream.hpp"
|
||||
#include "parquet_statistics.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
struct PageInformation {
|
||||
idx_t offset = 0;
|
||||
idx_t row_count = 0;
|
||||
idx_t empty_count = 0;
|
||||
idx_t estimated_page_size = 0;
|
||||
idx_t null_count = 0;
|
||||
};
|
||||
|
||||
struct PageWriteInformation {
|
||||
duckdb_parquet::PageHeader page_header;
|
||||
unique_ptr<MemoryStream> temp_writer;
|
||||
unique_ptr<ColumnWriterPageState> page_state;
|
||||
idx_t write_page_idx = 0;
|
||||
idx_t write_count = 0;
|
||||
idx_t max_write_count = 0;
|
||||
size_t compressed_size;
|
||||
data_ptr_t compressed_data;
|
||||
AllocatedData compressed_buf;
|
||||
};
|
||||
|
||||
class PrimitiveColumnWriterState : public ColumnWriterState {
|
||||
public:
|
||||
PrimitiveColumnWriterState(ParquetWriter &writer_p, duckdb_parquet::RowGroup &row_group, idx_t col_idx)
|
||||
: writer(writer_p), row_group(row_group), col_idx(col_idx) {
|
||||
page_info.emplace_back();
|
||||
}
|
||||
~PrimitiveColumnWriterState() override = default;
|
||||
|
||||
ParquetWriter &writer;
|
||||
duckdb_parquet::RowGroup &row_group;
|
||||
idx_t col_idx;
|
||||
vector<PageInformation> page_info;
|
||||
vector<PageWriteInformation> write_info;
|
||||
unique_ptr<ColumnWriterStatistics> stats_state;
|
||||
idx_t current_page = 0;
|
||||
|
||||
unique_ptr<ParquetBloomFilter> bloom_filter;
|
||||
};
|
||||
|
||||
//! Base class for writing non-compound types (ex. numerics, strings)
|
||||
class PrimitiveColumnWriter : public ColumnWriter {
|
||||
public:
|
||||
PrimitiveColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema, vector<string> schema_path,
|
||||
bool can_have_nulls);
|
||||
~PrimitiveColumnWriter() override = default;
|
||||
|
||||
//! We limit the uncompressed page size to 100MB
|
||||
//! The max size in Parquet is 2GB, but we choose a more conservative limit
|
||||
static constexpr const idx_t MAX_UNCOMPRESSED_PAGE_SIZE = 104857600ULL;
|
||||
//! Dictionary pages must be below 2GB. Unlike data pages, there's only one dictionary page.
|
||||
//! For this reason we go with a much higher, but still a conservative upper bound of 1GB;
|
||||
static constexpr const idx_t MAX_UNCOMPRESSED_DICT_PAGE_SIZE = 1073741824ULL;
|
||||
|
||||
public:
|
||||
unique_ptr<ColumnWriterState> InitializeWriteState(duckdb_parquet::RowGroup &row_group) override;
|
||||
void Prepare(ColumnWriterState &state, ColumnWriterState *parent, Vector &vector, idx_t count,
|
||||
bool vector_can_span_multiple_pages) override;
|
||||
void BeginWrite(ColumnWriterState &state) override;
|
||||
void Write(ColumnWriterState &state, Vector &vector, idx_t count) override;
|
||||
void FinalizeWrite(ColumnWriterState &state) override;
|
||||
|
||||
protected:
|
||||
static void WriteLevels(Allocator &allocator, WriteStream &temp_writer, const unsafe_vector<uint16_t> &levels,
|
||||
idx_t max_value, idx_t start_offset, idx_t count, optional_idx null_count = optional_idx());
|
||||
|
||||
virtual duckdb_parquet::Encoding::type GetEncoding(PrimitiveColumnWriterState &state);
|
||||
|
||||
void NextPage(PrimitiveColumnWriterState &state);
|
||||
void FlushPage(PrimitiveColumnWriterState &state);
|
||||
|
||||
//! Initializes the state used to track statistics during writing. Only used for scalar types.
|
||||
virtual unique_ptr<ColumnWriterStatistics> InitializeStatsState();
|
||||
|
||||
//! Initialize the writer for a specific page. Only used for scalar types.
|
||||
virtual unique_ptr<ColumnWriterPageState> InitializePageState(PrimitiveColumnWriterState &state, idx_t page_idx);
|
||||
|
||||
//! Flushes the writer for a specific page. Only used for scalar types.
|
||||
virtual void FlushPageState(WriteStream &temp_writer, ColumnWriterPageState *state);
|
||||
|
||||
//! Retrieves the row size of a vector at the specified location. Only used for scalar types.
|
||||
virtual idx_t GetRowSize(const Vector &vector, const idx_t index, const PrimitiveColumnWriterState &state) const;
|
||||
//! Writes a (subset of a) vector to the specified serializer. Only used for scalar types.
|
||||
virtual void WriteVector(WriteStream &temp_writer, ColumnWriterStatistics *stats, ColumnWriterPageState *page_state,
|
||||
Vector &vector, idx_t chunk_start, idx_t chunk_end) = 0;
|
||||
|
||||
virtual bool HasDictionary(PrimitiveColumnWriterState &state_p) {
|
||||
return false;
|
||||
}
|
||||
//! The number of elements in the dictionary
|
||||
virtual idx_t DictionarySize(PrimitiveColumnWriterState &state_p);
|
||||
void WriteDictionary(PrimitiveColumnWriterState &state, unique_ptr<MemoryStream> temp_writer, idx_t row_count);
|
||||
virtual void FlushDictionary(PrimitiveColumnWriterState &state, ColumnWriterStatistics *stats);
|
||||
|
||||
void SetParquetStatistics(PrimitiveColumnWriterState &state, duckdb_parquet::ColumnChunk &column);
|
||||
void RegisterToRowGroup(duckdb_parquet::RowGroup &row_group);
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
37
external/duckdb/extension/parquet/include/writer/struct_column_writer.hpp
vendored
Normal file
37
external/duckdb/extension/parquet/include/writer/struct_column_writer.hpp
vendored
Normal file
@@ -0,0 +1,37 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/struct_column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "column_writer.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class StructColumnWriter : public ColumnWriter {
|
||||
public:
|
||||
StructColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema, vector<string> schema_path_p,
|
||||
vector<unique_ptr<ColumnWriter>> child_writers_p, bool can_have_nulls)
|
||||
: ColumnWriter(writer, column_schema, std::move(schema_path_p), can_have_nulls) {
|
||||
child_writers = std::move(child_writers_p);
|
||||
}
|
||||
~StructColumnWriter() override = default;
|
||||
|
||||
public:
|
||||
unique_ptr<ColumnWriterState> InitializeWriteState(duckdb_parquet::RowGroup &row_group) override;
|
||||
bool HasAnalyze() override;
|
||||
void Analyze(ColumnWriterState &state, ColumnWriterState *parent, Vector &vector, idx_t count) override;
|
||||
void FinalizeAnalyze(ColumnWriterState &state) override;
|
||||
void Prepare(ColumnWriterState &state, ColumnWriterState *parent, Vector &vector, idx_t count,
|
||||
bool vector_can_span_multiple_pages) override;
|
||||
|
||||
void BeginWrite(ColumnWriterState &state) override;
|
||||
void Write(ColumnWriterState &state, Vector &vector, idx_t count) override;
|
||||
void FinalizeWrite(ColumnWriterState &state) override;
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
444
external/duckdb/extension/parquet/include/writer/templated_column_writer.hpp
vendored
Normal file
444
external/duckdb/extension/parquet/include/writer/templated_column_writer.hpp
vendored
Normal file
@@ -0,0 +1,444 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/templated_column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "writer/primitive_column_writer.hpp"
|
||||
#include "writer/parquet_write_operators.hpp"
|
||||
#include "parquet_dbp_encoder.hpp"
|
||||
#include "parquet_dlba_encoder.hpp"
|
||||
#include "parquet_rle_bp_encoder.hpp"
|
||||
#include "duckdb/common/primitive_dictionary.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
template <class SRC, class TGT, class OP = ParquetCastOperator, bool ALL_VALID>
|
||||
static void TemplatedWritePlain(Vector &col, ColumnWriterStatistics *stats, const idx_t chunk_start,
|
||||
const idx_t chunk_end, const ValidityMask &mask, WriteStream &ser) {
|
||||
static constexpr bool COPY_DIRECTLY_FROM_VECTOR = ALL_VALID && std::is_same<SRC, TGT>::value &&
|
||||
std::is_arithmetic<TGT>::value &&
|
||||
std::is_same<OP, ParquetCastOperator>::value;
|
||||
|
||||
const auto *const ptr = FlatVector::GetData<SRC>(col);
|
||||
|
||||
TGT local_write[STANDARD_VECTOR_SIZE];
|
||||
idx_t local_write_count = 0;
|
||||
|
||||
for (idx_t r = chunk_start; r < chunk_end; r++) {
|
||||
if (!ALL_VALID && !mask.RowIsValid(r)) {
|
||||
continue;
|
||||
}
|
||||
|
||||
TGT target_value = OP::template Operation<SRC, TGT>(ptr[r]);
|
||||
OP::template HandleStats<SRC, TGT>(stats, target_value);
|
||||
|
||||
if (COPY_DIRECTLY_FROM_VECTOR) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (std::is_arithmetic<TGT>::value) {
|
||||
local_write[local_write_count++] = target_value;
|
||||
if (local_write_count == STANDARD_VECTOR_SIZE) {
|
||||
ser.WriteData(data_ptr_cast(local_write), local_write_count * sizeof(TGT));
|
||||
local_write_count = 0;
|
||||
}
|
||||
} else {
|
||||
OP::template WriteToStream<SRC, TGT>(target_value, ser);
|
||||
}
|
||||
}
|
||||
|
||||
if (COPY_DIRECTLY_FROM_VECTOR) {
|
||||
ser.WriteData(const_data_ptr_cast(&ptr[chunk_start]), (chunk_end - chunk_start) * sizeof(TGT));
|
||||
return;
|
||||
}
|
||||
|
||||
if (std::is_arithmetic<TGT>::value) {
|
||||
ser.WriteData(data_ptr_cast(local_write), local_write_count * sizeof(TGT));
|
||||
}
|
||||
// Else we already wrote to stream
|
||||
}
|
||||
|
||||
template <class SRC, class TGT, class OP>
|
||||
class StandardColumnWriterState : public PrimitiveColumnWriterState {
|
||||
public:
|
||||
StandardColumnWriterState(ParquetWriter &writer, duckdb_parquet::RowGroup &row_group, idx_t col_idx)
|
||||
: PrimitiveColumnWriterState(writer, row_group, col_idx),
|
||||
dictionary(BufferAllocator::Get(writer.GetContext()),
|
||||
writer.DictionarySizeLimit().IsValid() ? writer.DictionarySizeLimit().GetIndex()
|
||||
: NumericCast<idx_t>(row_group.num_rows) / 5,
|
||||
writer.StringDictionaryPageSizeLimit()),
|
||||
encoding(duckdb_parquet::Encoding::PLAIN) {
|
||||
}
|
||||
~StandardColumnWriterState() override = default;
|
||||
|
||||
// analysis state for integer values for DELTA_BINARY_PACKED/DELTA_LENGTH_BYTE_ARRAY
|
||||
idx_t total_value_count = 0;
|
||||
idx_t total_string_size = 0;
|
||||
uint32_t key_bit_width = 0;
|
||||
|
||||
PrimitiveDictionary<SRC, TGT, OP> dictionary;
|
||||
duckdb_parquet::Encoding::type encoding;
|
||||
};
|
||||
|
||||
template <class SRC, class TGT, class OP>
|
||||
class StandardWriterPageState : public ColumnWriterPageState {
|
||||
public:
|
||||
explicit StandardWriterPageState(const idx_t total_value_count, const idx_t total_string_size,
|
||||
duckdb_parquet::Encoding::type encoding_p,
|
||||
const PrimitiveDictionary<SRC, TGT, OP> &dictionary_p)
|
||||
: encoding(encoding_p), dbp_initialized(false), dbp_encoder(total_value_count), dlba_initialized(false),
|
||||
dlba_encoder(total_value_count, total_string_size), bss_initialized(false),
|
||||
bss_encoder(total_value_count, sizeof(TGT)), dictionary(dictionary_p), dict_written_value(false),
|
||||
dict_bit_width(RleBpDecoder::ComputeBitWidth(dictionary.GetSize())), dict_encoder(dict_bit_width) {
|
||||
}
|
||||
duckdb_parquet::Encoding::type encoding;
|
||||
|
||||
bool dbp_initialized;
|
||||
DbpEncoder dbp_encoder;
|
||||
|
||||
bool dlba_initialized;
|
||||
DlbaEncoder dlba_encoder;
|
||||
|
||||
bool bss_initialized;
|
||||
BssEncoder bss_encoder;
|
||||
|
||||
const PrimitiveDictionary<SRC, TGT, OP> &dictionary;
|
||||
bool dict_written_value;
|
||||
uint32_t dict_bit_width;
|
||||
RleBpEncoder dict_encoder;
|
||||
};
|
||||
|
||||
template <class SRC, class TGT, class OP = ParquetCastOperator>
|
||||
class StandardColumnWriter : public PrimitiveColumnWriter {
|
||||
public:
|
||||
StandardColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema,
|
||||
vector<string> schema_path_p, // NOLINT
|
||||
bool can_have_nulls)
|
||||
: PrimitiveColumnWriter(writer, column_schema, std::move(schema_path_p), can_have_nulls) {
|
||||
}
|
||||
~StandardColumnWriter() override = default;
|
||||
|
||||
public:
|
||||
unique_ptr<ColumnWriterState> InitializeWriteState(duckdb_parquet::RowGroup &row_group) override {
|
||||
auto result = make_uniq<StandardColumnWriterState<SRC, TGT, OP>>(writer, row_group, row_group.columns.size());
|
||||
result->encoding = duckdb_parquet::Encoding::RLE_DICTIONARY;
|
||||
RegisterToRowGroup(row_group);
|
||||
return std::move(result);
|
||||
}
|
||||
|
||||
unique_ptr<ColumnWriterPageState> InitializePageState(PrimitiveColumnWriterState &state_p,
|
||||
idx_t page_idx) override {
|
||||
auto &state = state_p.Cast<StandardColumnWriterState<SRC, TGT, OP>>();
|
||||
const auto &page_info = state_p.page_info[page_idx];
|
||||
auto result = make_uniq<StandardWriterPageState<SRC, TGT, OP>>(
|
||||
page_info.row_count - (page_info.empty_count + page_info.null_count), state.total_string_size,
|
||||
state.encoding, state.dictionary);
|
||||
return std::move(result);
|
||||
}
|
||||
|
||||
void FlushPageState(WriteStream &temp_writer, ColumnWriterPageState *state_p) override {
|
||||
auto &page_state = state_p->Cast<StandardWriterPageState<SRC, TGT, OP>>();
|
||||
switch (page_state.encoding) {
|
||||
case duckdb_parquet::Encoding::DELTA_BINARY_PACKED:
|
||||
if (!page_state.dbp_initialized) {
|
||||
page_state.dbp_encoder.BeginWrite(temp_writer, 0);
|
||||
}
|
||||
page_state.dbp_encoder.FinishWrite(temp_writer);
|
||||
break;
|
||||
case duckdb_parquet::Encoding::RLE_DICTIONARY:
|
||||
D_ASSERT(page_state.dict_bit_width != 0);
|
||||
if (!page_state.dict_written_value) {
|
||||
// all values are null
|
||||
// just write the bit width
|
||||
temp_writer.Write<uint8_t>(page_state.dict_bit_width);
|
||||
return;
|
||||
}
|
||||
page_state.dict_encoder.FinishWrite(temp_writer);
|
||||
break;
|
||||
case duckdb_parquet::Encoding::DELTA_LENGTH_BYTE_ARRAY:
|
||||
if (!page_state.dlba_initialized) {
|
||||
page_state.dlba_encoder.BeginWrite(BufferAllocator::Get(writer.GetContext()), temp_writer,
|
||||
string_t(""));
|
||||
}
|
||||
page_state.dlba_encoder.FinishWrite(temp_writer);
|
||||
break;
|
||||
case duckdb_parquet::Encoding::BYTE_STREAM_SPLIT:
|
||||
if (!page_state.bss_initialized) {
|
||||
page_state.bss_encoder.BeginWrite(BufferAllocator::Get(writer.GetContext()));
|
||||
}
|
||||
page_state.bss_encoder.FinishWrite(temp_writer);
|
||||
break;
|
||||
case duckdb_parquet::Encoding::PLAIN:
|
||||
break;
|
||||
default:
|
||||
throw InternalException("Unknown encoding");
|
||||
}
|
||||
}
|
||||
|
||||
duckdb_parquet::Encoding::type GetEncoding(PrimitiveColumnWriterState &state_p) override {
|
||||
auto &state = state_p.Cast<StandardColumnWriterState<SRC, TGT, OP>>();
|
||||
return state.encoding;
|
||||
}
|
||||
|
||||
bool HasAnalyze() override {
|
||||
return true;
|
||||
}
|
||||
|
||||
void Analyze(ColumnWriterState &state_p, ColumnWriterState *parent, Vector &vector, idx_t count) override {
|
||||
auto &state = state_p.Cast<StandardColumnWriterState<SRC, TGT, OP>>();
|
||||
|
||||
auto data_ptr = FlatVector::GetData<SRC>(vector);
|
||||
idx_t vector_index = 0;
|
||||
|
||||
const bool check_parent_empty = parent && !parent->is_empty.empty();
|
||||
const idx_t parent_index = state.definition_levels.size();
|
||||
D_ASSERT(!check_parent_empty || parent_index < parent->is_empty.size());
|
||||
|
||||
const idx_t vcount =
|
||||
check_parent_empty ? parent->definition_levels.size() - state.definition_levels.size() : count;
|
||||
|
||||
const auto &validity = FlatVector::Validity(vector);
|
||||
|
||||
if (!check_parent_empty && validity.AllValid()) {
|
||||
// Fast path
|
||||
for (; vector_index < vcount; vector_index++) {
|
||||
const auto &src_value = data_ptr[vector_index];
|
||||
state.dictionary.Insert(src_value);
|
||||
state.total_value_count++;
|
||||
state.total_string_size += DlbaEncoder::GetStringSize(src_value);
|
||||
}
|
||||
} else {
|
||||
for (idx_t i = 0; i < vcount; i++) {
|
||||
if (check_parent_empty && parent->is_empty[parent_index + i]) {
|
||||
continue;
|
||||
}
|
||||
if (validity.RowIsValid(vector_index)) {
|
||||
const auto &src_value = data_ptr[vector_index];
|
||||
state.dictionary.Insert(src_value);
|
||||
state.total_value_count++;
|
||||
state.total_string_size += DlbaEncoder::GetStringSize(src_value);
|
||||
}
|
||||
vector_index++;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void FinalizeAnalyze(ColumnWriterState &state_p) override {
|
||||
const auto type = writer.GetType(SchemaIndex());
|
||||
|
||||
auto &state = state_p.Cast<StandardColumnWriterState<SRC, TGT, OP>>();
|
||||
if (state.dictionary.GetSize() == 0 || state.dictionary.IsFull()) {
|
||||
state.dictionary.Reset();
|
||||
if (writer.GetParquetVersion() == ParquetVersion::V1) {
|
||||
// Can't do the cool stuff for V1
|
||||
state.encoding = duckdb_parquet::Encoding::PLAIN;
|
||||
} else {
|
||||
// If we aren't doing dictionary encoding, these encodings are virtually always better than PLAIN
|
||||
switch (type) {
|
||||
case duckdb_parquet::Type::type::INT32:
|
||||
case duckdb_parquet::Type::type::INT64:
|
||||
state.encoding = duckdb_parquet::Encoding::DELTA_BINARY_PACKED;
|
||||
break;
|
||||
case duckdb_parquet::Type::type::BYTE_ARRAY:
|
||||
state.encoding = duckdb_parquet::Encoding::DELTA_LENGTH_BYTE_ARRAY;
|
||||
break;
|
||||
case duckdb_parquet::Type::type::FLOAT:
|
||||
case duckdb_parquet::Type::type::DOUBLE:
|
||||
state.encoding = duckdb_parquet::Encoding::BYTE_STREAM_SPLIT;
|
||||
break;
|
||||
default:
|
||||
state.encoding = duckdb_parquet::Encoding::PLAIN;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
state.key_bit_width = RleBpDecoder::ComputeBitWidth(state.dictionary.GetSize());
|
||||
}
|
||||
}
|
||||
|
||||
unique_ptr<ColumnWriterStatistics> InitializeStatsState() override {
|
||||
return OP::template InitializeStats<SRC, TGT>();
|
||||
}
|
||||
|
||||
bool HasDictionary(PrimitiveColumnWriterState &state_p) override {
|
||||
auto &state = state_p.Cast<StandardColumnWriterState<SRC, TGT, OP>>();
|
||||
return state.encoding == duckdb_parquet::Encoding::RLE_DICTIONARY;
|
||||
}
|
||||
|
||||
idx_t DictionarySize(PrimitiveColumnWriterState &state_p) override {
|
||||
auto &state = state_p.Cast<StandardColumnWriterState<SRC, TGT, OP>>();
|
||||
return state.dictionary.GetSize();
|
||||
}
|
||||
|
||||
void WriteVector(WriteStream &temp_writer, ColumnWriterStatistics *stats, ColumnWriterPageState *page_state_p,
|
||||
Vector &input_column, idx_t chunk_start, idx_t chunk_end) override {
|
||||
const auto &mask = FlatVector::Validity(input_column);
|
||||
if (mask.AllValid()) {
|
||||
WriteVectorInternal<true>(temp_writer, stats, page_state_p, input_column, chunk_start, chunk_end);
|
||||
} else {
|
||||
WriteVectorInternal<false>(temp_writer, stats, page_state_p, input_column, chunk_start, chunk_end);
|
||||
}
|
||||
}
|
||||
|
||||
void FlushDictionary(PrimitiveColumnWriterState &state_p, ColumnWriterStatistics *stats) override {
|
||||
auto &state = state_p.Cast<StandardColumnWriterState<SRC, TGT, OP>>();
|
||||
D_ASSERT(state.encoding == duckdb_parquet::Encoding::RLE_DICTIONARY);
|
||||
|
||||
if (writer.EnableBloomFilters()) {
|
||||
state.bloom_filter =
|
||||
make_uniq<ParquetBloomFilter>(state.dictionary.GetSize(), writer.BloomFilterFalsePositiveRatio());
|
||||
}
|
||||
|
||||
state.dictionary.IterateValues([&](const SRC &src_value, const TGT &tgt_value) {
|
||||
// update the statistics
|
||||
OP::template HandleStats<SRC, TGT>(stats, tgt_value);
|
||||
if (state.bloom_filter) {
|
||||
// update the bloom filter
|
||||
auto hash = OP::template XXHash64<SRC, TGT>(tgt_value);
|
||||
state.bloom_filter->FilterInsert(hash);
|
||||
}
|
||||
});
|
||||
|
||||
// flush the dictionary page and add it to the to-be-written pages
|
||||
WriteDictionary(state, state.dictionary.GetTargetMemoryStream(), state.dictionary.GetSize());
|
||||
// bloom filter will be queued for writing in ParquetWriter::BufferBloomFilter one level up
|
||||
}
|
||||
|
||||
idx_t GetRowSize(const Vector &vector, const idx_t index,
|
||||
const PrimitiveColumnWriterState &state_p) const override {
|
||||
auto &state = state_p.Cast<StandardColumnWriterState<SRC, TGT, OP>>();
|
||||
if (state.encoding == duckdb_parquet::Encoding::RLE_DICTIONARY) {
|
||||
return (state.key_bit_width + 7) / 8;
|
||||
} else {
|
||||
return OP::template GetRowSize<SRC, TGT>(vector, index);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
template <bool ALL_VALID>
|
||||
void WriteVectorInternal(WriteStream &temp_writer, ColumnWriterStatistics *stats,
|
||||
ColumnWriterPageState *page_state_p, Vector &input_column, idx_t chunk_start,
|
||||
idx_t chunk_end) {
|
||||
auto &page_state = page_state_p->Cast<StandardWriterPageState<SRC, TGT, OP>>();
|
||||
|
||||
const auto &mask = FlatVector::Validity(input_column);
|
||||
const auto *data_ptr = FlatVector::GetData<SRC>(input_column);
|
||||
|
||||
switch (page_state.encoding) {
|
||||
case duckdb_parquet::Encoding::RLE_DICTIONARY: {
|
||||
idx_t r = chunk_start;
|
||||
if (!page_state.dict_written_value) {
|
||||
// find first non-null value
|
||||
for (; r < chunk_end; r++) {
|
||||
if (!mask.RowIsValid(r)) {
|
||||
continue;
|
||||
}
|
||||
// write the bit-width as a one-byte entry and initialize writer
|
||||
temp_writer.Write<uint8_t>(page_state.dict_bit_width);
|
||||
page_state.dict_encoder.BeginWrite();
|
||||
page_state.dict_written_value = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
for (; r < chunk_end; r++) {
|
||||
if (!ALL_VALID && !mask.RowIsValid(r)) {
|
||||
continue;
|
||||
}
|
||||
const auto &src_value = data_ptr[r];
|
||||
const auto value_index = page_state.dictionary.GetIndex(src_value);
|
||||
page_state.dict_encoder.WriteValue(temp_writer, value_index);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case duckdb_parquet::Encoding::DELTA_BINARY_PACKED: {
|
||||
idx_t r = chunk_start;
|
||||
if (!page_state.dbp_initialized) {
|
||||
// find first non-null value
|
||||
for (; r < chunk_end; r++) {
|
||||
if (!mask.RowIsValid(r)) {
|
||||
continue;
|
||||
}
|
||||
const TGT target_value = OP::template Operation<SRC, TGT>(data_ptr[r]);
|
||||
OP::template HandleStats<SRC, TGT>(stats, target_value);
|
||||
page_state.dbp_encoder.BeginWrite(temp_writer, target_value);
|
||||
page_state.dbp_initialized = true;
|
||||
r++; // skip over
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
for (; r < chunk_end; r++) {
|
||||
if (!ALL_VALID && !mask.RowIsValid(r)) {
|
||||
continue;
|
||||
}
|
||||
const TGT target_value = OP::template Operation<SRC, TGT>(data_ptr[r]);
|
||||
OP::template HandleStats<SRC, TGT>(stats, target_value);
|
||||
page_state.dbp_encoder.WriteValue(temp_writer, target_value);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case duckdb_parquet::Encoding::DELTA_LENGTH_BYTE_ARRAY: {
|
||||
idx_t r = chunk_start;
|
||||
if (!page_state.dlba_initialized) {
|
||||
// find first non-null value
|
||||
for (; r < chunk_end; r++) {
|
||||
if (!mask.RowIsValid(r)) {
|
||||
continue;
|
||||
}
|
||||
const TGT target_value = OP::template Operation<SRC, TGT>(data_ptr[r]);
|
||||
OP::template HandleStats<SRC, TGT>(stats, target_value);
|
||||
page_state.dlba_encoder.BeginWrite(BufferAllocator::Get(writer.GetContext()), temp_writer,
|
||||
target_value);
|
||||
page_state.dlba_initialized = true;
|
||||
r++; // skip over
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
for (; r < chunk_end; r++) {
|
||||
if (!ALL_VALID && !mask.RowIsValid(r)) {
|
||||
continue;
|
||||
}
|
||||
const TGT target_value = OP::template Operation<SRC, TGT>(data_ptr[r]);
|
||||
OP::template HandleStats<SRC, TGT>(stats, target_value);
|
||||
page_state.dlba_encoder.WriteValue(temp_writer, target_value);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case duckdb_parquet::Encoding::BYTE_STREAM_SPLIT: {
|
||||
if (!page_state.bss_initialized) {
|
||||
page_state.bss_encoder.BeginWrite(BufferAllocator::Get(writer.GetContext()));
|
||||
page_state.bss_initialized = true;
|
||||
}
|
||||
for (idx_t r = chunk_start; r < chunk_end; r++) {
|
||||
if (!ALL_VALID && !mask.RowIsValid(r)) {
|
||||
continue;
|
||||
}
|
||||
const TGT target_value = OP::template Operation<SRC, TGT>(data_ptr[r]);
|
||||
OP::template HandleStats<SRC, TGT>(stats, target_value);
|
||||
page_state.bss_encoder.WriteValue(target_value);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case duckdb_parquet::Encoding::PLAIN: {
|
||||
D_ASSERT(page_state.encoding == duckdb_parquet::Encoding::PLAIN);
|
||||
if (mask.AllValid()) {
|
||||
TemplatedWritePlain<SRC, TGT, OP, true>(input_column, stats, chunk_start, chunk_end, mask, temp_writer);
|
||||
} else {
|
||||
TemplatedWritePlain<SRC, TGT, OP, false>(input_column, stats, chunk_start, chunk_end, mask,
|
||||
temp_writer);
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw InternalException("Unknown encoding");
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
30
external/duckdb/extension/parquet/include/writer/variant_column_writer.hpp
vendored
Normal file
30
external/duckdb/extension/parquet/include/writer/variant_column_writer.hpp
vendored
Normal file
@@ -0,0 +1,30 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// writer/variant_column_writer.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "struct_column_writer.hpp"
|
||||
#include "duckdb/planner/expression/bound_function_expression.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class VariantColumnWriter : public StructColumnWriter {
|
||||
public:
|
||||
VariantColumnWriter(ParquetWriter &writer, const ParquetColumnSchema &column_schema, vector<string> schema_path_p,
|
||||
vector<unique_ptr<ColumnWriter>> child_writers_p, bool can_have_nulls)
|
||||
: StructColumnWriter(writer, column_schema, std::move(schema_path_p), std::move(child_writers_p),
|
||||
can_have_nulls) {
|
||||
}
|
||||
~VariantColumnWriter() override = default;
|
||||
|
||||
public:
|
||||
static ScalarFunction GetTransformFunction();
|
||||
static LogicalType TransformTypedValueRecursive(const LogicalType &type);
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
33
external/duckdb/extension/parquet/include/zstd_file_system.hpp
vendored
Normal file
33
external/duckdb/extension/parquet/include/zstd_file_system.hpp
vendored
Normal file
@@ -0,0 +1,33 @@
|
||||
//===----------------------------------------------------------------------===//
|
||||
// DuckDB
|
||||
//
|
||||
// zstd_file_system.hpp
|
||||
//
|
||||
//
|
||||
//===----------------------------------------------------------------------===//
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "duckdb.hpp"
|
||||
#include "duckdb/common/compressed_file_system.hpp"
|
||||
|
||||
namespace duckdb {
|
||||
|
||||
class ZStdFileSystem : public CompressedFileSystem {
|
||||
public:
|
||||
unique_ptr<FileHandle> OpenCompressedFile(QueryContext context, unique_ptr<FileHandle> handle, bool write) override;
|
||||
|
||||
std::string GetName() const override {
|
||||
return "ZStdFileSystem";
|
||||
}
|
||||
|
||||
unique_ptr<StreamWrapper> CreateStream() override;
|
||||
idx_t InBufferSize() override;
|
||||
idx_t OutBufferSize() override;
|
||||
|
||||
static int64_t DefaultCompressionLevel();
|
||||
static int64_t MinimumCompressionLevel();
|
||||
static int64_t MaximumCompressionLevel();
|
||||
};
|
||||
|
||||
} // namespace duckdb
|
||||
Reference in New Issue
Block a user