ClickHouse/src/Storages/tests/gtest_storage_log.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

168 lines
4.7 KiB
C++
Raw Normal View History

2020-01-28 16:08:30 +00:00
#include <gtest/gtest.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
2020-01-28 16:09:24 +00:00
#include <Disks/tests/gtest_disk.h>
2020-01-28 16:08:30 +00:00
#include <Formats/FormatFactory.h>
#include <IO/ReadHelpers.h>
#include <Storages/StorageLog.h>
#include <Storages/SelectQueryInfo.h>
2020-01-28 16:08:30 +00:00
#include <Common/typeid_cast.h>
2020-02-07 09:51:48 +00:00
#include <Common/tests/gtest_global_context.h>
2020-11-04 10:14:23 +00:00
#include <Common/tests/gtest_global_register.h>
2020-01-28 16:08:30 +00:00
#include <memory>
2021-09-10 10:18:04 +00:00
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/Executors/PushingPipelineExecutor.h>
2021-10-11 16:11:50 +00:00
#include <Processors/Executors/CompletedPipelineExecutor.h>
2021-09-10 10:18:04 +00:00
#include <Processors/Sinks/SinkToStorage.h>
2021-10-16 14:03:50 +00:00
#include <QueryPipeline/QueryPipeline.h>
2022-05-23 13:46:57 +00:00
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
2022-05-23 19:47:32 +00:00
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
2022-05-23 13:46:57 +00:00
#include <QueryPipeline/QueryPipelineBuilder.h>
2020-01-28 16:08:30 +00:00
DB::StoragePtr createStorage(DB::DiskPtr & disk)
{
using namespace DB;
NamesAndTypesList names_and_types;
names_and_types.emplace_back("a", std::make_shared<DataTypeUInt64>());
StoragePtr table = std::make_shared<StorageLog>(
"Log", disk, "table/", StorageID("test", "test"), ColumnsDescription{names_and_types},
2022-07-14 11:33:23 +00:00
ConstraintsDescription{}, String{}, false, getContext().context);
2020-01-28 16:08:30 +00:00
table->startup();
return table;
}
class StorageLogTest : public testing::Test
{
public:
2020-01-28 16:08:30 +00:00
void SetUp() override
{
2023-01-11 14:20:18 +00:00
disk = createDisk();
table = createStorage(disk);
2020-01-28 16:08:30 +00:00
}
void TearDown() override
{
2021-05-13 07:03:00 +00:00
table->flushAndShutdown();
2023-01-11 14:20:18 +00:00
destroyDisk(disk);
2020-01-28 16:08:30 +00:00
}
const DB::DiskPtr & getDisk() { return disk; }
DB::StoragePtr & getTable() { return table; }
2020-01-28 16:08:30 +00:00
private:
DB::DiskPtr disk;
DB::StoragePtr table;
2020-01-28 16:08:30 +00:00
};
// Returns data written to table in Values format.
std::string writeData(int rows, DB::StoragePtr & table, const DB::ContextPtr context)
2020-01-28 16:08:30 +00:00
{
using namespace DB;
auto metadata_snapshot = table->getInMemoryMetadataPtr();
2020-01-28 16:08:30 +00:00
std::string data;
2020-01-28 16:08:30 +00:00
Block block;
{
const auto & storage_columns = metadata_snapshot->getColumns();
2020-01-28 16:08:30 +00:00
ColumnWithTypeAndName column;
column.name = "a";
2020-04-27 15:38:35 +00:00
column.type = storage_columns.getPhysical("a").type;
2020-01-28 16:08:30 +00:00
auto col = column.type->createColumn();
ColumnUInt64::Container & vec = typeid_cast<ColumnUInt64 &>(*col).getData();
vec.resize(rows);
for (size_t i = 0; i < rows; ++i)
{
2020-01-28 16:08:30 +00:00
vec[i] = i;
if (i > 0)
data += ",";
data += "(" + std::to_string(i) + ")";
}
2020-01-28 16:08:30 +00:00
column.column = std::move(col);
block.insert(column);
}
2021-09-17 12:05:54 +00:00
QueryPipeline pipeline(table->write({}, metadata_snapshot, context));
2021-09-10 10:18:04 +00:00
2021-09-17 12:05:54 +00:00
PushingPipelineExecutor executor(pipeline);
2021-09-10 10:18:04 +00:00
executor.push(block);
executor.finish();
2020-01-28 16:08:30 +00:00
return data;
}
2020-01-28 16:08:30 +00:00
// Returns all table data in Values format.
std::string readData(DB::StoragePtr & table, const DB::ContextPtr context)
{
using namespace DB;
auto metadata_snapshot = table->getInMemoryMetadataPtr();
auto storage_snapshot = table->getStorageSnapshot(metadata_snapshot, context);
2020-01-28 16:08:30 +00:00
Names column_names;
column_names.push_back("a");
SelectQueryInfo query_info;
QueryProcessingStage::Enum stage = table->getQueryProcessingStage(
context, QueryProcessingStage::Complete, storage_snapshot, query_info);
2020-01-28 16:08:30 +00:00
2022-05-23 13:46:57 +00:00
QueryPlan plan;
table->read(plan, column_names, storage_snapshot, query_info, context, stage, 8192, 1);
2022-05-24 20:06:08 +00:00
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*plan.buildQueryPipeline(
2022-05-23 13:46:57 +00:00
QueryPlanOptimizationSettings::fromContext(context),
BuildQueryPipelineSettings::fromContext(context))));
Block sample;
2020-01-28 16:08:30 +00:00
{
ColumnWithTypeAndName col;
col.type = std::make_shared<DataTypeUInt64>();
2021-08-08 01:02:48 +00:00
col.name = "a";
sample.insert(std::move(col));
}
2020-01-28 16:08:30 +00:00
2020-11-04 10:14:23 +00:00
tryRegisterFormats();
2020-11-09 19:07:38 +00:00
WriteBufferFromOwnString out_buf;
2021-10-11 16:11:50 +00:00
auto output = FormatFactory::instance().getOutputFormat("Values", out_buf, sample, context);
pipeline.complete(output);
2020-01-28 16:08:30 +00:00
2021-09-10 10:18:04 +00:00
Block data;
2020-01-28 16:08:30 +00:00
2021-10-11 16:11:50 +00:00
CompletedPipelineExecutor executor(pipeline);
executor.execute();
// output->flush();
2021-10-11 16:11:50 +00:00
out_buf.finalize();
2020-11-09 19:07:38 +00:00
return out_buf.str();
}
2020-01-28 16:08:30 +00:00
2023-01-11 14:20:18 +00:00
TEST_F(StorageLogTest, testReadWrite)
{
using namespace DB;
2020-05-28 18:52:08 +00:00
const auto & context_holder = getContext();
2020-01-28 16:08:30 +00:00
std::string data;
2020-01-28 16:08:30 +00:00
// Write several chunks of data.
2020-04-20 09:42:56 +00:00
data += writeData(10, this->getTable(), context_holder.context);
data += ",";
2020-04-20 09:42:56 +00:00
data += writeData(20, this->getTable(), context_holder.context);
data += ",";
2020-04-20 09:42:56 +00:00
data += writeData(10, this->getTable(), context_holder.context);
2020-01-28 16:08:30 +00:00
2020-04-20 09:42:56 +00:00
ASSERT_EQ(data, readData(this->getTable(), context_holder.context));
2020-01-28 16:08:30 +00:00
}