mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Add table function sqlite, fix type conversions
This commit is contained in:
parent
aecb344baa
commit
976f4dffda
@ -11,23 +11,33 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Poco/String.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
static DataTypePtr convertSQLiteDataType(std::string type /* , bool is_nullable */)
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SQLITE_ENGINE_ERROR;
|
||||
}
|
||||
|
||||
static DataTypePtr convertSQLiteDataType(String type)
|
||||
{
|
||||
DataTypePtr res;
|
||||
type = Poco::toLower(type);
|
||||
|
||||
std::transform(std::begin(type), std::end(type), std::begin(type), tolower);
|
||||
|
||||
// https://www.sqlite.org/datatype3.html#determination_of_column_affinity
|
||||
if (type.find("int") != std::string::npos)
|
||||
if (type == "tinyint")
|
||||
res = std::make_shared<DataTypeInt8>();
|
||||
else if (type == "smallint")
|
||||
res = std::make_shared<DataTypeInt16>();
|
||||
else if (type.starts_with("int") || type == "mediumint")
|
||||
res = std::make_shared<DataTypeInt32>();
|
||||
else if (type == "bigint")
|
||||
res = std::make_shared<DataTypeInt64>();
|
||||
else if (
|
||||
type.find("char") != std::string::npos || type.find("clob") != std::string::npos || type.find("text") != std::string::npos
|
||||
|| type.empty() || type.find("blob") != std::string::npos)
|
||||
res = std::make_shared<DataTypeString>();
|
||||
else if (type.find("real") != std::string::npos || type.find("floa") != std::string::npos || type.find("doub") != std::string::npos)
|
||||
else if (type == "float")
|
||||
res = std::make_shared<DataTypeFloat32>();
|
||||
else if (type.starts_with("double") || type == "real")
|
||||
res = std::make_shared<DataTypeFloat64>();
|
||||
else
|
||||
res = std::make_shared<DataTypeString>(); // No decimal when fetching data through API
|
||||
@ -35,15 +45,15 @@ static DataTypePtr convertSQLiteDataType(std::string type /* , bool is_nullable
|
||||
return res;
|
||||
}
|
||||
|
||||
std::shared_ptr<NamesAndTypesList> fetchSQLiteTableStructure(sqlite3 * connection, const String & sqlite_table_name /* , bool use_nulls */)
|
||||
|
||||
std::shared_ptr<NamesAndTypesList> fetchSQLiteTableStructure(sqlite3 * connection, const String & sqlite_table_name)
|
||||
{
|
||||
auto columns = NamesAndTypesList();
|
||||
auto query = fmt::format("pragma table_info({});", quoteString(sqlite_table_name));
|
||||
|
||||
std::string query = fmt::format("pragma table_info({});", quoteString(sqlite_table_name));
|
||||
|
||||
auto callback_get_data = [](void * res, int col_num, char ** data_by_col, char ** col_names) -> int {
|
||||
auto callback_get_data = [](void * res, int col_num, char ** data_by_col, char ** col_names) -> int
|
||||
{
|
||||
NameAndTypePair name_and_type;
|
||||
|
||||
bool is_nullable = false;
|
||||
|
||||
for (int i = 0; i < col_num; ++i)
|
||||
@ -71,14 +81,16 @@ std::shared_ptr<NamesAndTypesList> fetchSQLiteTableStructure(sqlite3 * connectio
|
||||
};
|
||||
|
||||
char * err_message = nullptr;
|
||||
|
||||
int status = sqlite3_exec(connection, query.c_str(), callback_get_data, &columns, &err_message);
|
||||
|
||||
if (status != SQLITE_OK)
|
||||
{
|
||||
String err_msg(err_message);
|
||||
sqlite3_free(err_message);
|
||||
throw Exception(status, "SQLITE_ERR {}: {}", status, err_msg);
|
||||
|
||||
throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR,
|
||||
"Failed to fetch SQLite data. Status: {}. Message: {}",
|
||||
status, err_msg);
|
||||
}
|
||||
|
||||
if (columns.empty())
|
||||
|
@ -5,14 +5,15 @@
|
||||
#endif
|
||||
|
||||
#if USE_SQLITE
|
||||
#include <sqlite3.h>
|
||||
|
||||
#include <Storages/StorageSQLite.h>
|
||||
#include <sqlite3.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
std::shared_ptr<NamesAndTypesList> fetchSQLiteTableStructure(sqlite3 * connection, const String & sqlite_table_name /* , bool use_nulls */);
|
||||
|
||||
std::shared_ptr<NamesAndTypesList> fetchSQLiteTableStructure(sqlite3 * connection,
|
||||
const String & sqlite_table_name);
|
||||
}
|
||||
|
||||
#endif
|
||||
|
89
src/TableFunctions/TableFunctionSQLite.cpp
Normal file
89
src/TableFunctions/TableFunctionSQLite.cpp
Normal file
@ -0,0 +1,89 @@
|
||||
#include <TableFunctions/TableFunctionSQLite.h>
|
||||
|
||||
#if USE_SQLITE
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Databases/SQLite/fetchSQLiteTableStructure.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int SQLITE_ENGINE_ERROR;
|
||||
}
|
||||
|
||||
|
||||
StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/,
|
||||
ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const
|
||||
{
|
||||
auto columns = getActualTableStructure(context);
|
||||
|
||||
auto storage = StorageSQLite::create(StorageID(getDatabaseName(), table_name),
|
||||
sqlite_db,
|
||||
remote_table_name,
|
||||
columns, ConstraintsDescription{}, context);
|
||||
|
||||
storage->startup();
|
||||
return storage;
|
||||
}
|
||||
|
||||
|
||||
ColumnsDescription TableFunctionSQLite::getActualTableStructure(ContextPtr /* context */) const
|
||||
{
|
||||
auto columns = fetchSQLiteTableStructure(sqlite_db.get(), remote_table_name);
|
||||
return ColumnsDescription{*columns};
|
||||
}
|
||||
|
||||
|
||||
void TableFunctionSQLite::parseArguments(const ASTPtr & ast_function, ContextPtr context)
|
||||
{
|
||||
const auto & func_args = ast_function->as<ASTFunction &>();
|
||||
|
||||
if (!func_args.arguments)
|
||||
throw Exception("Table function 'sqlite' must have arguments.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
ASTs & args = func_args.arguments->children;
|
||||
|
||||
if (args.size() != 2)
|
||||
throw Exception("SQLite database requires 2 arguments: database path, table name",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
|
||||
database_path = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
remote_table_name = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
sqlite3 * tmp_sqlite_db = nullptr;
|
||||
int status = sqlite3_open(database_path.c_str(), &tmp_sqlite_db);
|
||||
if (status != SQLITE_OK)
|
||||
throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR,
|
||||
"Failed to open sqlite database. Status: {}. Message: {}",
|
||||
status, sqlite3_errstr(status));
|
||||
|
||||
sqlite_db = std::shared_ptr<sqlite3>(tmp_sqlite_db, sqlite3_close);
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionSQLite(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionSQLite>();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
36
src/TableFunctions/TableFunctionSQLite.h
Normal file
36
src/TableFunctions/TableFunctionSQLite.h
Normal file
@ -0,0 +1,36 @@
|
||||
#pragma once
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_SQLITE
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Storages/StorageSQLite.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TableFunctionSQLite : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "sqlite";
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
private:
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function, ContextPtr context,
|
||||
const std::string & table_name, ColumnsDescription cached_columns) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "SQLite"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
|
||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||
|
||||
String database_path, remote_table_name;
|
||||
std::shared_ptr<sqlite3> sqlite_db;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -42,6 +42,10 @@ void registerTableFunctions()
|
||||
registerTableFunctionPostgreSQL(factory);
|
||||
#endif
|
||||
|
||||
#if USE_SQLITE
|
||||
registerTableFunctionSQLite(factory);
|
||||
#endif
|
||||
|
||||
registerTableFunctionDictionary(factory);
|
||||
}
|
||||
|
||||
|
@ -42,6 +42,10 @@ void registerTableFunctionMySQL(TableFunctionFactory & factory);
|
||||
void registerTableFunctionPostgreSQL(TableFunctionFactory & factory);
|
||||
#endif
|
||||
|
||||
#if USE_SQLITE
|
||||
void registerTableFunctionSQLite(TableFunctionFactory & factory);
|
||||
#endif
|
||||
|
||||
void registerTableFunctionDictionary(TableFunctionFactory & factory);
|
||||
|
||||
void registerTableFunctions();
|
||||
|
@ -5,12 +5,12 @@ table2
|
||||
table3
|
||||
describe table:
|
||||
col1 Nullable(String)
|
||||
col2 Nullable(Int64)
|
||||
col1 Nullable(Int64)
|
||||
col2 Nullable(Int16)
|
||||
col1 Nullable(Int32)
|
||||
col2 Nullable(String)
|
||||
describe table:
|
||||
CREATE TABLE SQLite.table1\n(\n `col1` Nullable(String),\n `col2` Nullable(Int64)\n)\nENGINE = SQLite
|
||||
CREATE TABLE SQLite.table2\n(\n `col1` Nullable(Int64),\n `col2` Nullable(String)\n)\nENGINE = SQLite
|
||||
CREATE TABLE SQLite.table1\n(\n `col1` Nullable(String),\n `col2` Nullable(Int16)\n)\nENGINE = SQLite
|
||||
CREATE TABLE SQLite.table2\n(\n `col1` Nullable(Int32),\n `col2` Nullable(String)\n)\nENGINE = SQLite
|
||||
select *:
|
||||
line1 1
|
||||
line2 2
|
||||
@ -36,3 +36,10 @@ CREATE TABLE default.sqlite_table3\n(\n `col1` String,\n `col2` Int32\n)\n
|
||||
not a null 2
|
||||
3
|
||||
4
|
||||
test types
|
||||
CREATE TABLE SQLite.table4\n(\n `a` Nullable(Int32),\n `b` Nullable(Int32),\n `c` Nullable(Int8),\n `d` Nullable(Int16),\n `e` Nullable(Int32),\n `bigint` Nullable(String),\n `int2` Nullable(String),\n `int8` Nullable(String)\n)\nENGINE = SQLite
|
||||
CREATE TABLE SQLite.table5\n(\n `a` Nullable(String),\n `b` Nullable(String),\n `c` Nullable(Float64),\n `d` Nullable(Float64),\n `e` Nullable(Float64),\n `f` Nullable(Float32)\n)\nENGINE = SQLite
|
||||
test table function
|
||||
line1 1
|
||||
line2 2
|
||||
line3 3
|
||||
|
@ -60,4 +60,15 @@ ${CLICKHOUSE_CLIENT} --query='SHOW CREATE TABLE sqlite_table3;' | sed -r 's/(.*S
|
||||
# ${CLICKHOUSE_CLIENT} --query="INSERT INTO sqlite_table3 VALUES ('line6', 6);"
|
||||
# ${CLICKHOUSE_CLIENT} --query="INSERT INTO sqlite_table3 VALUES (NULL, 7);"
|
||||
${CLICKHOUSE_CLIENT} --query='SELECT * FROM sqlite_table3 ORDER BY col2'
|
||||
|
||||
sqlite3 $CUR_DIR/db2 'DROP TABLE IF EXISTS table4'
|
||||
sqlite3 $CUR_DIR/db2 'CREATE TABLE table4 (a int, b integer, c tinyint, d smallint, e mediumint, bigint, int2, int8)'
|
||||
${CLICKHOUSE_CLIENT} --query="select 'test types'";
|
||||
${CLICKHOUSE_CLIENT} --query='SHOW CREATE TABLE sqlite_database_2.table4;' | sed -r 's/(.*SQLite)(.*)/\1/'
|
||||
sqlite3 $CUR_DIR/db2 'CREATE TABLE table5 (a character(20), b varchar(10), c real, d double, e double precision, f float)'
|
||||
${CLICKHOUSE_CLIENT} --query='SHOW CREATE TABLE sqlite_database_2.table5;' | sed -r 's/(.*SQLite)(.*)/\1/'
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="select 'test table function'";
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT * FROM sqlite('${DATA_FILE1}', 'table1') ORDER BY col2"
|
||||
|
||||
rm ${DATA_FILE2}
|
||||
|
Loading…
Reference in New Issue
Block a user