Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
9 changes: 6 additions & 3 deletions src/duckdb/extension/icu/icu-makedate.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2,13 +2,12 @@
#include "duckdb/common/operator/cast_operators.hpp"
#include "duckdb/common/operator/subtract.hpp"
#include "duckdb/common/types/date.hpp"
#include "duckdb/common/types/time.hpp"
#include "duckdb/common/types/timestamp.hpp"
#include "duckdb/common/vector_operations/senary_executor.hpp"
#include "duckdb/common/vector_operations/septenary_executor.hpp"
#include "duckdb/function/cast/cast_function_set.hpp"
#include "duckdb/main/extension/extension_loader.hpp"
#include "duckdb/parser/parsed_data/create_scalar_function_info.hpp"
#include "duckdb/main/settings.hpp"
#include "include/icu-casts.hpp"
#include "include/icu-datefunc.hpp"
#include "include/icu-datetrunc.hpp"
Expand Down Expand Up @@ -57,6 +56,10 @@ BoundCastInfo ICUMakeDate::BindCastToDate(BindCastInput &input, const LogicalTyp
if (!input.context) {
throw InternalException("Missing context for TIMESTAMPTZ to DATE cast.");
}
if (DBConfig::GetSetting<DisableTimestamptzCastsSetting>(*input.context)) {
throw BinderException("Casting from TIMESTAMP WITH TIME ZONE to DATE without an explicit time zone "
"has been disabled - use \"AT TIME ZONE ...\"");
}

auto cast_data = make_uniq<CastData>(make_uniq<BindData>(*input.context));

Expand All @@ -80,7 +83,7 @@ struct ICUMakeTimestampTZFunc : public ICUDateFunc {
ss -= secs;
ss *= Interval::MSECS_PER_SEC;
const auto millis = int32_t(ss);
int64_t micros = std::round((ss - millis) * Interval::MICROS_PER_MSEC);
int64_t micros = LossyNumericCast<int64_t, double>(std::round((ss - millis) * Interval::MICROS_PER_MSEC));

calendar->set(UCAL_YEAR, year);
calendar->set(UCAL_MONTH, month);
Expand Down
11 changes: 7 additions & 4 deletions src/duckdb/src/common/encryption_functions.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,9 @@ idx_t EncryptionNonce::size() const {
EncryptionEngine::EncryptionEngine() {
}

EncryptionEngine::~EncryptionEngine() {
}

const_data_ptr_t EncryptionEngine::GetKeyFromCache(DatabaseInstance &db, const string &key_name) {
auto &keys = EncryptionKeyManager::Get(db);
return keys.GetKey(key_name);
Expand All @@ -48,8 +51,8 @@ void EncryptionEngine::AddKeyToCache(DatabaseInstance &db, data_ptr_t key, const
if (!keys.HasKey(key_name)) {
keys.AddKey(key_name, key);
} else {
// wipe out the key
std::memset(key, 0, MainHeader::DEFAULT_ENCRYPTION_KEY_LENGTH);
duckdb_mbedtls::MbedTlsWrapper::AESStateMBEDTLS::SecureClearData(key,
MainHeader::DEFAULT_ENCRYPTION_KEY_LENGTH);
}
}

Expand All @@ -60,8 +63,8 @@ string EncryptionEngine::AddKeyToCache(DatabaseInstance &db, data_ptr_t key) {
if (!keys.HasKey(key_id)) {
keys.AddKey(key_id, key);
} else {
// wipe out the original key
std::memset(key, 0, MainHeader::DEFAULT_ENCRYPTION_KEY_LENGTH);
duckdb_mbedtls::MbedTlsWrapper::AESStateMBEDTLS::SecureClearData(key,
MainHeader::DEFAULT_ENCRYPTION_KEY_LENGTH);
}

return key_id;
Expand Down
16 changes: 15 additions & 1 deletion src/duckdb/src/common/encryption_key_manager.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ string EncryptionKeyManager::GenerateRandomKeyID() {
void EncryptionKeyManager::AddKey(const string &key_name, data_ptr_t key) {
lock_guard<mutex> guard(lock);
derived_keys.emplace(key_name, EncryptionKey(key));
// Zero-out the encryption key
// Zero-out the input encryption key
duckdb_mbedtls::MbedTlsWrapper::AESStateMBEDTLS::SecureClearData(key, DERIVED_KEY_LENGTH);
}

Expand All @@ -91,6 +91,19 @@ const_data_ptr_t EncryptionKeyManager::GetKey(const string &key_name) const {

void EncryptionKeyManager::DeleteKey(const string &key_name) {
lock_guard<mutex> guard(lock);
ClearKey(key_name);
EraseKey(key_name);
}

void EncryptionKeyManager::ClearKey(const string &key_name) {
D_ASSERT(HasKey(key_name));
auto const key_data = derived_keys.at(key_name).GetData();
// clear the key (zero-out its memory)
duckdb_mbedtls::MbedTlsWrapper::AESStateMBEDTLS::SecureClearData(key_data,
MainHeader::DEFAULT_ENCRYPTION_KEY_LENGTH);
}

void EncryptionKeyManager::EraseKey(const string &key_name) {
derived_keys.erase(key_name);
}

Expand Down Expand Up @@ -130,6 +143,7 @@ void EncryptionKeyManager::DeriveKey(string &user_key, data_ptr_t salt, data_ptr

KeyDerivationFunctionSHA256(reinterpret_cast<const_data_ptr_t>(decoded_key.data()), decoded_key.size(), salt,
derived_key);

duckdb_mbedtls::MbedTlsWrapper::AESStateMBEDTLS::SecureClearData(data_ptr_cast(&user_key[0]), user_key.size());
duckdb_mbedtls::MbedTlsWrapper::AESStateMBEDTLS::SecureClearData(data_ptr_cast(&decoded_key[0]),
decoded_key.size());
Expand Down
36 changes: 20 additions & 16 deletions src/duckdb/src/common/error_data.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -18,36 +18,40 @@ ErrorData::ErrorData(const std::exception &ex) : ErrorData(ex.what()) {
}

ErrorData::ErrorData(ExceptionType type, const string &message)
: initialized(true), type(type), raw_message(SanitizeErrorMessage(message)),
final_message(ConstructFinalMessage()) {
: initialized(true), type(type), raw_message(SanitizeErrorMessage(message)) {
// In the case of ExceptionType::INTERNAL, the stack trace is part of the final message.
// To construct it, we need to access extra_info, which has to be initialized first.
// Thus, we only set final_message in the constructor's body.
final_message = ConstructFinalMessage();
}

ErrorData::ErrorData(const string &message)
: initialized(true), type(ExceptionType::INVALID), raw_message(string()), final_message(string()) {

// parse the constructed JSON
if (message.empty() || message[0] != '{') {
// not JSON! Use the message as a raw Exception message and leave type as uninitialized

// Not a JSON-formatted message.
// Use the message as a raw Exception message and leave the type as uninitialized.
if (message == std::bad_alloc().what()) {
type = ExceptionType::OUT_OF_MEMORY;
raw_message = "Allocation failure";
} else {
raw_message = message;
}
} else {
auto info = StringUtil::ParseJSONMap(message)->Flatten();
for (auto &entry : info) {
if (entry.first == "exception_type") {
type = Exception::StringToExceptionType(entry.second);
} else if (entry.first == "exception_message") {
raw_message = SanitizeErrorMessage(entry.second);
} else {
extra_info[entry.first] = entry.second;
}
}
final_message = ConstructFinalMessage();
return;
}

// JSON-formatted message.
auto info = StringUtil::ParseJSONMap(message)->Flatten();
for (auto &entry : info) {
if (entry.first == "exception_type") {
type = Exception::StringToExceptionType(entry.second);
} else if (entry.first == "exception_message") {
raw_message = SanitizeErrorMessage(entry.second);
} else {
extra_info[entry.first] = entry.second;
}
}
final_message = ConstructFinalMessage();
}

Expand Down
27 changes: 7 additions & 20 deletions src/duckdb/src/common/hive_partitioning.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -245,26 +245,13 @@ static void TemplatedGetHivePartitionValues(Vector &input, vector<HivePartitionK

const auto &type = input.GetType();

const auto reinterpret = Value::CreateValue<T>(data[sel.get_index(0)]).GetTypeMutable() != type;
if (reinterpret) {
for (idx_t i = 0; i < count; i++) {
auto &key = keys[i];
const auto idx = sel.get_index(i);
if (validity.RowIsValid(idx)) {
key.values[col_idx] = GetHiveKeyValue(data[idx], type);
} else {
key.values[col_idx] = GetHiveKeyNullValue(type);
}
}
} else {
for (idx_t i = 0; i < count; i++) {
auto &key = keys[i];
const auto idx = sel.get_index(i);
if (validity.RowIsValid(idx)) {
key.values[col_idx] = GetHiveKeyValue(data[idx]);
} else {
key.values[col_idx] = GetHiveKeyNullValue(type);
}
for (idx_t i = 0; i < count; i++) {
auto &key = keys[i];
const auto idx = sel.get_index(i);
if (validity.RowIsValid(idx)) {
key.values[col_idx] = GetHiveKeyValue(data[idx], type);
} else {
key.values[col_idx] = GetHiveKeyNullValue(type);
}
}
}
Expand Down
12 changes: 12 additions & 0 deletions src/duckdb/src/execution/expression_executor/execute_function.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,18 @@ bool ExecuteFunctionState::TryExecuteDictionaryExpression(const BoundFunctionExp
return true;
}

void ExecuteFunctionState::ResetDictionaryStates() {

// Clear the cached dictionary information
current_input_dictionary_id.clear();
output_dictionary_id.clear();
output_dictionary.reset();

for (const auto &child_state : child_states) {
child_state->ResetDictionaryStates();
}
}

unique_ptr<ExpressionState> ExpressionExecutor::InitializeState(const BoundFunctionExpression &expr,
ExpressionExecutorState &root) {
auto result = make_uniq<ExecuteFunctionState>(expr, root);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,12 +123,19 @@ void ExpressionExecutor::Execute(const BoundOperatorExpression &expr, Expression
throw;
}
}

// On error, evaluate per row
SelectionVector selvec(1);
DataChunk intermediate;
intermediate.Initialize(GetAllocator(), {result.GetType()}, 1);
for (idx_t i = 0; i < count; i++) {
intermediate.Reset();
intermediate.SetCardinality(1);

// Make sure to clear any dictionary states in the child expression, so that it actually
// gets executed anew for every row
child_state.ResetDictionaryStates();

selvec.set_index(0, sel ? sel->get_index(i) : i);
Value val(result.GetType());
try {
Expand Down
6 changes: 6 additions & 0 deletions src/duckdb/src/execution/expression_executor_state.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,12 @@ void ExpressionState::Verify(ExpressionExecutorState &root_executor) {
}
}

void ExpressionState::ResetDictionaryStates() {
for (const auto &child : child_states) {
child->ResetDictionaryStates();
}
}

void ExpressionExecutorState::Verify() {
D_ASSERT(executor);
root_state->Verify(*this);
Expand Down
2 changes: 1 addition & 1 deletion src/duckdb/src/function/pragma/pragma_queries.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,7 @@ static string PragmaDatabaseSize(ClientContext &context, const FunctionParameter
}

static string PragmaStorageInfo(ClientContext &context, const FunctionParameters &parameters) {
return StringUtil::Format("SELECT * FROM pragma_storage_info('%s');", parameters.values[0].ToString());
return StringUtil::Format("SELECT * FROM pragma_storage_info(%s);", SQLString(parameters.values[0].ToString()));
}

static string PragmaMetadataInfo(ClientContext &context, const FunctionParameters &parameters) {
Expand Down
6 changes: 3 additions & 3 deletions src/duckdb/src/function/table/version/pragma_version.cpp
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
#ifndef DUCKDB_PATCH_VERSION
#define DUCKDB_PATCH_VERSION "4-dev175"
#define DUCKDB_PATCH_VERSION "4-dev252"
#endif
#ifndef DUCKDB_MINOR_VERSION
#define DUCKDB_MINOR_VERSION 4
Expand All @@ -8,10 +8,10 @@
#define DUCKDB_MAJOR_VERSION 1
#endif
#ifndef DUCKDB_VERSION
#define DUCKDB_VERSION "v1.4.4-dev175"
#define DUCKDB_VERSION "v1.4.4-dev252"
#endif
#ifndef DUCKDB_SOURCE_ID
#define DUCKDB_SOURCE_ID "a56ccd8040"
#define DUCKDB_SOURCE_ID "431ad092c9"
#endif
#include "duckdb/function/table/system_functions.hpp"
#include "duckdb/main/database.hpp"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,10 @@ class EncryptionKey {
return key;
}

data_ptr_t GetData() {
return key;
}

public:
static void LockEncryptionKey(data_ptr_t key, idx_t key_len = MainHeader::DEFAULT_ENCRYPTION_KEY_LENGTH);
static void UnlockEncryptionKey(data_ptr_t key, idx_t key_len = MainHeader::DEFAULT_ENCRYPTION_KEY_LENGTH);
Expand All @@ -52,6 +56,8 @@ class EncryptionKeyManager : public ObjectCacheEntry {
void AddKey(const string &key_name, data_ptr_t key);
bool HasKey(const string &key_name) const;
void DeleteKey(const string &key_name);
void ClearKey(const string &key_name);
void EraseKey(const string &key_name);
const_data_ptr_t GetKey(const string &key_name) const;

public:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,9 @@ struct ExpressionState {

void Verify(ExpressionExecutorState &root);

//! Reset any cached dictionary expression states in this expression state and its children
virtual void ResetDictionaryStates();

public:
template <class TARGET>
TARGET &Cast() {
Expand All @@ -67,6 +70,8 @@ struct ExecuteFunctionState : public ExpressionState {
bool TryExecuteDictionaryExpression(const BoundFunctionExpression &expr, DataChunk &args, ExpressionState &state,
Vector &result);

void ResetDictionaryStates() override;

public:
unique_ptr<FunctionLocalState> local_state;

Expand Down
2 changes: 2 additions & 0 deletions src/duckdb/src/include/duckdb/main/extension_entries.hpp
Original file line number Diff line number Diff line change
Expand Up @@ -622,6 +622,7 @@ static constexpr ExtensionFunctionEntry EXTENSION_FUNCTIONS[] = {
{"st_hasm", "spatial", CatalogType::SCALAR_FUNCTION_ENTRY},
{"st_hasz", "spatial", CatalogType::SCALAR_FUNCTION_ENTRY},
{"st_hilbert", "spatial", CatalogType::SCALAR_FUNCTION_ENTRY},
{"st_interiorringn", "spatial", CatalogType::SCALAR_FUNCTION_ENTRY},
{"st_interpolatepoint", "spatial", CatalogType::SCALAR_FUNCTION_ENTRY},
{"st_intersection", "spatial", CatalogType::SCALAR_FUNCTION_ENTRY},
{"st_intersection_agg", "spatial", CatalogType::AGGREGATE_FUNCTION_ENTRY},
Expand Down Expand Up @@ -1034,6 +1035,7 @@ static constexpr ExtensionEntry EXTENSION_SETTINGS[] = {
{"ducklake_retry_wait_ms", "ducklake"},
{"enable_curl_server_cert_verification", "httpfs"},
{"enable_geoparquet_conversion", "parquet"},
{"enable_global_s3_configuration", "httpfs"},
{"enable_server_cert_verification", "httpfs"},
{"force_download", "httpfs"},
{"hf_max_per_page", "httpfs"},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ class HavingBinder : public BaseSelectBinder {
BindResult BindColumnRef(unique_ptr<ParsedExpression> &expr_ptr, idx_t depth, bool root_expression) override;

unique_ptr<ParsedExpression> QualifyColumnName(ColumnRefExpression &col_ref, ErrorData &error) override;
bool QualifyColumnAlias(const ColumnRefExpression &colref) override;

private:
ColumnAliasBinder column_alias_binder;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,14 @@ class TableFunctionBinder : public ExpressionBinder {
TableFunctionBinder(Binder &binder, ClientContext &context, string table_function_name = string(),
string clause = "Table function");

public:
void DisableSQLValueFunctions() {
accept_sql_value_functions = false;
}
void EnableSQLValueFunctions() {
accept_sql_value_functions = true;
}

protected:
BindResult BindLambdaReference(LambdaRefExpression &expr, idx_t depth);
BindResult BindColumnReference(unique_ptr<ParsedExpression> &expr, idx_t depth, bool root_expression);
Expand All @@ -28,6 +36,8 @@ class TableFunctionBinder : public ExpressionBinder {
private:
string table_function_name;
string clause;
//! Whether sql_value_functions (GetSQLValueFunctionName) are considered when binding column refs
bool accept_sql_value_functions = true;
};

} // namespace duckdb
11 changes: 11 additions & 0 deletions src/duckdb/src/planner/binder/statement/bind_copy.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -422,10 +422,21 @@ vector<Value> BindCopyOption(ClientContext &context, TableFunctionBinder &option
return result;
}
}
const bool is_partition_by = StringUtil::CIEquals(name, "partition_by");

if (is_partition_by) {
//! When binding the 'partition_by' option, we don't want to resolve a column reference to a SQLValueFunction
//! (like 'user')
option_binder.DisableSQLValueFunctions();
}
auto bound_expr = option_binder.Bind(expr);
if (bound_expr->HasParameter()) {
throw ParameterNotResolvedException();
}
if (is_partition_by) {
option_binder.EnableSQLValueFunctions();
}

auto val = ExpressionExecutor::EvaluateScalar(context, *bound_expr, true);
if (val.IsNull()) {
throw BinderException("NULL is not supported as a valid option for COPY option \"" + name + "\"");
Expand Down
5 changes: 3 additions & 2 deletions src/duckdb/src/planner/binder/statement/bind_insert.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -392,9 +392,10 @@ unique_ptr<MergeIntoStatement> Binder::GenerateMergeInto(InsertStatement &stmt,
named_column_map.push_back(col.Logical());
}
} else {
// Ensure that the columns are valid.
for (auto &col_name : stmt.columns) {
auto &col = table.GetColumn(col_name);
named_column_map.push_back(col.Logical());
auto col_idx = table.GetColumnIndex(col_name);
named_column_map.push_back(col_idx);
}
}
ExpandDefaultInValuesList(stmt, table, values_list, named_column_map);
Expand Down
Loading