Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
27 commits
Select commit Hold shift + click to select a range
0447cb5
apply patches
Tishj Jul 14, 2025
687da17
update dependencies
Tishj Jul 14, 2025
363a96f
bump duckdb submodule pointer
Tmonster Jul 25, 2025
777e125
start using extension util, need to bump other extensions though
Tmonster Jul 25, 2025
3f676ce
bump extension ci tools as well
Tmonster Jul 25, 2025
3ffc31a
update extension_config.cmake as wellg
Tmonster Jul 25, 2025
ed621b3
does not compile. Need to modify avro bind function to expect manifes…
Tmonster Jul 28, 2025
3c8bd19
use main duckdb in distribution pipeline
Tmonster Jul 28, 2025
f74b39a
Merge remote-tracking branch 'upstream/main' into use_duckdb_v1.4
Tmonster Jul 29, 2025
b9bb55d
Merge remote-tracking branch 'upstream/v1.4-andium' into use_duckdb_v1.4
Tmonster Aug 6, 2025
cef2736
it compiles
Tmonster Aug 6, 2025
6ee0911
make format-fix
Tmonster Aug 6, 2025
dace7c3
remove bind function
Tmonster Aug 6, 2025
9c358b8
Merge remote-tracking branch 'upstream/main' into use_duckdb_v1.4
Tmonster Aug 13, 2025
fe45396
update duckdb submodule
Tmonster Aug 13, 2025
6442789
update avro as well
Tmonster Aug 14, 2025
b714c77
update ducklake, update tests
Tmonster Aug 14, 2025
326cadb
skip iceberg to ducklake
Tmonster Aug 14, 2025
68ee2a4
Merge pull request #408 from Tmonster/use_duckdb_v1.4
Tmonster Aug 15, 2025
760688c
Merge remote-tracking branch 'upstream/main' into merge_v1.3_ossivalis
Tmonster Aug 18, 2025
dac6bfb
Merge pull request #414 from Tmonster/merge_v1.3_ossivalis
Tmonster Aug 20, 2025
f8c4ba3
update duckdb to latest main (v1.4) and fix tests to reflect new logg…
Tmonster Aug 27, 2025
3e2abcf
fix one failing test
Tmonster Aug 27, 2025
fc2275a
Merge pull request #433 from Tmonster/update-v1.4-andium
Tmonster Aug 28, 2025
fddd525
Merge remote-tracking branch 'upstream/main' into test_v1.4
Tmonster Aug 29, 2025
536ff8c
changes needed to build with main
Tmonster Aug 29, 2025
2335ff7
Merge pull request #437 from Tmonster/update_v1.4_andium
Tmonster Aug 29, 2025
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions .github/workflows/MainDistributionPipeline.yml
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ jobs:
uses: duckdb/extension-ci-tools/.github/workflows/_extension_distribution.yml@main
with:
extension_name: iceberg
duckdb_version: v1.3.2
duckdb_version: main
ci_tools_version: main
exclude_archs: 'windows_amd64_mingw'
extra_toolchains: 'python3'
Expand All @@ -29,7 +29,7 @@ jobs:
secrets: inherit
with:
extension_name: iceberg
duckdb_version: v1.3.2
duckdb_version: main
ci_tools_version: main
exclude_archs: 'windows_amd64_mingw'
deploy_latest: ${{ startsWith(github.ref, 'refs/tags/v') || github.ref == 'refs/heads/main' }}
Expand Down
2 changes: 1 addition & 1 deletion duckdb
Submodule duckdb updated 2292 files
18 changes: 11 additions & 7 deletions extension_config.cmake
Original file line number Diff line number Diff line change
Expand Up @@ -12,13 +12,15 @@ duckdb_extension_load(icu)
duckdb_extension_load(ducklake
LOAD_TESTS
GIT_URL https://github.com/duckdb/ducklake
GIT_TAG 9cc2d903c51d360ff3fc6afb10cf38f8eac2e25b
GIT_TAG d2392c36f33151cf5cdd7d006375b0b669bd44ac
APPLY_PATCHES
)

duckdb_extension_load(avro
LOAD_TESTS
GIT_URL https://github.com/duckdb/duckdb-avro
GIT_TAG 180e41e8ad13b8712d207785a6bca0aa39341040
LOAD_TESTS
GIT_URL https://github.com/duckdb/duckdb-avro
GIT_TAG 0d7af391bd0aa201b2bdcfb994b7a575ad810155
APPLY_PATCHES
)

if (NOT EMSCRIPTEN)
Expand All @@ -27,14 +29,16 @@ if (NOT MINGW)
duckdb_extension_load(aws
LOAD_TESTS
GIT_URL https://github.com/duckdb/duckdb-aws
GIT_TAG main
GIT_TAG 880da03202acc973d6ee7f3a0423dae5a6dea83b
APPLY_PATCHES
)
endif ()
endif()

duckdb_extension_load(httpfs
GIT_URL https://github.com/duckdb/duckdb-httpfs
GIT_TAG e9bb99189d93c8ce6e0755907c38d283c963ae61
GIT_URL https://github.com/duckdb/duckdb-httpfs
GIT_TAG cb5b2825eff68fc91f47e917ba88bf2ed84c2dd3
INCLUDE_DIR extension/httpfs/include
APPLY_PATCHES
)

10 changes: 9 additions & 1 deletion src/avro_scan.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
#include "avro_scan.hpp"
#include "iceberg_extension.hpp"
#include "duckdb/main/extension_helper.hpp"
#include "duckdb/main/extension/extension_loader.hpp"
#include "duckdb/main/database.hpp"

#include "iceberg_multi_file_reader.hpp"
Expand All @@ -13,7 +14,14 @@ AvroScan::AvroScan(const string &scan_name, ClientContext &context, const string
auto &instance = DatabaseInstance::GetDatabase(context);
ExtensionHelper::AutoLoadExtension(instance, "avro");

auto &avro_scan_entry = ExtensionUtil::GetTableFunction(instance, "read_avro");
auto &system_catalog = Catalog::GetSystemCatalog(instance);
auto data = CatalogTransaction::GetSystemTransaction(instance);
auto &schema = system_catalog.GetSchema(data, DEFAULT_SCHEMA);
auto catalog_entry = schema.GetEntry(data, CatalogType::TABLE_FUNCTION_ENTRY, "read_avro");
if (!catalog_entry) {
throw InvalidInputException("Function with name \"read_avro\" not found!");
}
auto &avro_scan_entry = catalog_entry->Cast<TableFunctionCatalogEntry>();
avro_scan = avro_scan_entry.functions.functions[0];
avro_scan->get_multi_file_reader = IcebergAvroMultiFileReader::CreateInstance;

Expand Down
2 changes: 2 additions & 0 deletions src/common/api_utils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,8 @@
#include "duckdb/common/exception/http_exception.hpp"
#include "duckdb/common/string_util.hpp"
#include "duckdb/main/client_data.hpp"
#include "duckdb/main/database.hpp"
#include "duckdb/main/extension/extension_loader.hpp"

#include <sys/stat.h>

Expand Down
39 changes: 15 additions & 24 deletions src/iceberg_extension.cpp
Original file line number Diff line number Diff line change
@@ -1,5 +1,3 @@
#define DUCKDB_EXTENSION_MAIN

#include "iceberg_extension.hpp"
#include "storage/irc_catalog.hpp"
#include "storage/irc_transaction_manager.hpp"
Expand All @@ -9,7 +7,7 @@
#include "duckdb/common/exception/http_exception.hpp"
#include "duckdb/common/string_util.hpp"
#include "duckdb/function/scalar_function.hpp"
#include "duckdb/main/extension_util.hpp"
#include "duckdb/main/extension/extension_loader.hpp"
#include "duckdb/catalog/catalog_entry/macro_catalog_entry.hpp"
#include "duckdb/catalog/default/default_functions.hpp"
#include "duckdb/storage/storage_extension.hpp"
Expand All @@ -23,8 +21,8 @@

namespace duckdb {

static unique_ptr<TransactionManager> CreateTransactionManager(StorageExtensionInfo *storage_info, AttachedDatabase &db,
Catalog &catalog) {
static unique_ptr<TransactionManager> CreateTransactionManager(optional_ptr<StorageExtensionInfo> storage_info,
AttachedDatabase &db, Catalog &catalog) {
auto &ic_catalog = catalog.Cast<IRCatalog>();
return make_uniq<ICTransactionManager>(db, ic_catalog);
}
Expand All @@ -37,8 +35,10 @@ class IRCStorageExtension : public StorageExtension {
}
};

static void LoadInternal(DatabaseInstance &instance) {
static void LoadInternal(ExtensionLoader &loader) {
auto &instance = loader.GetDatabaseInstance();
ExtensionHelper::AutoLoadExtension(instance, "parquet");

if (!instance.ExtensionIsLoaded("parquet")) {
throw MissingExtensionException("The iceberg extension requires the parquet extension to be loaded!");
}
Expand All @@ -51,33 +51,33 @@ static void LoadInternal(DatabaseInstance &instance) {
LogicalType::BOOLEAN, Value::BOOLEAN(false));

// Iceberg Table Functions
for (auto &fun : IcebergFunctions::GetTableFunctions(instance)) {
ExtensionUtil::RegisterFunction(instance, std::move(fun));
for (auto &fun : IcebergFunctions::GetTableFunctions(loader)) {
loader.RegisterFunction(std::move(fun));
}

// Iceberg Scalar Functions
for (auto &fun : IcebergFunctions::GetScalarFunctions()) {
ExtensionUtil::RegisterFunction(instance, fun);
loader.RegisterFunction(fun);
}

SecretType secret_type;
secret_type.name = "iceberg";
secret_type.deserializer = KeyValueSecret::Deserialize<KeyValueSecret>;
secret_type.default_provider = "config";

ExtensionUtil::RegisterSecretType(instance, secret_type);
loader.RegisterSecretType(secret_type);
CreateSecretFunction secret_function = {"iceberg", "config", OAuth2Authorization::CreateCatalogSecretFunction};
OAuth2Authorization::SetCatalogSecretParameters(secret_function);
ExtensionUtil::RegisterFunction(instance, secret_function);
loader.RegisterFunction(secret_function);

auto &log_manager = instance.GetLogManager();
log_manager.RegisterLogType(make_uniq<IcebergLogType>());

config.storage_extensions["iceberg"] = make_uniq<IRCStorageExtension>();
}

void IcebergExtension::Load(DuckDB &db) {
LoadInternal(*db.instance);
void IcebergExtension::Load(ExtensionLoader &loader) {
LoadInternal(loader);
}
string IcebergExtension::Name() {
return "iceberg";
Expand All @@ -86,16 +86,7 @@ string IcebergExtension::Name() {
} // namespace duckdb

extern "C" {

DUCKDB_EXTENSION_API void iceberg_init(duckdb::DatabaseInstance &db) {
LoadInternal(db);
}

DUCKDB_EXTENSION_API const char *iceberg_version() {
return duckdb::DuckDB::LibraryVersion();
DUCKDB_CPP_EXTENSION_ENTRY(iceberg, loader) {
LoadInternal(loader);
}
}

#ifndef DUCKDB_EXTENSION_MAIN
#error DUCKDB_EXTENSION_MAIN not defined
#endif
7 changes: 4 additions & 3 deletions src/iceberg_functions.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -5,13 +5,14 @@
#include "duckdb/function/cast/default_casts.hpp"
#include "duckdb/parser/expression/function_expression.hpp"

namespace duckdb {
class ExtensionLoader;

vector<TableFunctionSet> IcebergFunctions::GetTableFunctions(DatabaseInstance &instance) {
namespace duckdb {
vector<TableFunctionSet> IcebergFunctions::GetTableFunctions(ExtensionLoader &loader) {
vector<TableFunctionSet> functions;

functions.push_back(std::move(GetIcebergSnapshotsFunction()));
functions.push_back(std::move(GetIcebergScanFunction(instance)));
functions.push_back(std::move(GetIcebergScanFunction(loader)));
functions.push_back(std::move(GetIcebergMetadataFunction()));
functions.push_back(std::move(GetIcebergToDuckLakeFunction()));

Expand Down
3 changes: 1 addition & 2 deletions src/iceberg_functions/iceberg_avro_multi_file_reader.cpp
Original file line number Diff line number Diff line change
@@ -1,7 +1,6 @@
#include "iceberg_avro_multi_file_reader.hpp"

#include "duckdb/common/exception.hpp"
#include "duckdb/main/extension_util.hpp"
#include "duckdb/parallel/thread_context.hpp"

namespace duckdb {
Expand All @@ -13,7 +12,7 @@ unique_ptr<MultiFileReader> IcebergAvroMultiFileReader::CreateInstance(const Tab

shared_ptr<MultiFileList> IcebergAvroMultiFileReader::CreateFileList(ClientContext &context,
const vector<string> &paths,
FileGlobOptions options) {
const FileGlobInput &glob_input) {

vector<OpenFileInfo> open_files;
for (auto &path : paths) {
Expand Down
10 changes: 8 additions & 2 deletions src/iceberg_functions/iceberg_multi_file_list.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@
#include "duckdb/common/exception.hpp"
#include "duckdb/execution/execution_context.hpp"
#include "duckdb/parallel/thread_context.hpp"
#include "duckdb/main/extension_util.hpp"
#include "duckdb/parser/tableref/table_function_ref.hpp"
#include "duckdb/parser/expression/constant_expression.hpp"
#include "duckdb/planner/filter/struct_filter.hpp"
Expand Down Expand Up @@ -656,7 +655,14 @@ void IcebergMultiFileList::ScanDeleteFile(const IcebergManifestEntry &entry,
auto &instance = DatabaseInstance::GetDatabase(context);
//! FIXME: delete files could also be made without row_ids,
//! in which case we need to rely on the `'schema.column-mapping.default'` property just like data files do.
auto &parquet_scan_entry = ExtensionUtil::GetTableFunction(instance, "parquet_scan");
auto &system_catalog = Catalog::GetSystemCatalog(instance);
auto data = CatalogTransaction::GetSystemTransaction(instance);
auto &schema = system_catalog.GetSchema(data, DEFAULT_SCHEMA);
auto catalog_entry = schema.GetEntry(data, CatalogType::TABLE_FUNCTION_ENTRY, "parquet_scan");
if (!catalog_entry) {
throw InvalidInputException("Function with name \"parquet_scan\" not found!");
}
auto &parquet_scan_entry = catalog_entry->Cast<TableFunctionCatalogEntry>();
auto &parquet_scan = parquet_scan_entry.functions.functions[0];

// Prepare the inputs for the bind
Expand Down
3 changes: 1 addition & 2 deletions src/iceberg_functions/iceberg_multi_file_reader.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,6 @@
#include "duckdb/catalog/catalog_entry/table_function_catalog_entry.hpp"
#include "duckdb/common/exception.hpp"
#include "duckdb/execution/execution_context.hpp"
#include "duckdb/main/extension_util.hpp"
#include "duckdb/parallel/thread_context.hpp"
#include "duckdb/parser/tableref/table_function_ref.hpp"
#include "duckdb/parser/expression/constant_expression.hpp"
Expand All @@ -31,7 +30,7 @@ unique_ptr<MultiFileReader> IcebergMultiFileReader::CreateInstance(const TableFu
}

shared_ptr<MultiFileList> IcebergMultiFileReader::CreateFileList(ClientContext &context, const vector<string> &paths,
FileGlobOptions) {
const FileGlobInput &glob_input) {
if (paths.size() != 1) {
throw BinderException("'iceberg_scan' only supports single path as input");
}
Expand Down
6 changes: 3 additions & 3 deletions src/iceberg_functions/iceberg_scan.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
#include "duckdb/planner/operator/logical_comparison_join.hpp"
#include "duckdb/common/file_opener.hpp"
#include "duckdb/common/file_system.hpp"
#include "duckdb/main/extension_util.hpp"
#include "duckdb/main/extension/extension_loader.hpp"
#include "iceberg_metadata.hpp"
#include "iceberg_utils.hpp"
#include "iceberg_multi_file_reader.hpp"
Expand All @@ -41,11 +41,11 @@ static void AddNamedParameters(TableFunction &fun) {
fun.named_parameters["snapshot_from_id"] = LogicalType::UBIGINT;
}

TableFunctionSet IcebergFunctions::GetIcebergScanFunction(DatabaseInstance &instance) {
TableFunctionSet IcebergFunctions::GetIcebergScanFunction(ExtensionLoader &loader) {
// The iceberg_scan function is constructed by grabbing the parquet scan from the Catalog, then injecting the
// IcebergMultiFileReader into it to create a Iceberg-based multi file read

auto &parquet_scan = ExtensionUtil::GetTableFunction(instance, "parquet_scan");
auto &parquet_scan = loader.GetTableFunction("parquet_scan");
auto parquet_scan_copy = parquet_scan.functions;

for (auto &function : parquet_scan_copy.functions) {
Expand Down
4 changes: 2 additions & 2 deletions src/iceberg_functions/iceberg_snapshots.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -91,8 +91,8 @@ static void IcebergSnapshotsFunction(ClientContext &context, TableFunctionInput
}

auto &snapshot = it->second;
FlatVector::GetData<int64_t>(output.data[0])[i] = snapshot.sequence_number;
FlatVector::GetData<int64_t>(output.data[1])[i] = snapshot.snapshot_id;
FlatVector::GetData<uint64_t>(output.data[0])[i] = snapshot.sequence_number;
FlatVector::GetData<uint64_t>(output.data[1])[i] = snapshot.snapshot_id;
FlatVector::GetData<timestamp_t>(output.data[2])[i] = snapshot.timestamp_ms;
string_t manifest_string_t = StringVector::AddString(output.data[3], string_t(snapshot.manifest_list));
FlatVector::GetData<string_t>(output.data[3])[i] = manifest_string_t;
Expand Down
1 change: 0 additions & 1 deletion src/include/avro_scan.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,6 @@
#include "duckdb/catalog/catalog_entry/table_function_catalog_entry.hpp"
#include "duckdb/common/exception.hpp"
#include "duckdb/execution/execution_context.hpp"
#include "duckdb/main/extension_util.hpp"
#include "duckdb/parallel/thread_context.hpp"
#include "duckdb/parser/tableref/table_function_ref.hpp"
#include "duckdb/parser/expression/constant_expression.hpp"
Expand Down
2 changes: 1 addition & 1 deletion src/include/iceberg_avro_multi_file_reader.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ namespace duckdb {

struct IcebergAvroMultiFileReader : public MultiFileReader {
shared_ptr<MultiFileList> CreateFileList(ClientContext &context, const vector<string> &paths,
FileGlobOptions options) override;
const FileGlobInput &glob_input) override;

static unique_ptr<MultiFileReader> CreateInstance(const TableFunction &table);
};
Expand Down
2 changes: 1 addition & 1 deletion src/include/iceberg_extension.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@ namespace duckdb {

class IcebergExtension : public Extension {
public:
void Load(DuckDB &db) override;
void Load(ExtensionLoader &db) override;
string Name() override;
};

Expand Down
5 changes: 3 additions & 2 deletions src/include/iceberg_functions.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -13,15 +13,16 @@
#include "duckdb/parser/parsed_data/create_table_function_info.hpp"

namespace duckdb {
class ExtensionLoader;

class IcebergFunctions {
public:
static vector<TableFunctionSet> GetTableFunctions(DatabaseInstance &instance);
static vector<TableFunctionSet> GetTableFunctions(ExtensionLoader &loader);
static vector<ScalarFunction> GetScalarFunctions();

private:
static TableFunctionSet GetIcebergSnapshotsFunction();
static TableFunctionSet GetIcebergScanFunction(DatabaseInstance &instance);
static TableFunctionSet GetIcebergScanFunction(ExtensionLoader &instance);
static TableFunctionSet GetIcebergMetadataFunction();
static TableFunctionSet GetIcebergToDuckLakeFunction();
};
Expand Down
2 changes: 1 addition & 1 deletion src/include/iceberg_multi_file_reader.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ struct IcebergMultiFileReader : public MultiFileReader {

public:
shared_ptr<MultiFileList> CreateFileList(ClientContext &context, const vector<string> &paths,
FileGlobOptions options) override;
const FileGlobInput &glob_input) override;
bool Bind(MultiFileOptions &options, MultiFileList &files, vector<LogicalType> &return_types, vector<string> &names,
MultiFileReaderBindData &bind_data) override;
void BindOptions(MultiFileOptions &options, MultiFileList &files, vector<LogicalType> &return_types,
Expand Down
1 change: 1 addition & 0 deletions src/include/metadata/iceberg_manifest.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@

#include "duckdb/function/copy_function.hpp"
#include "duckdb/execution/execution_context.hpp"
#include "duckdb/main/extension/extension_loader.hpp"
#include "duckdb/parallel/thread_context.hpp"
#include "duckdb/common/insertion_order_preserving_map.hpp"

Expand Down
7 changes: 5 additions & 2 deletions src/include/storage/iceberg_insert.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@

#pragma once

#include "duckdb/execution/physical_plan_generator.hpp"
#include "duckdb/execution/operator/persistent/physical_copy_to_file.hpp"
#include "duckdb/execution/physical_operator.hpp"
#include "duckdb/common/index_vector.hpp"
Expand All @@ -31,9 +32,11 @@ struct IcebergCopyInput {
class IcebergInsert : public PhysicalOperator {
public:
//! INSERT INTO
IcebergInsert(LogicalOperator &op, TableCatalogEntry &table, physical_index_vector_t<idx_t> column_index_map);
IcebergInsert(PhysicalPlan &physical_plan, LogicalOperator &op, TableCatalogEntry &table,
physical_index_vector_t<idx_t> column_index_map);
//! CREATE TABLE AS
IcebergInsert(LogicalOperator &op, SchemaCatalogEntry &schema, unique_ptr<BoundCreateTableInfo> info);
IcebergInsert(PhysicalPlan &physical_plan, LogicalOperator &op, SchemaCatalogEntry &schema,
unique_ptr<BoundCreateTableInfo> info);

//! The table to insert into
optional_ptr<TableCatalogEntry> table;
Expand Down
Loading
Loading