2020-04-16 12:31:57 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
2021-03-24 18:15:31 +00:00
|
|
|
#include "config_core.h"
|
2020-04-16 12:31:57 +00:00
|
|
|
#endif
|
2017-04-19 00:25:57 +00:00
|
|
|
|
2020-04-16 12:31:57 +00:00
|
|
|
#if USE_MYSQL
|
2021-03-24 18:15:31 +00:00
|
|
|
#include <vector>
|
|
|
|
#include <Columns/ColumnNullable.h>
|
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <Columns/ColumnDecimal.h>
|
|
|
|
#include <Columns/ColumnFixedString.h>
|
|
|
|
#include <DataTypes/IDataType.h>
|
2021-06-24 12:05:47 +00:00
|
|
|
#include <DataTypes/DataTypeEnum.h>
|
2021-03-24 18:15:31 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
#include <IO/ReadBufferFromString.h>
|
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <Common/assert_cast.h>
|
2021-06-15 19:55:21 +00:00
|
|
|
#include <common/range.h>
|
2021-03-24 18:15:31 +00:00
|
|
|
#include <common/logger_useful.h>
|
|
|
|
#include "MySQLBlockInputStream.h"
|
|
|
|
|
2016-12-08 02:49:04 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2021-03-24 18:15:31 +00:00
|
|
|
|
2016-12-08 02:49:04 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
2020-11-20 22:47:04 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2016-12-08 02:49:04 +00:00
|
|
|
}
|
|
|
|
|
2021-04-10 01:37:56 +00:00
|
|
|
StreamSettings::StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_, size_t max_retry_)
|
2021-04-16 01:42:55 +00:00
|
|
|
: max_read_mysql_row_nums((settings.external_storage_max_read_rows) ? settings.external_storage_max_read_rows : settings.max_block_size)
|
2021-04-14 05:41:15 +00:00
|
|
|
, max_read_mysql_bytes_size(settings.external_storage_max_read_bytes)
|
2021-04-13 01:46:33 +00:00
|
|
|
, auto_close(auto_close_)
|
2021-04-10 01:37:56 +00:00
|
|
|
, fetch_by_name(fetch_by_name_)
|
|
|
|
, default_num_tries_on_connection_loss(max_retry_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2020-09-25 11:12:48 +00:00
|
|
|
MySQLBlockInputStream::Connection::Connection(
|
|
|
|
const mysqlxx::PoolWithFailover::Entry & entry_,
|
|
|
|
const std::string & query_str)
|
|
|
|
: entry(entry_)
|
|
|
|
, query{entry->query(query_str)}
|
|
|
|
, result{query.use()}
|
|
|
|
{
|
|
|
|
}
|
2016-12-08 02:49:04 +00:00
|
|
|
|
2021-07-26 18:17:28 +00:00
|
|
|
/// Used in MaterializedMySQL and in doInvalidateQuery for dictionary source.
|
2016-12-08 02:49:04 +00:00
|
|
|
MySQLBlockInputStream::MySQLBlockInputStream(
|
2020-09-25 11:12:48 +00:00
|
|
|
const mysqlxx::PoolWithFailover::Entry & entry,
|
|
|
|
const std::string & query_str,
|
|
|
|
const Block & sample_block,
|
2021-04-10 01:37:56 +00:00
|
|
|
const StreamSettings & settings_)
|
2021-03-24 18:15:31 +00:00
|
|
|
: log(&Poco::Logger::get("MySQLBlockInputStream"))
|
|
|
|
, connection{std::make_unique<Connection>(entry, query_str)}
|
2021-04-10 01:37:56 +00:00
|
|
|
, settings{std::make_unique<StreamSettings>(settings_)}
|
2016-12-08 02:49:04 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
description.init(sample_block);
|
2020-12-09 08:19:09 +00:00
|
|
|
initPositionMappingFromQueryResultStructure();
|
2016-12-08 02:49:04 +00:00
|
|
|
}
|
|
|
|
|
2021-03-27 20:49:26 +00:00
|
|
|
/// For descendant MySQLWithFailoverBlockInputStream
|
2021-04-10 01:37:56 +00:00
|
|
|
MySQLBlockInputStream::MySQLBlockInputStream(const Block &sample_block_, const StreamSettings & settings_)
|
2021-03-24 18:15:31 +00:00
|
|
|
: log(&Poco::Logger::get("MySQLBlockInputStream"))
|
2021-04-10 01:37:56 +00:00
|
|
|
, settings(std::make_unique<StreamSettings>(settings_))
|
2021-03-24 18:15:31 +00:00
|
|
|
{
|
|
|
|
description.init(sample_block_);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Used by MySQL storage / table function and dictionary source.
|
|
|
|
MySQLWithFailoverBlockInputStream::MySQLWithFailoverBlockInputStream(
|
2021-03-27 14:35:44 +00:00
|
|
|
mysqlxx::PoolWithFailoverPtr pool_,
|
2021-03-24 18:15:31 +00:00
|
|
|
const std::string & query_str_,
|
|
|
|
const Block & sample_block_,
|
2021-04-10 01:37:56 +00:00
|
|
|
const StreamSettings & settings_)
|
|
|
|
: MySQLBlockInputStream(sample_block_, settings_)
|
|
|
|
, pool(pool_)
|
|
|
|
, query_str(query_str_)
|
2021-03-24 18:15:31 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
void MySQLWithFailoverBlockInputStream::readPrefix()
|
|
|
|
{
|
2021-03-27 20:49:26 +00:00
|
|
|
size_t count_connect_attempts = 0;
|
2021-03-24 18:15:31 +00:00
|
|
|
|
|
|
|
/// For recovering from "Lost connection to MySQL server during query" errors
|
|
|
|
while (true)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2021-03-27 14:35:44 +00:00
|
|
|
connection = std::make_unique<Connection>(pool->get(), query_str);
|
2021-03-24 18:15:31 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
catch (const mysqlxx::ConnectionLost & ecl) /// There are two retriable failures: CR_SERVER_GONE_ERROR, CR_SERVER_LOST
|
|
|
|
{
|
2021-04-10 01:37:56 +00:00
|
|
|
LOG_WARNING(log, "Failed connection ({}/{}). Trying to reconnect... (Info: {})", count_connect_attempts, settings->default_num_tries_on_connection_loss, ecl.displayText());
|
2021-03-24 18:15:31 +00:00
|
|
|
}
|
|
|
|
|
2021-04-10 01:37:56 +00:00
|
|
|
if (++count_connect_attempts > settings->default_num_tries_on_connection_loss)
|
2021-03-24 18:15:31 +00:00
|
|
|
{
|
2021-04-10 01:37:56 +00:00
|
|
|
LOG_ERROR(log, "Failed to create connection to MySQL. ({}/{})", count_connect_attempts, settings->default_num_tries_on_connection_loss);
|
2021-03-24 18:15:31 +00:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
initPositionMappingFromQueryResultStructure();
|
|
|
|
}
|
2016-12-08 02:49:04 +00:00
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using ValueType = ExternalResultDescription::ValueType;
|
|
|
|
|
2021-04-10 01:37:56 +00:00
|
|
|
void insertValue(const IDataType & data_type, IColumn & column, const ValueType type, const mysqlxx::Value & value, size_t & read_bytes_size)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
switch (type)
|
|
|
|
{
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtUInt8:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnUInt8 &>(column).insertValue(value.getUInt());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 1;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtUInt16:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnUInt16 &>(column).insertValue(value.getUInt());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 2;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtUInt32:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnUInt32 &>(column).insertValue(value.getUInt());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 4;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtUInt64:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnUInt64 &>(column).insertValue(value.getUInt());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 8;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtInt8:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnInt8 &>(column).insertValue(value.getInt());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 1;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtInt16:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnInt16 &>(column).insertValue(value.getInt());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 2;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtInt32:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnInt32 &>(column).insertValue(value.getInt());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 4;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtInt64:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnInt64 &>(column).insertValue(value.getInt());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 8;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtFloat32:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnFloat32 &>(column).insertValue(value.getDouble());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 4;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtFloat64:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnFloat64 &>(column).insertValue(value.getDouble());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 8;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2021-06-24 12:05:47 +00:00
|
|
|
case ValueType::vtEnum8:
|
2021-06-28 07:22:13 +00:00
|
|
|
assert_cast<ColumnInt8 &>(column).insertValue(assert_cast<const DataTypeEnum<Int8> &>(data_type).castToValue(value.data()).get<Int8>());
|
2021-06-24 12:05:47 +00:00
|
|
|
read_bytes_size += assert_cast<ColumnInt8 &>(column).byteSize();
|
|
|
|
break;
|
|
|
|
case ValueType::vtEnum16:
|
2021-06-28 07:22:13 +00:00
|
|
|
assert_cast<ColumnInt16 &>(column).insertValue(assert_cast<const DataTypeEnum<Int16> &>(data_type).castToValue(value.data()).get<Int16>());
|
2021-06-24 12:05:47 +00:00
|
|
|
read_bytes_size += assert_cast<ColumnInt16 &>(column).byteSize();
|
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtString:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnString &>(column).insertData(value.data(), value.size());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += assert_cast<ColumnString &>(column).byteSize();
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtDate:
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnUInt16 &>(column).insertValue(UInt16(value.getDate().getDayNum()));
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 2;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtDateTime:
|
2021-03-15 19:23:27 +00:00
|
|
|
{
|
|
|
|
ReadBufferFromString in(value);
|
|
|
|
time_t time = 0;
|
2021-06-21 08:18:38 +00:00
|
|
|
readDateTimeText(time, in, assert_cast<const DataTypeDateTime &>(data_type).getTimeZone());
|
2021-03-15 20:40:33 +00:00
|
|
|
if (time < 0)
|
|
|
|
time = 0;
|
2021-03-15 19:23:27 +00:00
|
|
|
assert_cast<ColumnUInt32 &>(column).insertValue(time);
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += 4;
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2021-03-15 19:23:27 +00:00
|
|
|
}
|
2019-08-03 11:02:40 +00:00
|
|
|
case ValueType::vtUUID:
|
2021-05-03 22:59:38 +00:00
|
|
|
assert_cast<ColumnUUID &>(column).insert(parse<UUID>(value.data(), value.size()));
|
|
|
|
read_bytes_size += assert_cast<ColumnUUID &>(column).byteSize();
|
2018-12-10 15:25:45 +00:00
|
|
|
break;
|
2020-09-09 12:18:02 +00:00
|
|
|
case ValueType::vtDateTime64:[[fallthrough]];
|
|
|
|
case ValueType::vtDecimal32: [[fallthrough]];
|
|
|
|
case ValueType::vtDecimal64: [[fallthrough]];
|
2020-09-10 03:37:00 +00:00
|
|
|
case ValueType::vtDecimal128:[[fallthrough]];
|
|
|
|
case ValueType::vtDecimal256:
|
2020-09-09 12:18:02 +00:00
|
|
|
{
|
|
|
|
ReadBuffer buffer(const_cast<char *>(value.data()), value.size(), 0);
|
2021-03-09 14:46:52 +00:00
|
|
|
data_type.getDefaultSerialization()->deserializeWholeText(column, buffer, FormatSettings{});
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += column.sizeOfValueIfFixed();
|
2020-09-09 12:18:02 +00:00
|
|
|
break;
|
|
|
|
}
|
2020-12-27 15:49:34 +00:00
|
|
|
case ValueType::vtFixedString:
|
|
|
|
assert_cast<ColumnFixedString &>(column).insertData(value.data(), value.size());
|
2021-04-10 01:37:56 +00:00
|
|
|
read_bytes_size += column.sizeOfValueIfFixed();
|
2020-12-27 15:49:34 +00:00
|
|
|
break;
|
2020-11-20 22:47:04 +00:00
|
|
|
default:
|
|
|
|
throw Exception("Unsupported value type", ErrorCodes::NOT_IMPLEMENTED);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2017-12-15 03:12:04 +00:00
|
|
|
|
2018-12-10 15:25:45 +00:00
|
|
|
void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); }
|
2016-12-08 02:49:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
Block MySQLBlockInputStream::readImpl()
|
|
|
|
{
|
2020-09-25 11:12:48 +00:00
|
|
|
auto row = connection->result.fetch();
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!row)
|
2019-05-23 13:09:07 +00:00
|
|
|
{
|
2021-04-10 01:37:56 +00:00
|
|
|
if (settings->auto_close)
|
2020-09-25 11:12:48 +00:00
|
|
|
connection->entry.disconnect();
|
2021-03-27 20:49:26 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return {};
|
2019-05-23 13:09:07 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-15 03:12:04 +00:00
|
|
|
MutableColumns columns(description.sample_block.columns());
|
2021-06-15 19:55:21 +00:00
|
|
|
for (const auto i : collections::range(0, columns.size()))
|
2017-12-15 03:12:04 +00:00
|
|
|
columns[i] = description.sample_block.getByPosition(i).column->cloneEmpty();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
size_t num_rows = 0;
|
2021-04-10 01:37:56 +00:00
|
|
|
size_t read_bytes_size = 0;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
while (row)
|
|
|
|
{
|
2020-11-24 15:31:43 +00:00
|
|
|
for (size_t index = 0; index < position_mapping.size(); ++index)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-11-24 15:31:43 +00:00
|
|
|
const auto value = row[position_mapping[index]];
|
|
|
|
const auto & sample = description.sample_block.getByPosition(index);
|
|
|
|
|
2021-01-03 20:05:14 +00:00
|
|
|
bool is_type_nullable = description.types[index].second;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!value.isNull())
|
2018-10-12 02:26:48 +00:00
|
|
|
{
|
2021-01-03 20:05:14 +00:00
|
|
|
if (is_type_nullable)
|
2018-10-12 02:26:48 +00:00
|
|
|
{
|
2020-11-24 15:31:43 +00:00
|
|
|
ColumnNullable & column_nullable = assert_cast<ColumnNullable &>(*columns[index]);
|
2020-09-09 12:18:02 +00:00
|
|
|
const auto & data_type = assert_cast<const DataTypeNullable &>(*sample.type);
|
2021-04-10 01:37:56 +00:00
|
|
|
insertValue(*data_type.getNestedType(), column_nullable.getNestedColumn(), description.types[index].first, value, read_bytes_size);
|
2021-01-03 20:05:14 +00:00
|
|
|
column_nullable.getNullMapData().emplace_back(false);
|
2018-10-12 02:26:48 +00:00
|
|
|
}
|
|
|
|
else
|
2021-01-03 20:05:14 +00:00
|
|
|
{
|
2021-04-10 01:37:56 +00:00
|
|
|
insertValue(*sample.type, *columns[index], description.types[index].first, value, read_bytes_size);
|
2021-01-03 20:05:14 +00:00
|
|
|
}
|
2018-10-12 02:26:48 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
2021-01-03 20:05:14 +00:00
|
|
|
{
|
2020-11-24 15:31:43 +00:00
|
|
|
insertDefaultValue(*columns[index], *sample.column);
|
2021-01-03 20:05:14 +00:00
|
|
|
|
|
|
|
if (is_type_nullable)
|
|
|
|
{
|
|
|
|
ColumnNullable & column_nullable = assert_cast<ColumnNullable &>(*columns[index]);
|
|
|
|
column_nullable.getNullMapData().back() = true;
|
|
|
|
}
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
++num_rows;
|
2021-04-14 05:41:15 +00:00
|
|
|
if (num_rows == settings->max_read_mysql_row_nums || (settings->max_read_mysql_bytes_size && read_bytes_size >= settings->max_read_mysql_bytes_size))
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
|
2020-09-25 11:12:48 +00:00
|
|
|
row = connection->result.fetch();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-12-15 03:12:04 +00:00
|
|
|
return description.sample_block.cloneWithColumns(std::move(columns));
|
2016-12-08 02:49:04 +00:00
|
|
|
}
|
|
|
|
|
2020-12-09 08:19:09 +00:00
|
|
|
void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure()
|
|
|
|
{
|
|
|
|
position_mapping.resize(description.sample_block.columns());
|
|
|
|
|
2021-04-10 01:37:56 +00:00
|
|
|
if (!settings->fetch_by_name)
|
2020-12-09 08:19:09 +00:00
|
|
|
{
|
|
|
|
if (description.sample_block.columns() != connection->result.getNumFields())
|
|
|
|
throw Exception{"mysqlxx::UseQueryResult contains " + toString(connection->result.getNumFields()) + " columns while "
|
|
|
|
+ toString(description.sample_block.columns()) + " expected", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
|
|
|
|
|
2021-06-15 19:55:21 +00:00
|
|
|
for (const auto idx : collections::range(0, connection->result.getNumFields()))
|
2020-12-09 08:19:09 +00:00
|
|
|
position_mapping[idx] = idx;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
const auto & sample_names = description.sample_block.getNames();
|
|
|
|
std::unordered_set<std::string> missing_names(sample_names.begin(), sample_names.end());
|
|
|
|
|
|
|
|
size_t fields_size = connection->result.getNumFields();
|
|
|
|
|
2021-06-15 19:55:21 +00:00
|
|
|
for (const size_t & idx : collections::range(0, fields_size))
|
2020-12-09 08:19:09 +00:00
|
|
|
{
|
|
|
|
const auto & field_name = connection->result.getFieldName(idx);
|
|
|
|
if (description.sample_block.has(field_name))
|
|
|
|
{
|
|
|
|
const auto & position = description.sample_block.getPositionByName(field_name);
|
|
|
|
position_mapping[position] = idx;
|
|
|
|
missing_names.erase(field_name);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!missing_names.empty())
|
|
|
|
{
|
|
|
|
WriteBufferFromOwnString exception_message;
|
|
|
|
for (auto iter = missing_names.begin(); iter != missing_names.end(); ++iter)
|
|
|
|
{
|
|
|
|
if (iter != missing_names.begin())
|
|
|
|
exception_message << ", ";
|
|
|
|
exception_message << *iter;
|
|
|
|
}
|
|
|
|
|
|
|
|
throw Exception("mysqlxx::UseQueryResult must be contain the" + exception_message.str() + " columns.",
|
|
|
|
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-12-08 02:49:04 +00:00
|
|
|
}
|
2017-04-19 00:25:57 +00:00
|
|
|
|
|
|
|
#endif
|