mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
More logs
This commit is contained in:
parent
27d138818d
commit
9778f7c2f3
@ -25,6 +25,9 @@
|
||||
# include "DictionaryStructure.h"
|
||||
# include "RedisBlockInputStream.h"
|
||||
|
||||
# include "Poco/Logger.h"
|
||||
# include "common/logger_useful.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -56,6 +59,12 @@ namespace DB
|
||||
template <typename T>
|
||||
void insertNumber(IColumn & column, const Poco::Redis::RedisType::Ptr & value, const std::string & name)
|
||||
{
|
||||
LOG_ERROR(&Logger::get("Redis"), "Got value: " + value->toString() + "with type=" +
|
||||
", isInteger=" + DB::toString(value->isInteger()) +
|
||||
", isSimpleString=" + DB::toString(value->isSimpleString()) +
|
||||
", isBulkString=" + DB::toString(value->isBulkString()) +
|
||||
", isArray=" + DB::toString(value->isArray()) +
|
||||
", isError=" + DB::toString(value->isError()));
|
||||
switch (value->type())
|
||||
{
|
||||
case Poco::Redis::RedisTypeTraits<Poco::Int64>::TypeId:
|
||||
@ -68,7 +77,7 @@ namespace DB
|
||||
break;
|
||||
case Poco::Redis::RedisTypeTraits<Poco::Redis::BulkString>::TypeId:
|
||||
{
|
||||
const auto &bs =
|
||||
const auto & bs =
|
||||
static_cast<const Poco::Redis::Type<Poco::Redis::BulkString> *>(value.get())->value();
|
||||
if (bs.isNull())
|
||||
static_cast<ColumnVector<T> &>(column).getData().emplace_back();
|
||||
@ -78,7 +87,8 @@ namespace DB
|
||||
}
|
||||
default:
|
||||
throw Exception(
|
||||
"Type mismatch, expected a number, got type id = " + toString(value->type()) + " for column " + name,
|
||||
"Type mismatch, expected a number, got " + value->toString() +
|
||||
" with type id = " + toString(value->type()) + " for column " + name,
|
||||
ErrorCodes::TYPE_MISMATCH);
|
||||
}
|
||||
}
|
||||
@ -189,6 +199,9 @@ namespace DB
|
||||
if (all_read)
|
||||
return {};
|
||||
|
||||
for (size_t i = 0; i < 3; ++i)
|
||||
if (description.sample_block.columns() >= i + 1)
|
||||
LOG_ERROR(&Logger::get("Redis"), description.sample_block.getByPosition(i).dumpStructure());
|
||||
const size_t size = 2;
|
||||
if (size != description.sample_block.columns())
|
||||
throw Exception{"Unsupported number of columns for key-value storage: "
|
||||
@ -225,21 +238,27 @@ namespace DB
|
||||
break;
|
||||
}
|
||||
|
||||
|
||||
LOG_ERROR(&Logger::get("Redis"), "Get key: " + DB::toString(cursor));
|
||||
const auto & key = *(keys.begin() + cursor);
|
||||
insertValueByIdx(0, key);
|
||||
commandForValues.addRedisType(key);
|
||||
|
||||
LOG_ERROR(&Logger::get("Redis"), "Key has read: " + DB::toString(cursor));
|
||||
|
||||
++num_rows;
|
||||
++cursor;
|
||||
}
|
||||
|
||||
LOG_ERROR(&Logger::get("Redis"), "All " + DB::toString(num_rows) + " rows added");
|
||||
|
||||
if (num_rows == 0)
|
||||
return {};
|
||||
|
||||
LOG_ERROR(&Logger::get("Redis"), "Req to get values");
|
||||
Poco::Redis::Array values = client->execute<Poco::Redis::Array>(commandForValues);
|
||||
LOG_ERROR(&Logger::get("Redis"), "Req executed");
|
||||
for (size_t i = 0; i < num_rows; ++i)
|
||||
{
|
||||
LOG_ERROR(&Logger::get("Redis"), "Get value from : " + DB::toString(i));
|
||||
const Poco::Redis::RedisType::Ptr & value = *(values.begin() + i);
|
||||
if (value.isNull())
|
||||
insertDefaultValue(*columns[1], *description.sample_block.getByPosition(1).column);
|
||||
@ -249,7 +268,6 @@ namespace DB
|
||||
|
||||
return description.sample_block.cloneWithColumns(std::move(columns));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -52,6 +52,9 @@ namespace DB
|
||||
# include <ext/enumerate.h>
|
||||
# include "RedisBlockInputStream.h"
|
||||
|
||||
# include "Poco/Logger.h"
|
||||
# include "common/logger_useful.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -120,17 +123,25 @@ namespace DB
|
||||
|
||||
BlockInputStreamPtr RedisDictionarySource::loadAll()
|
||||
{
|
||||
LOG_ERROR(&Logger::get("Redis"), "Redis in loadAll");
|
||||
|
||||
Poco::Redis::Array commandForKeys;
|
||||
commandForKeys << "KEYS" << "*";
|
||||
LOG_ERROR(&Logger::get("Redis"), "Command for keys: " + commandForKeys.toString());
|
||||
|
||||
Poco::Redis::Array keys = client->execute<Poco::Redis::Array>(commandForKeys);
|
||||
|
||||
LOG_ERROR(&Logger::get("Redis"), "Command for keys executed");
|
||||
LOG_ERROR(&Logger::get("Redis"), "KEYS: " + keys.toString());
|
||||
|
||||
return std::make_shared<RedisBlockInputStream>(client, std::move(keys), sample_block, max_block_size);
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreamPtr RedisDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||
{
|
||||
LOG_ERROR(&Logger::get("Redis"), "Redis in loadIds");
|
||||
|
||||
if (!dict_struct.id)
|
||||
throw Exception{"'id' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
|
||||
@ -139,6 +150,8 @@ namespace DB
|
||||
for (UInt64 id : ids)
|
||||
keys << static_cast<Int64>(id);
|
||||
|
||||
LOG_ERROR(&Logger::get("Redis"), "KEYS: " + keys.toString());
|
||||
|
||||
return std::make_shared<RedisBlockInputStream>(client, std::move(keys), sample_block, max_block_size);
|
||||
}
|
||||
|
||||
|
@ -118,6 +118,9 @@ class DictionaryStructure(object):
|
||||
fields_strs = []
|
||||
for field in self.ordinary_fields:
|
||||
fields_strs.append(field.get_attribute_str())
|
||||
if self.is_kv:
|
||||
break
|
||||
|
||||
key_strs = []
|
||||
if self.layout.is_complex:
|
||||
for key_field in self.keys:
|
||||
|
@ -206,7 +206,7 @@ def test_complex_dictionaries(started_cluster):
|
||||
'my', 255.543, 3332221.44]),
|
||||
]
|
||||
|
||||
complex_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "complex"]
|
||||
complex_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "complex" and not d.is_kv]
|
||||
for dct in complex_dicts:
|
||||
dct.load_data(data)
|
||||
|
||||
@ -225,8 +225,6 @@ def test_complex_dictionaries(started_cluster):
|
||||
|
||||
for query in dct.get_select_get_or_default_queries(field, row):
|
||||
queries_with_answers.append((query, field.default_value_for_get))
|
||||
if dct.is_kv:
|
||||
break
|
||||
|
||||
for query, answer in queries_with_answers:
|
||||
print query
|
||||
@ -249,7 +247,7 @@ def test_ranged_dictionaries(started_cluster):
|
||||
32.543, 3332543.4]),
|
||||
]
|
||||
|
||||
ranged_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "ranged"]
|
||||
ranged_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "ranged" and not d.is_kv]
|
||||
for dct in ranged_dicts:
|
||||
dct.load_data(data)
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user