Merge pull request #65250 from ClickHouse/revert-51008-nopresigned

Revert "Change default s3_throw_on_zero_files_match to true, document that presigned S3 URLs are not supported"
This commit is contained in:
Max K 2024-06-13 21:12:21 +00:00 committed by GitHub
commit bc183d8e04
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 24 additions and 57 deletions

View File

@ -248,6 +248,7 @@ FROM s3(
LIMIT 5;
```
## Working with archives
Suppose that we have several archive files with following URIs on S3:
@ -265,16 +266,6 @@ FROM s3(
);
```
## Presigned URL
Presigned URLs are currently not supported. Use `url()` table function instead:
```sql
SELECT *
FROM url(
'https://example.amazonaws.com/f.csv?X-Amz-Security-Token=[...]'
)
```
## Virtual Columns {#virtual-columns}

View File

@ -115,9 +115,9 @@ class IColumn;
M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \
M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \
M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \
M(Bool, s3_throw_on_zero_files_match, true, "Throw an error, when ListObjects request cannot match any files", 0) \
M(Bool, hdfs_throw_on_zero_files_match, true, "Throw an error, when ListObjects request cannot match any files", 0) \
M(Bool, azure_throw_on_zero_files_match, true, "Throw an error, when ListObjects request cannot match any files", 0) \
M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \
M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \
M(Bool, azure_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \
M(Bool, s3_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in S3 table engine", 0) \
M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in HDFS table engine", 0) \
M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \

View File

@ -88,9 +88,8 @@ static const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges
{"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"},
{"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"},
{"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."},
{"s3_throw_on_zero_files_match", false, true, "Report error if S3 URL with wildcards didn't match any files. Useful in particular when trying to use pre-signed URL (which is not supported) with '?' character that is inadvertently interpreted as a wildcard."},
{"hdfs_throw_on_zero_files_match", false, true, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"},
{"azure_throw_on_zero_files_match", false, true, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"},
{"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"},
{"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"},
{"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"},
{"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"},
{"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"},
@ -106,7 +105,6 @@ static const std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges
{"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."},
{"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"},
{"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"},
{"backup_restore_s3_retry_attempts", 0, 1000, "A new setting."},
}},
{"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"},
{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},

View File

@ -118,7 +118,6 @@ StorageObjectStorage::QuerySettings StorageAzureConfiguration::getQuerySettings(
.skip_empty_files = settings.azure_skip_empty_files,
.list_object_keys_size = settings.azure_list_object_keys_size,
.throw_on_zero_files_match = settings.azure_throw_on_zero_files_match,
.throw_on_zero_files_match_setting_name = "azure_throw_on_zero_files_match",
.ignore_non_existent_file = settings.azure_ignore_file_doesnt_exist,
};
}

View File

@ -76,7 +76,6 @@ StorageObjectStorage::QuerySettings StorageHDFSConfiguration::getQuerySettings(c
.skip_empty_files = settings.hdfs_skip_empty_files,
.list_object_keys_size = 0, /// HDFS does not support listing in batches.
.throw_on_zero_files_match = settings.hdfs_throw_on_zero_files_match,
.throw_on_zero_files_match_setting_name = "hdfs_throw_on_zero_files_match",
.ignore_non_existent_file = settings.hdfs_ignore_file_doesnt_exist,
};
}

View File

@ -96,7 +96,6 @@ StorageObjectStorage::QuerySettings StorageS3Configuration::getQuerySettings(con
.skip_empty_files = settings.s3_skip_empty_files,
.list_object_keys_size = settings.s3_list_object_keys_size,
.throw_on_zero_files_match = settings.s3_throw_on_zero_files_match,
.throw_on_zero_files_match_setting_name = "s3_throw_on_zero_files_match",
.ignore_non_existent_file = settings.s3_ignore_file_doesnt_exist,
};
}

View File

@ -41,7 +41,6 @@ public:
bool skip_empty_files;
size_t list_object_keys_size;
bool throw_on_zero_files_match;
const char * throw_on_zero_files_match_setting_name = nullptr;
bool ignore_non_existent_file;
};

View File

@ -126,8 +126,7 @@ std::shared_ptr<StorageObjectStorageSource::IIterator> StorageObjectStorageSourc
iterator = std::make_unique<GlobIterator>(
object_storage, configuration, predicate, virtual_columns,
local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size,
settings.throw_on_zero_files_match, settings.throw_on_zero_files_match_setting_name,
file_progress_callback);
settings.throw_on_zero_files_match, file_progress_callback);
}
else
{
@ -426,7 +425,6 @@ StorageObjectStorageSource::GlobIterator::GlobIterator(
ObjectInfos * read_keys_,
size_t list_object_keys_size,
bool throw_on_zero_files_match_,
const char * throw_on_zero_files_match_setting_name_,
std::function<void(FileProgress)> file_progress_callback_)
: IIterator("GlobIterator")
, WithContext(context_)
@ -434,7 +432,6 @@ StorageObjectStorageSource::GlobIterator::GlobIterator(
, configuration(configuration_)
, virtual_columns(virtual_columns_)
, throw_on_zero_files_match(throw_on_zero_files_match_)
, throw_on_zero_files_match_setting_name(throw_on_zero_files_match_setting_name_)
, read_keys(read_keys_)
, file_progress_callback(file_progress_callback_)
{
@ -487,11 +484,8 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne
if (first_iteration && !object_info && throw_on_zero_files_match)
{
throw Exception(ErrorCodes::FILE_DOESNT_EXIST,
"Can not match any files with path {}{}",
configuration->getPath(),
throw_on_zero_files_match_setting_name
? fmt::format(" (this error can be suppressed by setting {} = false)", throw_on_zero_files_match_setting_name)
: "");
"Can not match any files with path {}",
configuration->getPath());
}
first_iteration = false;
return object_info;

View File

@ -168,7 +168,6 @@ public:
ObjectInfos * read_keys_,
size_t list_object_keys_size,
bool throw_on_zero_files_match_,
const char * throw_on_zero_files_match_setting_name_,
std::function<void(FileProgress)> file_progress_callback_ = {});
~GlobIterator() override = default;
@ -185,7 +184,6 @@ private:
const ConfigurationPtr configuration;
const NamesAndTypesList virtual_columns;
const bool throw_on_zero_files_match;
const char * throw_on_zero_files_match_setting_name;
size_t index = 0;

View File

@ -486,7 +486,7 @@ std::shared_ptr<StorageS3Queue::FileIterator> StorageS3Queue::createFileIterator
{
auto settings = configuration->getQuerySettings(local_context);
auto glob_iterator = std::make_unique<StorageObjectStorageSource::GlobIterator>(
object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match, settings.throw_on_zero_files_match_setting_name);
object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match);
return std::make_shared<FileIterator>(files_metadata, std::move(glob_iterator), shutdown_called, log);
}

View File

@ -111,7 +111,7 @@ def test_storage_with_multidirectory_glob(started_cluster):
try:
node1.query(
"SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p4/path1,p2/path3}/postfix/data{1,2}.nonexist', TSV) SETTINGS hdfs_throw_on_zero_files_match=0"
"SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p4/path1,p2/path3}/postfix/data{1,2}.nonexist', TSV)"
)
assert False, "Exception have to be thrown"
except Exception as ex:
@ -220,22 +220,14 @@ def test_globs_in_read_table(started_cluster):
)
print("inside_table_func ", inside_table_func)
assert (
node1.query(
"select * from hdfs("
+ inside_table_func
+ ") settings hdfs_throw_on_zero_files_match=0"
)
node1.query("select * from hdfs(" + inside_table_func + ")")
== paths_amount * some_data
)
assert node1.query(
"select count(distinct _path) from hdfs("
+ inside_table_func
+ ") settings hdfs_throw_on_zero_files_match=0"
"select count(distinct _path) from hdfs(" + inside_table_func + ")"
).rstrip() == str(paths_amount)
assert node1.query(
"select count(distinct _file) from hdfs("
+ inside_table_func
+ ") settings hdfs_throw_on_zero_files_match=0"
"select count(distinct _file) from hdfs(" + inside_table_func + ")"
).rstrip() == str(files_amount)
@ -643,7 +635,6 @@ def test_cluster_join(started_cluster):
SELECT l.id,r.id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as l
JOIN hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') as r
ON l.id = r.id
SETTINGS hdfs_throw_on_zero_files_match=0
"""
)
assert "AMBIGUOUS_COLUMN_NAME" not in result
@ -652,13 +643,13 @@ def test_cluster_join(started_cluster):
def test_cluster_macro(started_cluster):
with_macro = node1.query(
"""
SELECT id FROM hdfsCluster('{default_cluster_macro}', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') SETTINGS hdfs_throw_on_zero_files_match=0
SELECT id FROM hdfsCluster('{default_cluster_macro}', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32')
"""
)
no_macro = node1.query(
"""
SELECT id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') SETTINGS hdfs_throw_on_zero_files_match=0
SELECT id FROM hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32')
"""
)

View File

@ -3,5 +3,5 @@ drop table if exists test_02481_mismatch_files;
create table test_02481_mismatch_files (a UInt64, b String) engine = S3(s3_conn, filename='test_02481_mismatch_files_{_partition_id}', format=Parquet) partition by a;
set s3_truncate_on_insert=1;
insert into test_02481_mismatch_files values (1, 'a'), (22, 'b'), (333, 'c');
select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE }
select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError FILE_DOESNT_EXIST }
select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE }
select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=1; -- { serverError FILE_DOESNT_EXIST }

View File

@ -7,6 +7,6 @@ create table test_02481_mismatch_files (a UInt64, b String) engine = S3(s3_conn,
set s3_truncate_on_insert=1;
insert into test_02481_mismatch_files values (1, 'a'), (22, 'b'), (333, 'c');
select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE }
select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE }
select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet); -- { serverError FILE_DOESNT_EXIST }
select a, b from s3(s3_conn, filename='test_02481_mismatch_filesxxx*', format=Parquet) settings s3_throw_on_zero_files_match=1; -- { serverError FILE_DOESNT_EXIST }

View File

@ -1,5 +1,5 @@
-- Tags: no-fasttest
select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet', NOSIGN) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE }
select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE }
select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD', NOSIGN) settings s3_throw_on_zero_files_match=0; -- { serverError CANNOT_DETECT_FORMAT }
select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD'); -- { serverError CANNOT_DETECT_FORMAT }

View File

@ -1,4 +1,4 @@
personal_ws-1.1 en 2912
personal_ws-1.1 en 2758
AArch
ACLs
ALTERs
@ -722,7 +722,6 @@ Postgres
PostgresSQL
Precompiled
Preprocess
Presigned
PrettyCompact
PrettyCompactMonoBlock
PrettyCompactNoEscapes
@ -1937,9 +1936,9 @@ loghouse
london
lookups
loongarch
lowcardinality
lowCardinalityIndices
lowCardinalityKeys
lowcardinality
lowerUTF
lowercased
lttb