mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-12 17:32:32 +00:00
08f4f45fd9
CI founds after LSan had been fixed [1]: 01889_sqlite_read_write: [ FAIL ] 8.32 sec. - return code: 1 ================================================================= ==20649==ERROR: LeakSanitizer: detected memory leaks Indirect leak of 1968 byte(s) in 1 object(s) allocated from: 0 0xc5c1ffd in operator new(unsigned long) (/usr/bin/clickhouse+0xc5c1ffd) 1 0x25e32d0d in std::__1::__unique_if<DB::StorageInMemoryMetadata>::__unique_single std::__1::make_unique<DB::StorageInMemoryMetadata, DB::StorageInMemoryMetadata const&>(DB::StorageInMemoryMetadata c> 2 0x25e32d0d in DB::IStorage::setInMemoryMetadata(DB::StorageInMemoryMetadata const&) obj-x86_64-linux-gnu/../src/Storages/IStorage.h:194:22 3 0x29bdee98 in DB::StorageSQLite::StorageSQLite(DB::StorageID const&, std::__1::shared_ptr<sqlite3>, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std:> 4 0x25ee61d6 in std::__1::shared_ptr<DB::StorageSQLite> shared_ptr_helper<DB::StorageSQLite>::create<DB::StorageID, std::__1::shared_ptr<sqlite3> const&, std::__1::basic_string<char, std::__1::char_tr> 5 0x25ee61d6 in DB::TableFunctionSQLite::executeImpl(std::__1::shared_ptr<DB::IAST> const&, std::__1::shared_ptr<DB::Context const>, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1:> SUMMARY: AddressSanitizer: 171256 byte(s) leaked in 130 allocation(s). [1]: https://github.com/ClickHouse/ClickHouse/runs/4929706698?check_suite_focus=true Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
186 lines
5.8 KiB
C++
186 lines
5.8 KiB
C++
#include "StorageSQLite.h"
|
|
|
|
#if USE_SQLITE
|
|
#include <base/range.h>
|
|
#include <base/logger_useful.h>
|
|
#include <Processors/Sources/SQLiteSource.h>
|
|
#include <Databases/SQLite/SQLiteUtils.h>
|
|
#include <DataTypes/DataTypeString.h>
|
|
#include <Interpreters/Context.h>
|
|
#include <Formats/FormatFactory.h>
|
|
#include <Processors/Formats/IOutputFormat.h>
|
|
#include <IO/Operators.h>
|
|
#include <IO/WriteHelpers.h>
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
#include <Parsers/ASTLiteral.h>
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
|
#include <Storages/StorageFactory.h>
|
|
#include <Storages/transformQueryForExternalDatabase.h>
|
|
#include <Common/filesystemHelpers.h>
|
|
|
|
|
|
namespace DB
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
{
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
extern const int SQLITE_ENGINE_ERROR;
|
|
}
|
|
|
|
StorageSQLite::StorageSQLite(
|
|
const StorageID & table_id_,
|
|
SQLitePtr sqlite_db_,
|
|
const String & database_path_,
|
|
const String & remote_table_name_,
|
|
const ColumnsDescription & columns_,
|
|
const ConstraintsDescription & constraints_,
|
|
ContextPtr context_)
|
|
: IStorage(table_id_)
|
|
, WithContext(context_->getGlobalContext())
|
|
, remote_table_name(remote_table_name_)
|
|
, database_path(database_path_)
|
|
, sqlite_db(sqlite_db_)
|
|
, log(&Poco::Logger::get("StorageSQLite (" + table_id_.table_name + ")"))
|
|
{
|
|
StorageInMemoryMetadata storage_metadata;
|
|
storage_metadata.setColumns(columns_);
|
|
storage_metadata.setConstraints(constraints_);
|
|
setInMemoryMetadata(storage_metadata);
|
|
}
|
|
|
|
|
|
Pipe StorageSQLite::read(
|
|
const Names & column_names,
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
SelectQueryInfo & query_info,
|
|
ContextPtr context_,
|
|
QueryProcessingStage::Enum,
|
|
size_t max_block_size,
|
|
unsigned int)
|
|
{
|
|
if (!sqlite_db)
|
|
sqlite_db = openSQLiteDB(database_path, getContext(), /* throw_on_error */true);
|
|
|
|
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
|
|
|
|
String query = transformQueryForExternalDatabase(
|
|
query_info,
|
|
metadata_snapshot->getColumns().getOrdinary(),
|
|
IdentifierQuotingStyle::DoubleQuotes,
|
|
"",
|
|
remote_table_name,
|
|
context_);
|
|
LOG_TRACE(log, "Query: {}", query);
|
|
|
|
Block sample_block;
|
|
for (const String & column_name : column_names)
|
|
{
|
|
auto column_data = metadata_snapshot->getColumns().getPhysical(column_name);
|
|
sample_block.insert({column_data.type, column_data.name});
|
|
}
|
|
|
|
return Pipe(std::make_shared<SQLiteSource>(sqlite_db, query, sample_block, max_block_size));
|
|
}
|
|
|
|
|
|
class SQLiteSink : public SinkToStorage
|
|
{
|
|
public:
|
|
explicit SQLiteSink(
|
|
const StorageSQLite & storage_,
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
|
StorageSQLite::SQLitePtr sqlite_db_,
|
|
const String & remote_table_name_)
|
|
: SinkToStorage(metadata_snapshot_->getSampleBlock())
|
|
, storage{storage_}
|
|
, metadata_snapshot(metadata_snapshot_)
|
|
, sqlite_db(sqlite_db_)
|
|
, remote_table_name(remote_table_name_)
|
|
{
|
|
}
|
|
|
|
String getName() const override { return "SQLiteSink"; }
|
|
|
|
void consume(Chunk chunk) override
|
|
{
|
|
auto block = getHeader().cloneWithColumns(chunk.getColumns());
|
|
WriteBufferFromOwnString sqlbuf;
|
|
|
|
sqlbuf << "INSERT INTO ";
|
|
sqlbuf << doubleQuoteString(remote_table_name);
|
|
sqlbuf << " (";
|
|
|
|
for (auto it = block.begin(); it != block.end(); ++it)
|
|
{
|
|
if (it != block.begin())
|
|
sqlbuf << ", ";
|
|
sqlbuf << quoteString(it->name);
|
|
}
|
|
|
|
sqlbuf << ") VALUES ";
|
|
|
|
auto writer = FormatFactory::instance().getOutputFormat("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.getContext());
|
|
writer->write(block);
|
|
|
|
sqlbuf << ";";
|
|
|
|
char * err_message = nullptr;
|
|
int status = sqlite3_exec(sqlite_db.get(), sqlbuf.str().c_str(), nullptr, nullptr, &err_message);
|
|
|
|
if (status != SQLITE_OK)
|
|
{
|
|
String err_msg(err_message);
|
|
sqlite3_free(err_message);
|
|
throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR,
|
|
"Failed to execute sqlite INSERT query. Status: {}. Message: {}",
|
|
status, err_msg);
|
|
}
|
|
}
|
|
|
|
private:
|
|
const StorageSQLite & storage;
|
|
StorageMetadataPtr metadata_snapshot;
|
|
StorageSQLite::SQLitePtr sqlite_db;
|
|
String remote_table_name;
|
|
};
|
|
|
|
|
|
SinkToStoragePtr StorageSQLite::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr)
|
|
{
|
|
if (!sqlite_db)
|
|
sqlite_db = openSQLiteDB(database_path, getContext(), /* throw_on_error */true);
|
|
return std::make_shared<SQLiteSink>(*this, metadata_snapshot, sqlite_db, remote_table_name);
|
|
}
|
|
|
|
|
|
void registerStorageSQLite(StorageFactory & factory)
|
|
{
|
|
factory.registerStorage("SQLite", [](const StorageFactory::Arguments & args) -> StoragePtr
|
|
{
|
|
ASTs & engine_args = args.engine_args;
|
|
|
|
if (engine_args.size() != 2)
|
|
throw Exception("SQLite database requires 2 arguments: database path, table name",
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
for (auto & engine_arg : engine_args)
|
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
|
|
|
|
const auto database_path = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
|
const auto table_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
auto sqlite_db = openSQLiteDB(database_path, args.getContext(), /* throw_on_error */!args.attach);
|
|
|
|
return StorageSQLite::create(args.table_id, sqlite_db, database_path,
|
|
table_name, args.columns, args.constraints, args.getContext());
|
|
},
|
|
{
|
|
.source_access_type = AccessType::SQLITE,
|
|
});
|
|
}
|
|
|
|
}
|
|
|
|
#endif
|