Merge pull request #21052 from ClickHouse/file-engine-settings

Merging #20620
This commit is contained in:
alexey-milovidov 2021-02-22 13:10:31 +03:00 committed by GitHub
commit 50639b7dba
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 93 additions and 6 deletions

View File

@ -2659,3 +2659,23 @@ Result:
Note that this setting influences [Materialized view](../../sql-reference/statements/create/view.md#materialized) and [MaterializeMySQL](../../engines/database-engines/materialize-mysql.md) behaviour.
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->
## engine_file_empty_if_not_exists {#engine-file-empty_if-not-exists}
Allows to select data from a file engine table without file.
Possible values:
- 0 — `SELECT` throws exception.
- 1 — `SELECT` returns empty result.
Default value: `0`.
## engine_file_truncate_on_insert {#engine-file-truncate-on-insert}
Enables or disables truncate before insert in file engine tables.
Possible values:
- 0 — Disabled.
- 1 — Enabled.
Default value: `0`.

View File

@ -422,6 +422,8 @@ class IColumn;
M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \
M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \
M(Bool, allow_experimental_query_deduplication, false, "Allow sending parts' UUIDs for a query in order to deduplicate data parts if any", 0) \
M(Bool, engine_file_empty_if_not_exists, false, "Allows to select data from a file engine table without file", 0) \
M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \
M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \
M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \
M(Bool, database_replicated_ddl_output, true, "Return table with query execution status as a result of DDL query", 0) \

View File

@ -34,6 +34,7 @@
#include <Storages/Distributed/DirectoryMonitor.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Pipe.h>
namespace fs = std::filesystem;
@ -427,7 +428,12 @@ Pipe StorageFile::read(
paths = {""}; /// when use fd, paths are empty
else
if (paths.size() == 1 && !Poco::File(paths[0]).exists())
throw Exception("File " + paths[0] + " doesn't exist", ErrorCodes::FILE_DOESNT_EXIST);
{
if (context.getSettingsRef().engine_file_empty_if_not_exists)
return Pipe(std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())));
else
throw Exception("File " + paths[0] + " doesn't exist", ErrorCodes::FILE_DOESNT_EXIST);
}
auto files_info = std::make_shared<StorageFileSource::FilesInfo>();
@ -469,7 +475,8 @@ public:
std::unique_lock<std::shared_timed_mutex> && lock_,
const CompressionMethod compression_method,
const Context & context,
const std::optional<FormatSettings> & format_settings)
const std::optional<FormatSettings> & format_settings,
int & flags)
: storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, lock(std::move(lock_))
@ -485,13 +492,14 @@ public:
* INSERT data; SELECT *; last SELECT returns only insert_data
*/
storage.table_fd_was_used = true;
naked_buffer = std::make_unique<WriteBufferFromFileDescriptor>(storage.table_fd);
naked_buffer = std::make_unique<WriteBufferFromFileDescriptor>(storage.table_fd, DBMS_DEFAULT_BUFFER_SIZE);
}
else
{
if (storage.paths.size() != 1)
throw Exception("Table '" + storage.getStorageID().getNameForLogs() + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED);
naked_buffer = std::make_unique<WriteBufferFromFile>(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT);
flags |= O_WRONLY | O_APPEND | O_CREAT;
naked_buffer = std::make_unique<WriteBufferFromFile>(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, flags);
}
/// In case of CSVWithNames we have already written prefix.
@ -546,7 +554,12 @@ BlockOutputStreamPtr StorageFile::write(
if (format_name == "Distributed")
throw Exception("Method write is not implemented for Distributed format", ErrorCodes::NOT_IMPLEMENTED);
int flags = 0;
std::string path;
if (context.getSettingsRef().engine_file_truncate_on_insert)
flags |= O_TRUNC;
if (!paths.empty())
{
path = paths[0];
@ -559,7 +572,8 @@ BlockOutputStreamPtr StorageFile::write(
std::unique_lock{rwlock, getLockTimeout(context)},
chooseCompressionMethod(path, compression_method),
context,
format_settings);
format_settings,
flags);
}
bool StorageFile::storesDataOnDisk() const

View File

@ -0,0 +1,16 @@
DROP TABLE IF EXISTS file_engine_table;
CREATE TABLE file_engine_table (id UInt32) ENGINE=File(TSV);
SELECT * FROM file_engine_table; --{ serverError 107 }
SET engine_file_empty_if_not_exists=0;
SELECT * FROM file_engine_table; --{ serverError 107 }
SET engine_file_empty_if_not_exists=1;
SELECT * FROM file_engine_table;
SET engine_file_empty_if_not_exists=0;
DROP TABLE file_engine_table;

View File

@ -0,0 +1,13 @@
1
2
3
4
1
2
3
4
5
6
0
1
2

View File

@ -0,0 +1,21 @@
DROP TABLE IF EXISTS test;
INSERT INTO TABLE FUNCTION file('01721_file/test/data.TSV', 'TSV', 'id UInt32') VALUES (1);
ATTACH TABLE test FROM '01721_file/test' (id UInt8) ENGINE=File(TSV);
INSERT INTO test VALUES (2), (3);
INSERT INTO test VALUES (4);
SELECT * FROM test;
SET engine_file_truncate_on_insert=0;
INSERT INTO test VALUES (5), (6);
SELECT * FROM test;
SET engine_file_truncate_on_insert=1;
INSERT INTO test VALUES (0), (1), (2);
SELECT * FROM test;
SET engine_file_truncate_on_insert=0;
DROP TABLE test;

View File

@ -747,6 +747,7 @@
"memory_leak",
"memory_limit",
"polygon_dicts", // they use an explicitly specified database
"01658_read_file_to_stringcolumn"
"01658_read_file_to_stringcolumn",
"01721_engine_file_truncate_on_insert" // It's ok to execute in parallel but not several instances of the same test.
]
}