mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
remove read_only option
Signed-off-by: Lloyd-Pottiger <yan1579196623@gamil.com>
This commit is contained in:
parent
51f53af482
commit
85a1a36167
@ -15,13 +15,12 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
|
||||
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
|
||||
...
|
||||
) ENGINE = EmbeddedRocksDB([ttl, read_only]) PRIMARY KEY(primary_key_name)
|
||||
) ENGINE = EmbeddedRocksDB([ttl]) PRIMARY KEY(primary_key_name)
|
||||
```
|
||||
|
||||
Engine parameters:
|
||||
|
||||
- `ttl` - time to live for values. TTL is accepted in seconds. If TTL is 0, regular RocksDB instance is used (without TTL).
|
||||
- `read_only` - when `read_only` is set to `true`, read-only mode is used. For storage with TTL, compaction will not be triggered (neither manual nor automatic), so no expired entries are removed.
|
||||
- `primary_key_name` – any column name in the column list.
|
||||
- `primary key` must be specified, it supports only one column in the primary key. The primary key will be serialized in binary as a `rocksdb key`.
|
||||
- columns other than the primary key will be serialized in binary as `rocksdb` value in corresponding order.
|
||||
|
@ -166,13 +166,11 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_,
|
||||
bool attach,
|
||||
ContextPtr context_,
|
||||
const String & primary_key_,
|
||||
Int32 ttl_,
|
||||
bool read_only_)
|
||||
Int32 ttl_)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, primary_key{primary_key_}
|
||||
, ttl(ttl_)
|
||||
, read_only(read_only_)
|
||||
{
|
||||
setInMemoryMetadata(metadata_);
|
||||
rocksdb_dir = context_->getPath() + relative_data_path_;
|
||||
@ -271,7 +269,7 @@ void StorageEmbeddedRocksDB::initDB()
|
||||
if (ttl > 0)
|
||||
{
|
||||
rocksdb::DBWithTTL * db;
|
||||
status = rocksdb::DBWithTTL::Open(merged, rocksdb_dir, &db, ttl, read_only);
|
||||
status = rocksdb::DBWithTTL::Open(merged, rocksdb_dir, &db, ttl);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Failed to open rocksdb path at: {}: {}",
|
||||
@ -282,14 +280,7 @@ void StorageEmbeddedRocksDB::initDB()
|
||||
else
|
||||
{
|
||||
rocksdb::DB * db;
|
||||
if (read_only)
|
||||
{
|
||||
status = rocksdb::DB::OpenForReadOnly(merged, rocksdb_dir, &db);
|
||||
}
|
||||
else
|
||||
{
|
||||
status = rocksdb::DB::Open(merged, rocksdb_dir, &db);
|
||||
}
|
||||
status = rocksdb::DB::Open(merged, rocksdb_dir, &db);
|
||||
if (!status.ok())
|
||||
{
|
||||
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Failed to open rocksdb path at: {}: {}",
|
||||
@ -360,18 +351,15 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
{
|
||||
// TODO custom RocksDBSettings, table function
|
||||
auto engine_args = args.engine_args;
|
||||
if (engine_args.size() > 2)
|
||||
if (engine_args.size() > 1)
|
||||
{
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Engine {} requires at most 2 parameters. ({} given). Correct usage: EmbeddedRocksDB([ttl, read_only])",
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Engine {} requires at most 1 parameter. ({} given). Correct usage: EmbeddedRocksDB([ttl])",
|
||||
args.engine_name, engine_args.size());
|
||||
}
|
||||
|
||||
Int32 ttl{0};
|
||||
bool read_only{false};
|
||||
if (!engine_args.empty())
|
||||
ttl = checkAndGetLiteralArgument<UInt64>(engine_args[0], "ttl");
|
||||
if (engine_args.size() > 1)
|
||||
read_only = checkAndGetLiteralArgument<bool>(engine_args[1], "read_only");
|
||||
|
||||
StorageInMemoryMetadata metadata;
|
||||
metadata.setColumns(args.columns);
|
||||
@ -386,7 +374,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
{
|
||||
throw Exception("StorageEmbeddedRocksDB must require one column in primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
return std::make_shared<StorageEmbeddedRocksDB>(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0], ttl, read_only);
|
||||
return std::make_shared<StorageEmbeddedRocksDB>(args.table_id, args.relative_data_path, metadata, args.attach, args.getContext(), primary_key_names[0], ttl);
|
||||
}
|
||||
|
||||
std::shared_ptr<rocksdb::Statistics> StorageEmbeddedRocksDB::getRocksDBStatistics() const
|
||||
@ -482,6 +470,7 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory)
|
||||
{
|
||||
StorageFactory::StorageFeatures features{
|
||||
.supports_sort_order = true,
|
||||
.supports_ttl = true,
|
||||
.supports_parallel_insert = true,
|
||||
};
|
||||
|
||||
|
@ -33,8 +33,7 @@ public:
|
||||
bool attach,
|
||||
ContextPtr context_,
|
||||
const String & primary_key_,
|
||||
Int32 ttl_ = 0,
|
||||
bool read_only = false);
|
||||
Int32 ttl_ = 0);
|
||||
|
||||
std::string getName() const override { return "EmbeddedRocksDB"; }
|
||||
|
||||
@ -83,7 +82,6 @@ private:
|
||||
mutable std::shared_mutex rocksdb_ptr_mx;
|
||||
String rocksdb_dir;
|
||||
Int32 ttl;
|
||||
bool read_only;
|
||||
|
||||
void initDB();
|
||||
};
|
||||
|
@ -38,13 +38,7 @@ def test_valid_options(start_cluster):
|
||||
)
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(0, 1) PRIMARY KEY(key);
|
||||
DROP TABLE test;
|
||||
"""
|
||||
)
|
||||
node.query(
|
||||
"""
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(10, 1) PRIMARY KEY(key);
|
||||
CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB(10) PRIMARY KEY(key);
|
||||
DROP TABLE test;
|
||||
"""
|
||||
)
|
||||
|
@ -1 +0,0 @@
|
||||
--
|
@ -1,14 +0,0 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
DROP TABLE IF EXISTS 02381_test;
|
||||
|
||||
CREATE TABLE 02381_test (key UInt64, value String) Engine=EmbeddedRocksDB(0, 1) PRIMARY KEY(key);
|
||||
|
||||
SELECT value FROM system.rocksdb WHERE database = currentDatabase() and table = '02381_test' and name = 'number.keys.written';
|
||||
INSERT INTO 02381_test SELECT number, format('Hello, world ({})', toString(number)) FROM numbers(10000);
|
||||
SELECT value FROM system.rocksdb WHERE database = currentDatabase() and table = '02381_test' and name = 'number.keys.written';
|
||||
|
||||
SELECT * FROM 02381_test WHERE key = 123;
|
||||
SELECT '--';
|
||||
|
||||
DROP TABLE IF EXISTS 02381_test;
|
Loading…
Reference in New Issue
Block a user