2017-10-14 00:53:01 +00:00
|
|
|
#include <Common/config.h>
|
2017-10-21 21:51:21 +00:00
|
|
|
#include <Common/config_version.h>
|
2017-10-14 00:53:01 +00:00
|
|
|
#if USE_RDKAFKA
|
|
|
|
|
|
|
|
#include <thread>
|
2017-12-16 01:45:11 +00:00
|
|
|
#include <boost/algorithm/string/replace.hpp>
|
2018-03-23 14:13:09 +00:00
|
|
|
#include <boost/algorithm/string/split.hpp>
|
2018-03-24 06:51:35 +00:00
|
|
|
#include <boost/algorithm/string/trim.hpp>
|
2018-06-03 20:48:36 +00:00
|
|
|
#include <Poco/Util/AbstractConfiguration.h>
|
2018-04-20 13:48:42 +00:00
|
|
|
#include <Common/Macros.h>
|
2017-10-03 23:39:24 +00:00
|
|
|
#include <Common/Exception.h>
|
|
|
|
#include <Common/setThreadName.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
2017-10-03 23:39:24 +00:00
|
|
|
#include <DataStreams/IProfilingBlockInputStream.h>
|
|
|
|
#include <DataStreams/LimitBlockInputStream.h>
|
2017-12-16 01:45:11 +00:00
|
|
|
#include <DataStreams/UnionBlockInputStream.h>
|
2017-10-03 23:39:24 +00:00
|
|
|
#include <DataStreams/copyData.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/InterpreterInsertQuery.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
2017-10-03 23:39:24 +00:00
|
|
|
#include <Parsers/ASTExpressionList.h>
|
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/ASTInsertQuery.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Parsers/ASTLiteral.h>
|
2018-08-01 17:23:50 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
|
|
|
#include <Storages/Kafka/KafkaSettings.h>
|
|
|
|
#include <Storages/Kafka/StorageKafka.h> // Y_IGNORE
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2018-06-05 19:46:49 +00:00
|
|
|
#include <IO/ReadBuffer.h>
|
2017-10-03 23:39:24 +00:00
|
|
|
#include <common/logger_useful.h>
|
|
|
|
|
2017-10-27 14:00:36 +00:00
|
|
|
#if __has_include(<rdkafka.h>) // maybe bundled
|
2018-06-21 15:23:20 +00:00
|
|
|
#include <rdkafka.h> // Y_IGNORE
|
2017-10-27 14:00:36 +00:00
|
|
|
#else // system
|
2018-06-21 15:23:20 +00:00
|
|
|
#include <librdkafka/rdkafka.h> // Y_IGNORE
|
2017-10-27 14:00:36 +00:00
|
|
|
#endif
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2018-06-05 19:46:49 +00:00
|
|
|
|
2017-10-03 23:39:24 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int INCORRECT_DATA;
|
|
|
|
extern const int UNKNOWN_EXCEPTION;
|
|
|
|
extern const int CANNOT_READ_FROM_ISTREAM;
|
2017-12-16 01:45:11 +00:00
|
|
|
extern const int INVALID_CONFIG_PARAMETER;
|
|
|
|
extern const int LOGICAL_ERROR;
|
2017-12-30 00:36:06 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
using namespace Poco::Util;
|
|
|
|
|
2017-10-03 23:39:24 +00:00
|
|
|
/// How long to wait for a single message (applies to each individual message)
|
2017-12-16 01:45:11 +00:00
|
|
|
static const auto READ_POLL_MS = 500;
|
|
|
|
static const auto CLEANUP_TIMEOUT_MS = 3000;
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
/// Configuration prefix
|
|
|
|
static const String CONFIG_PREFIX = "kafka";
|
2017-10-03 23:39:24 +00:00
|
|
|
|
|
|
|
class ReadBufferFromKafkaConsumer : public ReadBuffer
|
|
|
|
{
|
|
|
|
rd_kafka_t * consumer;
|
2017-12-16 01:45:11 +00:00
|
|
|
rd_kafka_message_t * current;
|
2018-07-18 05:22:01 +00:00
|
|
|
bool current_pending;
|
2017-10-03 23:39:24 +00:00
|
|
|
Poco::Logger * log;
|
2017-12-23 00:32:48 +00:00
|
|
|
size_t read_messages;
|
2018-07-18 05:22:01 +00:00
|
|
|
char row_delimiter;
|
2017-10-03 23:39:24 +00:00
|
|
|
|
|
|
|
bool nextImpl() override
|
|
|
|
{
|
2018-07-18 05:22:01 +00:00
|
|
|
if (current_pending)
|
|
|
|
{
|
|
|
|
BufferBase::set(reinterpret_cast<char *>(current->payload), current->len, 0);
|
|
|
|
current_pending = false;
|
|
|
|
return true;
|
|
|
|
}
|
2017-10-03 23:39:24 +00:00
|
|
|
|
|
|
|
// Process next buffered message
|
2017-12-16 01:45:11 +00:00
|
|
|
rd_kafka_message_t * msg = rd_kafka_consumer_poll(consumer, READ_POLL_MS);
|
|
|
|
if (msg == nullptr)
|
|
|
|
return false;
|
|
|
|
|
2017-10-03 23:39:24 +00:00
|
|
|
if (msg->err)
|
|
|
|
{
|
|
|
|
if (msg->err != RD_KAFKA_RESP_ERR__PARTITION_EOF)
|
2017-11-14 01:31:45 +00:00
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
LOG_ERROR(log, "Consumer error: " << rd_kafka_err2str(msg->err) << " " << rd_kafka_message_errstr(msg));
|
|
|
|
rd_kafka_message_destroy(msg);
|
|
|
|
return false;
|
2017-11-14 01:31:45 +00:00
|
|
|
}
|
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
// Reach EOF while reading current batch, skip it
|
|
|
|
LOG_TRACE(log, "EOF reached for partition " << msg->partition << " offset " << msg->offset);
|
|
|
|
rd_kafka_message_destroy(msg);
|
|
|
|
return nextImpl();
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
2018-07-18 05:22:01 +00:00
|
|
|
++read_messages;
|
|
|
|
|
|
|
|
// Now we've received a new message. Check if we need to produce a delimiter
|
|
|
|
if (row_delimiter != '\0' && current != nullptr)
|
|
|
|
{
|
|
|
|
BufferBase::set(&row_delimiter, 1, 0);
|
|
|
|
reset();
|
|
|
|
current = msg;
|
|
|
|
current_pending = true;
|
|
|
|
return true;
|
|
|
|
}
|
2017-10-03 23:39:24 +00:00
|
|
|
|
|
|
|
// Consume message and mark the topic/partition offset
|
2018-07-18 05:22:01 +00:00
|
|
|
// The offsets will be committed in the readSuffix() method after the block is completed
|
|
|
|
// If an exception is thrown before that would occur, the client will rejoin without committing offsets
|
|
|
|
reset();
|
2017-12-16 01:45:11 +00:00
|
|
|
current = msg;
|
2018-07-18 05:22:01 +00:00
|
|
|
BufferBase::set(reinterpret_cast<char *>(current->payload), current->len, 0);
|
2017-10-03 23:39:24 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
void reset()
|
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
if (current != nullptr)
|
|
|
|
{
|
|
|
|
rd_kafka_message_destroy(current);
|
|
|
|
current = nullptr;
|
|
|
|
}
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
public:
|
2018-07-18 05:22:01 +00:00
|
|
|
ReadBufferFromKafkaConsumer(rd_kafka_t * consumer_, Poco::Logger * log_, char row_delimiter_)
|
|
|
|
: ReadBuffer(nullptr, 0), consumer(consumer_), current(nullptr),
|
2018-07-27 21:33:30 +00:00
|
|
|
current_pending(false), log(log_), read_messages(0), row_delimiter(row_delimiter_)
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Row delimiter is: " << row_delimiter);
|
2018-07-18 05:22:01 +00:00
|
|
|
}
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2018-08-10 04:02:56 +00:00
|
|
|
~ReadBufferFromKafkaConsumer() override { reset(); }
|
2017-12-23 00:32:48 +00:00
|
|
|
|
|
|
|
/// Commit messages read with this consumer
|
2018-01-06 18:10:44 +00:00
|
|
|
void commit()
|
|
|
|
{
|
2017-12-23 00:32:48 +00:00
|
|
|
LOG_TRACE(log, "Committing " << read_messages << " messages");
|
|
|
|
if (read_messages == 0)
|
|
|
|
return;
|
|
|
|
|
2018-08-10 04:02:56 +00:00
|
|
|
auto err = rd_kafka_commit(consumer, nullptr, 1 /* async */);
|
2017-12-23 00:32:48 +00:00
|
|
|
if (err)
|
|
|
|
throw Exception("Failed to commit offsets: " + String(rd_kafka_err2str(err)), ErrorCodes::UNKNOWN_EXCEPTION);
|
|
|
|
|
|
|
|
read_messages = 0;
|
|
|
|
}
|
2017-10-03 23:39:24 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
class KafkaBlockInputStream : public IProfilingBlockInputStream
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
KafkaBlockInputStream(StorageKafka & storage_, StorageKafka::ConsumerPtr consumer_, const Context & context_, const String & schema, size_t max_block_size)
|
|
|
|
: storage(storage_), consumer(consumer_)
|
2017-10-03 23:39:24 +00:00
|
|
|
{
|
|
|
|
// Always skip unknown fields regardless of the context (JSON or TSKV)
|
|
|
|
Context context = context_;
|
2017-10-14 06:39:48 +00:00
|
|
|
context.setSetting("input_format_skip_unknown_fields", UInt64(1));
|
2017-10-03 23:39:24 +00:00
|
|
|
if (schema.size() > 0)
|
2017-10-19 07:33:24 +00:00
|
|
|
context.setSetting("format_schema", schema);
|
2017-12-16 01:45:11 +00:00
|
|
|
|
2017-10-03 23:39:24 +00:00
|
|
|
// Create a formatted reader on Kafka messages
|
|
|
|
LOG_TRACE(storage.log, "Creating formatted reader");
|
2018-07-18 05:22:01 +00:00
|
|
|
read_buf = std::make_unique<ReadBufferFromKafkaConsumer>(consumer->stream, storage.log, storage.row_delimiter);
|
2018-06-10 19:22:49 +00:00
|
|
|
reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size);
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
~KafkaBlockInputStream() override
|
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
// An error was thrown during the stream or it did not finish successfully
|
|
|
|
// The read offsets weren't comitted, so consumer must rejoin the group from the original starting point
|
|
|
|
if (!finalized)
|
|
|
|
{
|
|
|
|
LOG_TRACE(storage.log, "KafkaBlockInputStream did not finish successfully, unsubscribing from assignments and rejoining");
|
|
|
|
consumer->unsubscribe();
|
|
|
|
consumer->subscribe(storage.topics);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Return consumer for another reader
|
|
|
|
storage.pushConsumer(consumer);
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
return storage.getName();
|
|
|
|
}
|
|
|
|
|
|
|
|
Block readImpl() override
|
|
|
|
{
|
2018-03-05 21:09:39 +00:00
|
|
|
if (isCancelledOrThrowIfKilled())
|
2017-10-03 23:39:24 +00:00
|
|
|
return {};
|
|
|
|
|
|
|
|
return reader->read();
|
|
|
|
}
|
|
|
|
|
2018-08-10 04:02:56 +00:00
|
|
|
Block getHeader() const override { return reader->getHeader(); }
|
2018-01-06 18:10:44 +00:00
|
|
|
|
2017-10-03 23:39:24 +00:00
|
|
|
void readPrefixImpl() override
|
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
// Start reading data
|
|
|
|
finalized = false;
|
2017-10-03 23:39:24 +00:00
|
|
|
reader->readPrefix();
|
|
|
|
}
|
|
|
|
|
|
|
|
void readSuffixImpl() override
|
|
|
|
{
|
|
|
|
reader->readSuffix();
|
2017-12-16 01:45:11 +00:00
|
|
|
|
2017-12-23 00:32:48 +00:00
|
|
|
// Store offsets read in this stream
|
|
|
|
read_buf->commit();
|
2017-12-16 01:45:11 +00:00
|
|
|
|
|
|
|
// Mark as successfully finished
|
|
|
|
finalized = true;
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
StorageKafka & storage;
|
2017-12-16 01:45:11 +00:00
|
|
|
StorageKafka::ConsumerPtr consumer;
|
2017-10-03 23:39:24 +00:00
|
|
|
Block sample_block;
|
|
|
|
std::unique_ptr<ReadBufferFromKafkaConsumer> read_buf;
|
|
|
|
BlockInputStreamPtr reader;
|
2017-12-16 01:45:11 +00:00
|
|
|
bool finalized = false;
|
2017-10-03 23:39:24 +00:00
|
|
|
};
|
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
static void loadFromConfig(struct rd_kafka_conf_s * conf, const AbstractConfiguration & config, const std::string & path)
|
|
|
|
{
|
|
|
|
AbstractConfiguration::Keys keys;
|
|
|
|
std::vector<char> errstr(512);
|
|
|
|
|
|
|
|
config.keys(path, keys);
|
|
|
|
|
|
|
|
for (const auto & key : keys)
|
|
|
|
{
|
|
|
|
const String key_path = path + "." + key;
|
|
|
|
const String key_name = boost::replace_all_copy(key, "_", ".");
|
|
|
|
if (rd_kafka_conf_set(conf, key_name.c_str(), config.getString(key_path).c_str(), errstr.data(), errstr.size()) != RD_KAFKA_CONF_OK)
|
|
|
|
throw Exception("Invalid Kafka setting " + key_path + " in config: " + String(errstr.data()), ErrorCodes::INVALID_CONFIG_PARAMETER);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-10-03 23:39:24 +00:00
|
|
|
StorageKafka::StorageKafka(
|
|
|
|
const std::string & table_name_,
|
|
|
|
const std::string & database_name_,
|
|
|
|
Context & context_,
|
2018-03-06 20:18:34 +00:00
|
|
|
const ColumnsDescription & columns_,
|
2017-11-17 20:42:03 +00:00
|
|
|
const String & brokers_, const String & group_, const Names & topics_,
|
2018-07-18 05:22:01 +00:00
|
|
|
const String & format_name_, char row_delimiter_, const String & schema_name_, size_t num_consumers_)
|
2018-03-06 20:18:34 +00:00
|
|
|
: IStorage{columns_},
|
2017-10-03 23:39:24 +00:00
|
|
|
table_name(table_name_), database_name(database_name_), context(context_),
|
2018-04-20 13:48:42 +00:00
|
|
|
topics(context.getMacros()->expand(topics_)),
|
|
|
|
brokers(context.getMacros()->expand(brokers_)),
|
|
|
|
group(context.getMacros()->expand(group_)),
|
|
|
|
format_name(context.getMacros()->expand(format_name_)),
|
2018-07-18 05:22:01 +00:00
|
|
|
row_delimiter(row_delimiter_),
|
2018-04-20 13:48:42 +00:00
|
|
|
schema_name(context.getMacros()->expand(schema_name_)),
|
2017-12-16 01:45:11 +00:00
|
|
|
num_consumers(num_consumers_), log(&Logger::get("StorageKafka (" + table_name_ + ")")),
|
|
|
|
semaphore(0, num_consumers_), mutex(), consumers(), event_update()
|
2017-10-03 23:39:24 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
BlockInputStreams StorageKafka::read(
|
|
|
|
const Names & column_names,
|
2017-12-01 21:13:25 +00:00
|
|
|
const SelectQueryInfo & /*query_info*/,
|
2017-10-03 23:39:24 +00:00
|
|
|
const Context & context,
|
2018-04-19 14:47:09 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
2017-10-03 23:39:24 +00:00
|
|
|
size_t max_block_size,
|
|
|
|
unsigned num_streams)
|
|
|
|
{
|
2018-04-19 14:47:09 +00:00
|
|
|
checkQueryProcessingStage(processed_stage, context);
|
2017-10-03 23:39:24 +00:00
|
|
|
check(column_names);
|
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
if (num_consumers == 0)
|
2017-11-17 20:42:03 +00:00
|
|
|
return BlockInputStreams();
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2017-12-20 19:09:13 +00:00
|
|
|
const size_t stream_count = std::min(num_streams, num_consumers);
|
|
|
|
|
2017-10-03 23:39:24 +00:00
|
|
|
BlockInputStreams streams;
|
2017-12-20 19:09:13 +00:00
|
|
|
streams.reserve(stream_count);
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
// Claim as many consumers as requested, but don't block
|
2017-12-20 19:09:13 +00:00
|
|
|
for (size_t i = 0; i < stream_count; ++i)
|
2017-12-16 01:45:11 +00:00
|
|
|
{
|
2017-12-20 19:09:13 +00:00
|
|
|
auto consumer = tryClaimConsumer(0);
|
2017-12-16 01:45:11 +00:00
|
|
|
if (consumer == nullptr)
|
|
|
|
break;
|
|
|
|
|
2017-12-23 00:48:24 +00:00
|
|
|
// Use block size of 1, otherwise LIMIT won't work properly as it will buffer excess messages in the last block
|
2017-12-16 01:45:11 +00:00
|
|
|
streams.push_back(std::make_shared<KafkaBlockInputStream>(*this, consumer, context, schema_name, 1));
|
|
|
|
}
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
LOG_DEBUG(log, "Starting reading " << streams.size() << " streams, " << max_block_size << " block size");
|
2017-10-03 23:39:24 +00:00
|
|
|
return streams;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageKafka::startup()
|
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
for (size_t i = 0; i < num_consumers; ++i)
|
2017-10-03 23:39:24 +00:00
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
// Building configuration may throw, the consumer configuration must be destroyed in that case
|
|
|
|
auto consumer_conf = rd_kafka_conf_new();
|
|
|
|
try
|
|
|
|
{
|
|
|
|
consumerConfiguration(consumer_conf);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
rd_kafka_conf_destroy(consumer_conf);
|
|
|
|
throw;
|
|
|
|
}
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
// Create a consumer and subscribe to topics
|
|
|
|
// Note: consumer takes ownership of the configuration
|
|
|
|
auto consumer = std::make_shared<StorageKafka::Consumer>(consumer_conf);
|
|
|
|
consumer->subscribe(topics);
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
// Make consumer available
|
|
|
|
pushConsumer(consumer);
|
2018-03-19 17:52:48 +00:00
|
|
|
++num_created_consumers;
|
2017-12-16 01:45:11 +00:00
|
|
|
}
|
2017-10-03 23:39:24 +00:00
|
|
|
|
|
|
|
// Start the reader thread
|
|
|
|
stream_thread = std::thread(&StorageKafka::streamThread, this);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageKafka::shutdown()
|
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
// Interrupt streaming thread
|
|
|
|
stream_cancelled = true;
|
|
|
|
event_update.set();
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
// Unsubscribe from assignments
|
2017-10-03 23:39:24 +00:00
|
|
|
LOG_TRACE(log, "Unsubscribing from assignments");
|
2018-03-19 17:52:48 +00:00
|
|
|
for (size_t i = 0; i < num_created_consumers; ++i)
|
2017-10-03 23:39:24 +00:00
|
|
|
{
|
2017-12-20 19:09:13 +00:00
|
|
|
auto consumer = claimConsumer();
|
2017-12-16 01:45:11 +00:00
|
|
|
consumer->unsubscribe();
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
// Wait for stream thread to finish
|
2017-10-03 23:39:24 +00:00
|
|
|
if (stream_thread.joinable())
|
|
|
|
stream_thread.join();
|
|
|
|
|
|
|
|
rd_kafka_wait_destroyed(CLEANUP_TIMEOUT_MS);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageKafka::updateDependencies()
|
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
event_update.set();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageKafka::consumerConfiguration(struct rd_kafka_conf_s * conf)
|
|
|
|
{
|
|
|
|
std::vector<char> errstr(512);
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Setting brokers: " << brokers);
|
|
|
|
if (rd_kafka_conf_set(conf, "metadata.broker.list", brokers.c_str(), errstr.data(), errstr.size()) != RD_KAFKA_CONF_OK)
|
|
|
|
throw Exception(String(errstr.data()), ErrorCodes::INCORRECT_DATA);
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Setting Group ID: " << group << " Client ID: clickhouse");
|
|
|
|
|
|
|
|
if (rd_kafka_conf_set(conf, "group.id", group.c_str(), errstr.data(), errstr.size()) != RD_KAFKA_CONF_OK)
|
|
|
|
throw Exception(String(errstr.data()), ErrorCodes::INCORRECT_DATA);
|
|
|
|
|
|
|
|
if (rd_kafka_conf_set(conf, "client.id", VERSION_FULL, errstr.data(), errstr.size()) != RD_KAFKA_CONF_OK)
|
|
|
|
throw Exception(String(errstr.data()), ErrorCodes::INCORRECT_DATA);
|
|
|
|
|
|
|
|
// We manually commit offsets after a stream successfully finished
|
|
|
|
rd_kafka_conf_set(conf, "enable.auto.commit", "false", nullptr, 0);
|
|
|
|
|
|
|
|
// Update consumer configuration from the configuration
|
|
|
|
const auto & config = context.getConfigRef();
|
|
|
|
if (config.has(CONFIG_PREFIX))
|
|
|
|
loadFromConfig(conf, config, CONFIG_PREFIX);
|
|
|
|
|
|
|
|
// Update consumer topic-specific configuration
|
|
|
|
for (const auto & topic : topics)
|
|
|
|
{
|
|
|
|
const auto topic_config_key = CONFIG_PREFIX + "_" + topic;
|
|
|
|
if (config.has(topic_config_key))
|
|
|
|
loadFromConfig(conf, config, topic_config_key);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-12-20 19:09:13 +00:00
|
|
|
StorageKafka::ConsumerPtr StorageKafka::claimConsumer()
|
|
|
|
{
|
|
|
|
return tryClaimConsumer(-1L);
|
|
|
|
}
|
2017-12-16 01:45:11 +00:00
|
|
|
|
2017-12-20 19:09:13 +00:00
|
|
|
StorageKafka::ConsumerPtr StorageKafka::tryClaimConsumer(long wait_ms)
|
2017-12-16 01:45:11 +00:00
|
|
|
{
|
|
|
|
// Wait for the first free consumer
|
|
|
|
if (wait_ms >= 0)
|
|
|
|
{
|
|
|
|
if (!semaphore.tryWait(wait_ms))
|
|
|
|
return nullptr;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
semaphore.wait();
|
|
|
|
|
|
|
|
// Take the first available consumer from the list
|
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
|
|
|
auto consumer = consumers.back();
|
|
|
|
consumers.pop_back();
|
|
|
|
return consumer;
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
void StorageKafka::pushConsumer(StorageKafka::ConsumerPtr c)
|
|
|
|
{
|
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
|
|
|
consumers.push_back(c);
|
|
|
|
semaphore.set();
|
|
|
|
}
|
2017-10-03 23:39:24 +00:00
|
|
|
|
|
|
|
void StorageKafka::streamThread()
|
|
|
|
{
|
2017-12-30 00:36:06 +00:00
|
|
|
setThreadName("KafkaStreamThr");
|
2018-09-06 02:47:59 +00:00
|
|
|
CurrentThread::initializeQuery();
|
2017-10-03 23:39:24 +00:00
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
while (!stream_cancelled)
|
2017-10-03 23:39:24 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
// Keep streaming as long as there are attached views and streaming is not cancelled
|
|
|
|
while (!stream_cancelled)
|
2017-10-03 23:39:24 +00:00
|
|
|
{
|
2017-12-16 01:45:11 +00:00
|
|
|
// Check if all dependencies are attached
|
|
|
|
auto dependencies = context.getDependencies(database_name, table_name);
|
|
|
|
if (dependencies.size() == 0)
|
|
|
|
break;
|
2018-05-30 15:11:53 +00:00
|
|
|
// Check the dependencies are ready?
|
|
|
|
bool ready = true;
|
|
|
|
for (const auto & db_tab : dependencies)
|
|
|
|
{
|
|
|
|
if (!context.tryGetTable(db_tab.first, db_tab.second))
|
|
|
|
ready = false;
|
|
|
|
}
|
|
|
|
if (!ready)
|
|
|
|
break;
|
2017-12-16 01:45:11 +00:00
|
|
|
|
2017-10-03 23:39:24 +00:00
|
|
|
LOG_DEBUG(log, "Started streaming to " << dependencies.size() << " attached views");
|
|
|
|
streamToViews();
|
|
|
|
LOG_DEBUG(log, "Stopped streaming to views");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
|
2017-12-16 01:45:11 +00:00
|
|
|
// Wait for attached views
|
|
|
|
event_update.tryWait(READ_POLL_MS);
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Stream thread finished");
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageKafka::streamToViews()
|
|
|
|
{
|
|
|
|
auto table = context.getTable(database_name, table_name);
|
|
|
|
if (!table)
|
2017-12-16 01:45:11 +00:00
|
|
|
throw Exception("Engine table " + database_name + "." + table_name + " doesn't exist.", ErrorCodes::LOGICAL_ERROR);
|
2017-10-03 23:39:24 +00:00
|
|
|
|
|
|
|
// Create an INSERT query for streaming data
|
|
|
|
auto insert = std::make_shared<ASTInsertQuery>();
|
|
|
|
insert->database = database_name;
|
|
|
|
insert->table = table_name;
|
|
|
|
insert->no_destination = true; // Only insert into dependent views
|
|
|
|
|
|
|
|
// Limit the number of batched messages to allow early cancellations
|
|
|
|
const Settings & settings = context.getSettingsRef();
|
2017-12-16 01:45:11 +00:00
|
|
|
const size_t block_size = settings.max_block_size.value;
|
|
|
|
|
|
|
|
// Create a stream for each consumer and join them in a union stream
|
|
|
|
BlockInputStreams streams;
|
|
|
|
streams.reserve(num_consumers);
|
|
|
|
for (size_t i = 0; i < num_consumers; ++i)
|
|
|
|
{
|
2017-12-20 19:09:13 +00:00
|
|
|
auto consumer = claimConsumer();
|
2017-12-23 00:48:24 +00:00
|
|
|
auto stream = std::make_shared<KafkaBlockInputStream>(*this, consumer, context, schema_name, block_size);
|
|
|
|
streams.push_back(stream);
|
|
|
|
|
|
|
|
// Limit read batch to maximum block size to allow DDL
|
|
|
|
IProfilingBlockInputStream::LocalLimits limits;
|
|
|
|
limits.max_execution_time = settings.stream_flush_interval_ms;
|
|
|
|
limits.timeout_overflow_mode = OverflowMode::BREAK;
|
|
|
|
if (IProfilingBlockInputStream * p_stream = dynamic_cast<IProfilingBlockInputStream *>(stream.get()))
|
|
|
|
p_stream->setLimits(limits);
|
2017-12-16 01:45:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
auto in = std::make_shared<UnionBlockInputStream<>>(streams, nullptr, num_consumers);
|
2017-10-03 23:39:24 +00:00
|
|
|
|
|
|
|
// Execute the query
|
|
|
|
InterpreterInsertQuery interpreter{insert, context};
|
|
|
|
auto block_io = interpreter.execute();
|
2017-12-16 01:45:11 +00:00
|
|
|
copyData(*in, *block_io.out, &stream_cancelled);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
StorageKafka::Consumer::Consumer(struct rd_kafka_conf_s * conf)
|
|
|
|
{
|
|
|
|
std::vector<char> errstr(512);
|
|
|
|
stream = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr.data(), errstr.size());
|
|
|
|
if (stream == nullptr)
|
|
|
|
{
|
|
|
|
rd_kafka_conf_destroy(conf);
|
|
|
|
throw Exception("Failed to create consumer handle: " + String(errstr.data()), ErrorCodes::UNKNOWN_EXCEPTION);
|
|
|
|
}
|
|
|
|
|
|
|
|
rd_kafka_poll_set_consumer(stream);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
StorageKafka::Consumer::~Consumer()
|
|
|
|
{
|
|
|
|
if (stream != nullptr)
|
|
|
|
{
|
|
|
|
rd_kafka_consumer_close(stream);
|
|
|
|
rd_kafka_destroy(stream);
|
|
|
|
stream = nullptr;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageKafka::Consumer::subscribe(const Names & topics)
|
|
|
|
{
|
|
|
|
if (stream == nullptr)
|
|
|
|
throw Exception("Cannot subscribe to topics when consumer is closed", ErrorCodes::UNKNOWN_EXCEPTION);
|
|
|
|
|
|
|
|
// Create a list of partitions
|
|
|
|
auto * topicList = rd_kafka_topic_partition_list_new(topics.size());
|
|
|
|
for (const auto & t : topics)
|
|
|
|
{
|
|
|
|
rd_kafka_topic_partition_list_add(topicList, t.c_str(), RD_KAFKA_PARTITION_UA);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Subscribe to requested topics
|
|
|
|
auto err = rd_kafka_subscribe(stream, topicList);
|
|
|
|
if (err)
|
|
|
|
{
|
|
|
|
rd_kafka_topic_partition_list_destroy(topicList);
|
|
|
|
throw Exception("Failed to subscribe: " + String(rd_kafka_err2str(err)), ErrorCodes::UNKNOWN_EXCEPTION);
|
|
|
|
}
|
|
|
|
|
|
|
|
rd_kafka_topic_partition_list_destroy(topicList);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageKafka::Consumer::unsubscribe()
|
|
|
|
{
|
|
|
|
if (stream != nullptr)
|
|
|
|
rd_kafka_unsubscribe(stream);
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
void registerStorageKafka(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("Kafka", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
ASTs & engine_args = args.engine_args;
|
2018-08-01 17:23:50 +00:00
|
|
|
size_t args_count = engine_args.size();
|
|
|
|
bool has_settings = args.storage_def->settings;
|
|
|
|
|
|
|
|
KafkaSettings kafka_settings;
|
|
|
|
if (has_settings)
|
|
|
|
{
|
|
|
|
kafka_settings.loadFromQuery(*args.storage_def);
|
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
|
|
|
|
/** Arguments of engine is following:
|
|
|
|
* - Kafka broker list
|
|
|
|
* - List of topics
|
|
|
|
* - Group ID (may be a constaint expression with a string result)
|
|
|
|
* - Message format (string)
|
2018-08-01 17:23:50 +00:00
|
|
|
* - Row delimiter
|
2017-12-30 00:36:06 +00:00
|
|
|
* - Schema (optional, if the format supports it)
|
2018-08-01 17:23:50 +00:00
|
|
|
* - Number of consumers
|
2017-12-30 00:36:06 +00:00
|
|
|
*/
|
|
|
|
|
2018-08-01 17:23:50 +00:00
|
|
|
// Check arguments and settings
|
|
|
|
#define CHECK_KAFKA_STORAGE_ARGUMENT(ARG_NUM, PAR_NAME) \
|
|
|
|
/* One of the four required arguments is not specified */ \
|
|
|
|
if (args_count < ARG_NUM && ARG_NUM <= 4 && \
|
|
|
|
!kafka_settings.PAR_NAME.changed) \
|
|
|
|
{ \
|
|
|
|
throw Exception( \
|
|
|
|
"Required parameter '" #PAR_NAME "' " \
|
|
|
|
"for storage Kafka not specified", \
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); \
|
|
|
|
} \
|
|
|
|
/* The same argument is given in two places */ \
|
|
|
|
if (has_settings && \
|
|
|
|
kafka_settings.PAR_NAME.changed && \
|
|
|
|
args_count >= ARG_NUM) \
|
|
|
|
{ \
|
|
|
|
throw Exception( \
|
|
|
|
"The argument №" #ARG_NUM " of storage Kafka " \
|
|
|
|
"and the parameter '" #PAR_NAME "' " \
|
|
|
|
"in SETTINGS cannot be specified at the same time", \
|
|
|
|
ErrorCodes::BAD_ARGUMENTS); \
|
|
|
|
}
|
|
|
|
|
|
|
|
CHECK_KAFKA_STORAGE_ARGUMENT(1, kafka_broker_list)
|
|
|
|
CHECK_KAFKA_STORAGE_ARGUMENT(2, kafka_topic_list)
|
|
|
|
CHECK_KAFKA_STORAGE_ARGUMENT(3, kafka_group_name)
|
|
|
|
CHECK_KAFKA_STORAGE_ARGUMENT(4, kafka_format)
|
|
|
|
CHECK_KAFKA_STORAGE_ARGUMENT(5, kafka_row_delimiter)
|
|
|
|
CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema)
|
|
|
|
CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers)
|
|
|
|
#undef CHECK_KAFKA_STORAGE_ARGUMENT
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2018-08-01 17:23:50 +00:00
|
|
|
// Get and check broker list
|
2017-12-30 00:36:06 +00:00
|
|
|
String brokers;
|
2018-08-01 17:23:50 +00:00
|
|
|
if (args_count >= 1)
|
|
|
|
{
|
|
|
|
auto ast = typeid_cast<const ASTLiteral *>(engine_args[0].get());
|
|
|
|
if (ast && ast->value.getType() == Field::Types::String)
|
|
|
|
{
|
|
|
|
brokers = safeGet<String>(ast->value);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
throw Exception(String("Kafka broker list must be a string"), ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (kafka_settings.kafka_broker_list.changed)
|
|
|
|
{
|
|
|
|
brokers = kafka_settings.kafka_broker_list.value;
|
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2018-08-01 17:23:50 +00:00
|
|
|
// Get and check topic list
|
|
|
|
String topic_list;
|
|
|
|
if (args_count >= 2)
|
|
|
|
{
|
|
|
|
engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.local_context);
|
|
|
|
topic_list = static_cast<const ASTLiteral &>(*engine_args[1]).value.safeGet<String>();
|
|
|
|
}
|
|
|
|
else if (kafka_settings.kafka_topic_list.changed)
|
|
|
|
{
|
|
|
|
topic_list = kafka_settings.kafka_topic_list.value;
|
|
|
|
}
|
|
|
|
Names topics;
|
|
|
|
boost::split(topics, topic_list , [](char c){ return c == ','; });
|
|
|
|
for (String & topic : topics)
|
|
|
|
{
|
|
|
|
boost::trim(topic);
|
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2018-08-01 17:23:50 +00:00
|
|
|
// Get and check group name
|
|
|
|
String group;
|
|
|
|
if (args_count >= 3)
|
|
|
|
{
|
|
|
|
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context);
|
|
|
|
group = static_cast<const ASTLiteral &>(*engine_args[2]).value.safeGet<String>();
|
|
|
|
}
|
|
|
|
else if (kafka_settings.kafka_group_name.changed)
|
|
|
|
{
|
|
|
|
group = kafka_settings.kafka_group_name.value;
|
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2018-08-01 17:23:50 +00:00
|
|
|
// Get and check message format name
|
|
|
|
String format;
|
|
|
|
if (args_count >= 4)
|
|
|
|
{
|
|
|
|
engine_args[3] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[3], args.local_context);
|
|
|
|
|
|
|
|
auto ast = typeid_cast<const ASTLiteral *>(engine_args[3].get());
|
|
|
|
if (ast && ast->value.getType() == Field::Types::String)
|
|
|
|
{
|
|
|
|
format = safeGet<String>(ast->value);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
throw Exception("Format must be a string", ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (kafka_settings.kafka_format.changed)
|
|
|
|
{
|
|
|
|
format = kafka_settings.kafka_format.value;
|
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2018-07-18 05:22:01 +00:00
|
|
|
// Parse row delimiter (optional)
|
|
|
|
char row_delimiter = '\0';
|
2018-08-01 17:23:50 +00:00
|
|
|
if (args_count >= 5)
|
2017-12-30 00:36:06 +00:00
|
|
|
{
|
|
|
|
engine_args[4] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[4], args.local_context);
|
|
|
|
|
|
|
|
auto ast = typeid_cast<const ASTLiteral *>(engine_args[4].get());
|
2018-07-18 05:22:01 +00:00
|
|
|
String arg;
|
2017-12-30 00:36:06 +00:00
|
|
|
if (ast && ast->value.getType() == Field::Types::String)
|
2018-08-01 17:23:50 +00:00
|
|
|
{
|
2018-07-18 05:22:01 +00:00
|
|
|
arg = safeGet<String>(ast->value);
|
2018-08-01 17:23:50 +00:00
|
|
|
}
|
2018-07-18 05:22:01 +00:00
|
|
|
else
|
2018-08-01 17:23:50 +00:00
|
|
|
{
|
2018-07-18 05:22:01 +00:00
|
|
|
throw Exception("Row delimiter must be a char", ErrorCodes::BAD_ARGUMENTS);
|
2018-08-01 17:23:50 +00:00
|
|
|
}
|
2018-07-18 05:22:01 +00:00
|
|
|
if (arg.size() > 1)
|
2018-08-01 17:23:50 +00:00
|
|
|
{
|
2018-07-18 05:22:01 +00:00
|
|
|
throw Exception("Row delimiter must be a char", ErrorCodes::BAD_ARGUMENTS);
|
2018-08-01 17:23:50 +00:00
|
|
|
}
|
2018-07-18 05:22:01 +00:00
|
|
|
else if (arg.size() == 0)
|
2018-08-01 17:23:50 +00:00
|
|
|
{
|
2018-07-18 05:22:01 +00:00
|
|
|
row_delimiter = '\0';
|
2018-08-01 17:23:50 +00:00
|
|
|
}
|
2018-07-18 05:22:01 +00:00
|
|
|
else
|
2018-08-01 17:23:50 +00:00
|
|
|
{
|
2018-07-18 05:22:01 +00:00
|
|
|
row_delimiter = arg[0];
|
2018-08-01 17:23:50 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (kafka_settings.kafka_row_delimiter.changed)
|
|
|
|
{
|
|
|
|
row_delimiter = kafka_settings.kafka_row_delimiter.value;
|
2018-07-18 05:22:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Parse format schema if supported (optional)
|
|
|
|
String schema;
|
2018-08-01 17:23:50 +00:00
|
|
|
if (args_count >= 6)
|
2018-07-18 05:22:01 +00:00
|
|
|
{
|
|
|
|
engine_args[5] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[4], args.local_context);
|
|
|
|
|
|
|
|
auto ast = typeid_cast<const ASTLiteral *>(engine_args[5].get());
|
2017-12-30 00:36:06 +00:00
|
|
|
if (ast && ast->value.getType() == Field::Types::String)
|
2018-08-01 17:23:50 +00:00
|
|
|
{
|
2017-12-30 00:36:06 +00:00
|
|
|
schema = safeGet<String>(ast->value);
|
2018-08-01 17:23:50 +00:00
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
else
|
2018-08-01 17:23:50 +00:00
|
|
|
{
|
2017-12-30 00:36:06 +00:00
|
|
|
throw Exception("Format schema must be a string", ErrorCodes::BAD_ARGUMENTS);
|
2018-08-01 17:23:50 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (kafka_settings.kafka_schema.changed)
|
|
|
|
{
|
|
|
|
schema = kafka_settings.kafka_schema.value;
|
2017-12-30 00:36:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Parse number of consumers (optional)
|
|
|
|
UInt64 num_consumers = 1;
|
2018-08-01 17:23:50 +00:00
|
|
|
if (args_count >= 7)
|
2017-12-30 00:36:06 +00:00
|
|
|
{
|
2018-07-18 05:22:01 +00:00
|
|
|
auto ast = typeid_cast<const ASTLiteral *>(engine_args[6].get());
|
2017-12-30 00:36:06 +00:00
|
|
|
if (ast && ast->value.getType() == Field::Types::UInt64)
|
2018-08-01 17:23:50 +00:00
|
|
|
{
|
2017-12-30 00:36:06 +00:00
|
|
|
num_consumers = safeGet<UInt64>(ast->value);
|
2018-08-01 17:23:50 +00:00
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
else
|
2018-08-01 17:23:50 +00:00
|
|
|
{
|
2017-12-30 00:36:06 +00:00
|
|
|
throw Exception("Number of consumers must be a positive integer", ErrorCodes::BAD_ARGUMENTS);
|
2018-08-01 17:23:50 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (kafka_settings.kafka_num_consumers.changed)
|
|
|
|
{
|
|
|
|
num_consumers = kafka_settings.kafka_num_consumers.value;
|
2017-12-30 00:36:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return StorageKafka::create(
|
|
|
|
args.table_name, args.database_name, args.context, args.columns,
|
2018-07-18 05:22:01 +00:00
|
|
|
brokers, group, topics, format, row_delimiter, schema, num_consumers);
|
2017-12-30 00:36:06 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-10-03 23:39:24 +00:00
|
|
|
}
|
2017-10-14 00:53:01 +00:00
|
|
|
|
|
|
|
#endif
|