mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
fix some bugs, fix some code styles
This commit is contained in:
parent
200076b593
commit
5d91b4f2fd
@ -25,6 +25,7 @@ namespace DB
|
|||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
extern const int BAD_ARGUMENTS;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -32,7 +33,7 @@ StorageMySQL::StorageMySQL(const std::string & name,
|
|||||||
mysqlxx::Pool && pool,
|
mysqlxx::Pool && pool,
|
||||||
const std::string & remote_database_name,
|
const std::string & remote_database_name,
|
||||||
const std::string & remote_table_name,
|
const std::string & remote_table_name,
|
||||||
const bool & replace_query,
|
const bool replace_query,
|
||||||
const std::string & on_duplicate_clause,
|
const std::string & on_duplicate_clause,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const Context & context)
|
const Context & context)
|
||||||
@ -74,8 +75,7 @@ BlockInputStreams StorageMySQL::read(
|
|||||||
class StorageMySQLBlockOutputStream : public IBlockOutputStream
|
class StorageMySQLBlockOutputStream : public IBlockOutputStream
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
explicit StorageMySQLBlockOutputStream(
|
explicit StorageMySQLBlockOutputStream(const StorageMySQL & storage,
|
||||||
const StorageMySQL & storage,
|
|
||||||
const std::string & remote_database_name,
|
const std::string & remote_database_name,
|
||||||
const std::string & remote_table_name ,
|
const std::string & remote_table_name ,
|
||||||
const mysqlxx::PoolWithFailover::Entry & entry,
|
const mysqlxx::PoolWithFailover::Entry & entry,
|
||||||
@ -112,15 +112,16 @@ public:
|
|||||||
void writeBlockData(const Block & block)
|
void writeBlockData(const Block & block)
|
||||||
{
|
{
|
||||||
WriteBufferFromOwnString sqlbuf;
|
WriteBufferFromOwnString sqlbuf;
|
||||||
// If both `replace_query` and `on_duplicate_clause` are specified, only use the `on_duplicate_clause`.
|
sqlbuf << (storage.replace_query ? "REPLACE" : "INSERT") << " INTO ";
|
||||||
sqlbuf << ( (storage.replace_query && storage.on_duplicate_clause.empty()) ? "REPLACE" : "INSERT");
|
sqlbuf << backQuoteIfNeed(remote_database_name) << "." << backQuoteIfNeed(remote_table_name);
|
||||||
sqlbuf << " INTO `" << remote_database_name << "`.`" << remote_table_name << "`"
|
sqlbuf << " ( " << dumpNamesWithBackQuote(block) << " ) VALUES ";
|
||||||
<< " ( " << block.dumpNames() << " ) VALUES ";
|
|
||||||
|
|
||||||
auto writer = FormatFactory().getOutput("Values", sqlbuf, storage.getSampleBlock(), storage.context);
|
auto writer = FormatFactory().getOutput("Values", sqlbuf, storage.getSampleBlock(), storage.context);
|
||||||
writer->write(block);
|
writer->write(block);
|
||||||
|
|
||||||
if (!storage.on_duplicate_clause.empty())
|
if (!storage.on_duplicate_clause.empty())
|
||||||
sqlbuf << " ON DUPLICATE KEY " << storage.on_duplicate_clause;
|
sqlbuf << " ON DUPLICATE KEY " << storage.on_duplicate_clause;
|
||||||
|
|
||||||
sqlbuf << ";";
|
sqlbuf << ";";
|
||||||
|
|
||||||
auto query = this->entry->query(sqlbuf.str());
|
auto query = this->entry->query(sqlbuf.str());
|
||||||
@ -129,8 +130,9 @@ public:
|
|||||||
|
|
||||||
Blocks splitBlocks(const Block & block, const size_t & max_rows) const
|
Blocks splitBlocks(const Block & block, const size_t & max_rows) const
|
||||||
{
|
{
|
||||||
// Avoid Excessive copy when block is small enough
|
/// Avoid Excessive copy when block is small enough
|
||||||
if(block.rows() <= max_rows) return Blocks{std::move(block)};
|
if (block.rows() <= max_rows)
|
||||||
|
return Blocks{std::move(block)};
|
||||||
|
|
||||||
const size_t splited_block_size = ceil(block.rows() * 1.0 / max_rows);
|
const size_t splited_block_size = ceil(block.rows() * 1.0 / max_rows);
|
||||||
Blocks splitted_blocks(splited_block_size);
|
Blocks splitted_blocks(splited_block_size);
|
||||||
@ -144,7 +146,7 @@ public:
|
|||||||
size_t limits = max_batch_rows;
|
size_t limits = max_batch_rows;
|
||||||
for (size_t idx = 0; idx < splited_block_size; ++idx)
|
for (size_t idx = 0; idx < splited_block_size; ++idx)
|
||||||
{
|
{
|
||||||
// For last batch, limits should be the remain size
|
/// For last batch, limits should be the remain size
|
||||||
if (idx == splited_block_size - 1) limits = rows - offsets;
|
if (idx == splited_block_size - 1) limits = rows - offsets;
|
||||||
for (size_t col_idx = 0; col_idx < columns; ++col_idx)
|
for (size_t col_idx = 0; col_idx < columns; ++col_idx)
|
||||||
{
|
{
|
||||||
@ -156,6 +158,18 @@ public:
|
|||||||
return splitted_blocks;
|
return splitted_blocks;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::string dumpNamesWithBackQuote(const Block & block) const
|
||||||
|
{
|
||||||
|
WriteBufferFromOwnString out;
|
||||||
|
for (auto it = block.begin(); it != block.end(); ++it)
|
||||||
|
{
|
||||||
|
if (it != block.begin())
|
||||||
|
out << ", ";
|
||||||
|
out << backQuoteIfNeed(it->name);
|
||||||
|
}
|
||||||
|
return out.str();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const StorageMySQL & storage;
|
const StorageMySQL & storage;
|
||||||
@ -198,8 +212,15 @@ void registerStorageMySQL(StorageFactory & factory)
|
|||||||
|
|
||||||
bool replace_query = false;
|
bool replace_query = false;
|
||||||
std::string on_duplicate_clause;
|
std::string on_duplicate_clause;
|
||||||
if(engine_args.size() >= 6) replace_query = static_cast<const ASTLiteral &>(*engine_args[5]).value.safeGet<UInt64>() > 0;
|
if (engine_args.size() >= 6)
|
||||||
if(engine_args.size() == 7) on_duplicate_clause = static_cast<const ASTLiteral &>(*engine_args[6]).value.safeGet<String>();
|
replace_query = static_cast<const ASTLiteral &>(*engine_args[5]).value.safeGet<UInt64>() > 0;
|
||||||
|
if (engine_args.size() == 7)
|
||||||
|
on_duplicate_clause = static_cast<const ASTLiteral &>(*engine_args[6]).value.safeGet<String>();
|
||||||
|
|
||||||
|
if (replace_query && !on_duplicate_clause.empty())
|
||||||
|
throw Exception(
|
||||||
|
"Only one of 'replace_query' and 'on_duplicate_clause' can be specified, or none of them",
|
||||||
|
ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
return StorageMySQL::create(
|
return StorageMySQL::create(
|
||||||
args.table_name,
|
args.table_name,
|
||||||
|
@ -24,7 +24,7 @@ public:
|
|||||||
mysqlxx::Pool && pool,
|
mysqlxx::Pool && pool,
|
||||||
const std::string & remote_database_name,
|
const std::string & remote_database_name,
|
||||||
const std::string & remote_table_name,
|
const std::string & remote_table_name,
|
||||||
const bool & replace_query,
|
const bool replace_query,
|
||||||
const std::string & on_duplicate_clause,
|
const std::string & on_duplicate_clause,
|
||||||
const ColumnsDescription & columns,
|
const ColumnsDescription & columns,
|
||||||
const Context & context);
|
const Context & context);
|
||||||
|
@ -29,8 +29,8 @@ namespace DB
|
|||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int LOGICAL_ERROR;
|
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
extern const int BAD_ARGUMENTS;;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -90,7 +90,7 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co
|
|||||||
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.arguments).children;
|
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.arguments).children;
|
||||||
|
|
||||||
if (args.size() < 5 || args.size() > 7)
|
if (args.size() < 5 || args.size() > 7)
|
||||||
throw Exception("Storage MySQL requires 5-7 parameters: MySQL('host:port', database, table, 'user', 'password'[, replace_query, 'on_duplicate_clause' ]).",
|
throw Exception("Table function 'mysql' requires 5-7 parameters: MySQL('host:port', database, table, 'user', 'password'[, replace_query, 'on_duplicate_clause']).",
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
for (size_t i = 0; i < args.size(); ++i)
|
for (size_t i = 0; i < args.size(); ++i)
|
||||||
@ -109,6 +109,11 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co
|
|||||||
if (args.size() == 7)
|
if (args.size() == 7)
|
||||||
on_duplicate_clause = static_cast<const ASTLiteral &>(*args[6]).value.safeGet<String>();
|
on_duplicate_clause = static_cast<const ASTLiteral &>(*args[6]).value.safeGet<String>();
|
||||||
|
|
||||||
|
if (replace_query && !on_duplicate_clause.empty())
|
||||||
|
throw Exception(
|
||||||
|
"Only one of 'replace_query' and 'on_duplicate_clause' can be specified, or none of them",
|
||||||
|
ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
/// 3306 is the default MySQL port number
|
/// 3306 is the default MySQL port number
|
||||||
auto parsed_host_port = parseAddress(host_port, 3306);
|
auto parsed_host_port = parseAddress(host_port, 3306);
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user