2021-05-17 11:02:35 +00:00
|
|
|
#include "StorageSQLite.h"
|
|
|
|
|
2021-07-08 13:27:30 +00:00
|
|
|
#if USE_SQLITE
|
2022-04-27 15:05:45 +00:00
|
|
|
#include <Common/logger_useful.h>
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <Processors/Sources/SQLiteSource.h>
|
2021-07-14 13:13:37 +00:00
|
|
|
#include <Databases/SQLite/SQLiteUtils.h>
|
2023-05-19 00:44:27 +00:00
|
|
|
#include <Databases/SQLite/fetchSQLiteTableStructure.h>
|
2021-05-17 11:02:35 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
2021-05-24 18:59:10 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
2021-10-11 16:11:50 +00:00
|
|
|
#include <Processors/Formats/IOutputFormat.h>
|
2021-05-24 18:59:10 +00:00
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2021-05-17 11:02:35 +00:00
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
2021-07-23 14:25:35 +00:00
|
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
2021-05-17 11:02:35 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <Storages/transformQueryForExternalDatabase.h>
|
2022-06-23 20:04:06 +00:00
|
|
|
#include <Storages/checkAndGetLiteralArgument.h>
|
2022-05-20 19:49:31 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2021-07-13 19:34:31 +00:00
|
|
|
#include <Common/filesystemHelpers.h>
|
2021-07-10 08:05:23 +00:00
|
|
|
|
2021-05-17 11:02:35 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2021-07-10 08:05:23 +00:00
|
|
|
|
2021-05-17 11:02:35 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2021-07-10 11:00:59 +00:00
|
|
|
extern const int SQLITE_ENGINE_ERROR;
|
2021-05-17 11:02:35 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
StorageSQLite::StorageSQLite(
|
|
|
|
const StorageID & table_id_,
|
2021-07-10 08:05:23 +00:00
|
|
|
SQLitePtr sqlite_db_,
|
2021-07-26 07:05:28 +00:00
|
|
|
const String & database_path_,
|
2021-05-17 11:02:35 +00:00
|
|
|
const String & remote_table_name_,
|
|
|
|
const ColumnsDescription & columns_,
|
|
|
|
const ConstraintsDescription & constraints_,
|
2021-06-01 20:52:12 +00:00
|
|
|
ContextPtr context_)
|
2021-05-17 11:02:35 +00:00
|
|
|
: IStorage(table_id_)
|
2021-05-24 18:59:10 +00:00
|
|
|
, WithContext(context_->getGlobalContext())
|
2021-05-17 11:02:35 +00:00
|
|
|
, remote_table_name(remote_table_name_)
|
2021-07-26 07:05:28 +00:00
|
|
|
, database_path(database_path_)
|
2021-07-10 08:05:23 +00:00
|
|
|
, sqlite_db(sqlite_db_)
|
2021-09-28 22:17:26 +00:00
|
|
|
, log(&Poco::Logger::get("StorageSQLite (" + table_id_.table_name + ")"))
|
2021-05-17 11:02:35 +00:00
|
|
|
{
|
|
|
|
StorageInMemoryMetadata storage_metadata;
|
2023-05-19 00:44:27 +00:00
|
|
|
|
|
|
|
if (columns_.empty())
|
|
|
|
{
|
|
|
|
auto columns = getTableStructureFromData(sqlite_db, remote_table_name);
|
|
|
|
storage_metadata.setColumns(columns);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
storage_metadata.setColumns(columns_);
|
|
|
|
|
2021-05-17 11:02:35 +00:00
|
|
|
storage_metadata.setConstraints(constraints_);
|
|
|
|
setInMemoryMetadata(storage_metadata);
|
|
|
|
}
|
2021-07-08 13:27:30 +00:00
|
|
|
|
|
|
|
|
2023-05-19 00:44:27 +00:00
|
|
|
ColumnsDescription StorageSQLite::getTableStructureFromData(
|
|
|
|
const SQLitePtr & sqlite_db_,
|
|
|
|
const String & table)
|
|
|
|
{
|
|
|
|
auto columns = fetchSQLiteTableStructure(sqlite_db_.get(), table);
|
|
|
|
|
|
|
|
if (!columns)
|
|
|
|
throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, "Failed to fetch table structure for {}", table);
|
|
|
|
|
|
|
|
return ColumnsDescription{*columns};
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-05-17 11:02:35 +00:00
|
|
|
Pipe StorageSQLite::read(
|
|
|
|
const Names & column_names,
|
2021-07-20 15:20:21 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2021-05-17 11:02:35 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-05-24 18:59:10 +00:00
|
|
|
ContextPtr context_,
|
2021-05-17 11:02:35 +00:00
|
|
|
QueryProcessingStage::Enum,
|
|
|
|
size_t max_block_size,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t /*num_streams*/)
|
2021-05-17 11:02:35 +00:00
|
|
|
{
|
2021-07-26 07:05:28 +00:00
|
|
|
if (!sqlite_db)
|
|
|
|
sqlite_db = openSQLiteDB(database_path, getContext(), /* throw_on_error */true);
|
|
|
|
|
2021-07-20 15:20:21 +00:00
|
|
|
storage_snapshot->check(column_names);
|
2021-05-17 11:02:35 +00:00
|
|
|
|
|
|
|
String query = transformQueryForExternalDatabase(
|
|
|
|
query_info,
|
2023-03-07 20:39:26 +00:00
|
|
|
column_names,
|
2021-07-20 15:20:21 +00:00
|
|
|
storage_snapshot->metadata->getColumns().getOrdinary(),
|
2021-05-17 11:02:35 +00:00
|
|
|
IdentifierQuotingStyle::DoubleQuotes,
|
2021-06-01 20:52:12 +00:00
|
|
|
"",
|
2021-05-17 11:02:35 +00:00
|
|
|
remote_table_name,
|
2021-05-24 18:59:10 +00:00
|
|
|
context_);
|
2021-09-28 22:17:26 +00:00
|
|
|
LOG_TRACE(log, "Query: {}", query);
|
2021-05-17 11:02:35 +00:00
|
|
|
|
|
|
|
Block sample_block;
|
|
|
|
for (const String & column_name : column_names)
|
|
|
|
{
|
2021-07-20 15:20:21 +00:00
|
|
|
auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name);
|
2021-05-17 11:02:35 +00:00
|
|
|
sample_block.insert({column_data.type, column_data.name});
|
|
|
|
}
|
|
|
|
|
2021-08-11 16:44:34 +00:00
|
|
|
return Pipe(std::make_shared<SQLiteSource>(sqlite_db, query, sample_block, max_block_size));
|
2021-05-17 11:02:35 +00:00
|
|
|
}
|
|
|
|
|
2021-07-08 13:27:30 +00:00
|
|
|
|
2021-07-23 14:25:35 +00:00
|
|
|
class SQLiteSink : public SinkToStorage
|
2021-05-24 18:59:10 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-07-23 14:25:35 +00:00
|
|
|
explicit SQLiteSink(
|
2021-05-24 18:59:10 +00:00
|
|
|
const StorageSQLite & storage_,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
2021-07-10 08:05:23 +00:00
|
|
|
StorageSQLite::SQLitePtr sqlite_db_,
|
|
|
|
const String & remote_table_name_)
|
2021-07-26 10:08:40 +00:00
|
|
|
: SinkToStorage(metadata_snapshot_->getSampleBlock())
|
2021-07-23 14:25:35 +00:00
|
|
|
, storage{storage_}
|
2021-05-24 18:59:10 +00:00
|
|
|
, metadata_snapshot(metadata_snapshot_)
|
2021-07-10 08:05:23 +00:00
|
|
|
, sqlite_db(sqlite_db_)
|
2021-05-24 18:59:10 +00:00
|
|
|
, remote_table_name(remote_table_name_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
String getName() const override { return "SQLiteSink"; }
|
2021-05-24 18:59:10 +00:00
|
|
|
|
2021-07-23 14:25:35 +00:00
|
|
|
void consume(Chunk chunk) override
|
2021-05-24 18:59:10 +00:00
|
|
|
{
|
2021-09-03 17:29:36 +00:00
|
|
|
auto block = getHeader().cloneWithColumns(chunk.getColumns());
|
2021-05-24 18:59:10 +00:00
|
|
|
WriteBufferFromOwnString sqlbuf;
|
2021-07-10 08:05:23 +00:00
|
|
|
|
2021-05-24 18:59:10 +00:00
|
|
|
sqlbuf << "INSERT INTO ";
|
2021-06-01 20:52:12 +00:00
|
|
|
sqlbuf << doubleQuoteString(remote_table_name);
|
2021-05-24 18:59:10 +00:00
|
|
|
sqlbuf << " (";
|
|
|
|
|
|
|
|
for (auto it = block.begin(); it != block.end(); ++it)
|
|
|
|
{
|
|
|
|
if (it != block.begin())
|
|
|
|
sqlbuf << ", ";
|
2021-06-01 20:52:12 +00:00
|
|
|
sqlbuf << quoteString(it->name);
|
2021-05-24 18:59:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
sqlbuf << ") VALUES ";
|
|
|
|
|
2021-10-11 16:11:50 +00:00
|
|
|
auto writer = FormatFactory::instance().getOutputFormat("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.getContext());
|
2021-05-24 18:59:10 +00:00
|
|
|
writer->write(block);
|
|
|
|
|
|
|
|
sqlbuf << ";";
|
|
|
|
|
|
|
|
char * err_message = nullptr;
|
2021-07-10 08:05:23 +00:00
|
|
|
int status = sqlite3_exec(sqlite_db.get(), sqlbuf.str().c_str(), nullptr, nullptr, &err_message);
|
2021-05-24 18:59:10 +00:00
|
|
|
|
|
|
|
if (status != SQLITE_OK)
|
|
|
|
{
|
|
|
|
String err_msg(err_message);
|
|
|
|
sqlite3_free(err_message);
|
2021-07-10 11:00:59 +00:00
|
|
|
throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR,
|
|
|
|
"Failed to execute sqlite INSERT query. Status: {}. Message: {}",
|
|
|
|
status, err_msg);
|
2021-05-24 18:59:10 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
const StorageSQLite & storage;
|
|
|
|
StorageMetadataPtr metadata_snapshot;
|
2021-07-10 08:05:23 +00:00
|
|
|
StorageSQLite::SQLitePtr sqlite_db;
|
|
|
|
String remote_table_name;
|
2021-05-24 18:59:10 +00:00
|
|
|
};
|
|
|
|
|
2021-07-08 13:27:30 +00:00
|
|
|
|
2023-06-07 18:33:08 +00:00
|
|
|
SinkToStoragePtr StorageSQLite::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool /*async_insert*/)
|
2021-05-24 18:59:10 +00:00
|
|
|
{
|
2021-07-26 07:05:28 +00:00
|
|
|
if (!sqlite_db)
|
|
|
|
sqlite_db = openSQLiteDB(database_path, getContext(), /* throw_on_error */true);
|
2021-07-23 14:25:35 +00:00
|
|
|
return std::make_shared<SQLiteSink>(*this, metadata_snapshot, sqlite_db, remote_table_name);
|
2021-05-24 18:59:10 +00:00
|
|
|
}
|
|
|
|
|
2021-07-10 08:05:23 +00:00
|
|
|
|
2021-05-17 11:02:35 +00:00
|
|
|
void registerStorageSQLite(StorageFactory & factory)
|
|
|
|
{
|
2021-07-10 08:05:23 +00:00
|
|
|
factory.registerStorage("SQLite", [](const StorageFactory::Arguments & args) -> StoragePtr
|
|
|
|
{
|
|
|
|
ASTs & engine_args = args.engine_args;
|
|
|
|
|
|
|
|
if (engine_args.size() != 2)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "SQLite database requires 2 arguments: database path, table name");
|
2021-07-10 08:05:23 +00:00
|
|
|
|
|
|
|
for (auto & engine_arg : engine_args)
|
|
|
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
|
|
|
|
|
2022-06-23 20:04:06 +00:00
|
|
|
const auto database_path = checkAndGetLiteralArgument<String>(engine_args[0], "database_path");
|
|
|
|
const auto table_name = checkAndGetLiteralArgument<String>(engine_args[1], "table_name");
|
2021-07-10 08:05:23 +00:00
|
|
|
|
2021-07-26 07:05:28 +00:00
|
|
|
auto sqlite_db = openSQLiteDB(database_path, args.getContext(), /* throw_on_error */!args.attach);
|
2021-07-13 12:38:37 +00:00
|
|
|
|
2022-04-19 20:47:29 +00:00
|
|
|
return std::make_shared<StorageSQLite>(args.table_id, sqlite_db, database_path,
|
2021-07-10 08:05:23 +00:00
|
|
|
table_name, args.columns, args.constraints, args.getContext());
|
|
|
|
},
|
|
|
|
{
|
2023-05-19 00:44:27 +00:00
|
|
|
.supports_schema_inference = true,
|
2021-07-10 08:05:23 +00:00
|
|
|
.source_access_type = AccessType::SQLITE,
|
|
|
|
});
|
2021-05-17 11:02:35 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|
2021-07-08 13:27:30 +00:00
|
|
|
|
|
|
|
#endif
|