mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge pull request #41336 from CurtizJ/dynamic-columns-21
Fix writing of empty columns of type `Object`
This commit is contained in:
commit
6f7aaef5ee
@ -1,8 +1,11 @@
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Columns/ColumnObject.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <Common/randomSeed.h>
|
||||
@ -118,3 +121,36 @@ TEST(ColumnObject, InsertRangeFrom)
|
||||
checkFieldsAreEqual(subcolumn_dst, fields_dst);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(ColumnObject, Unflatten)
|
||||
{
|
||||
auto check_empty_tuple = [](const auto & type, const auto & column)
|
||||
{
|
||||
const auto & type_tuple = assert_cast<const DataTypeTuple &>(*type);
|
||||
const auto & column_tuple = assert_cast<const ColumnTuple &>(*column);
|
||||
|
||||
ASSERT_EQ(type_tuple.getElements().size(), 1);
|
||||
ASSERT_EQ(type_tuple.getElements()[0]->getName(), "UInt8");
|
||||
ASSERT_EQ(type_tuple.getElementNames()[0], ColumnObject::COLUMN_NAME_DUMMY);
|
||||
|
||||
ASSERT_EQ(column_tuple.getColumns().size(), 1);
|
||||
ASSERT_EQ(column_tuple.getColumns()[0]->getName(), "UInt8");
|
||||
};
|
||||
|
||||
{
|
||||
auto column_object = ColumnObject::create(false);
|
||||
auto [column, type] = unflattenObjectToTuple(*column_object);
|
||||
|
||||
check_empty_tuple(type, column);
|
||||
ASSERT_EQ(column->size(), 0);
|
||||
}
|
||||
|
||||
{
|
||||
auto column_object = ColumnObject::create(false);
|
||||
column_object->insertManyDefaults(5);
|
||||
auto [column, type] = unflattenObjectToTuple(*column_object);
|
||||
|
||||
check_empty_tuple(type, column);
|
||||
ASSERT_EQ(column->size(), 5);
|
||||
}
|
||||
}
|
||||
|
@ -453,15 +453,19 @@ using SubcolumnsTreeWithColumns = SubcolumnsTree<ColumnWithTypeAndDimensions>;
|
||||
using Node = SubcolumnsTreeWithColumns::Node;
|
||||
|
||||
/// Creates data type and column from tree of subcolumns.
|
||||
ColumnWithTypeAndDimensions createTypeFromNode(const Node * node)
|
||||
ColumnWithTypeAndDimensions createTypeFromNode(const Node & node)
|
||||
{
|
||||
auto collect_tuple_elemets = [](const auto & children)
|
||||
{
|
||||
if (children.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create type from empty Tuple or Nested node");
|
||||
|
||||
std::vector<std::tuple<String, ColumnWithTypeAndDimensions>> tuple_elements;
|
||||
tuple_elements.reserve(children.size());
|
||||
for (const auto & [name, child] : children)
|
||||
{
|
||||
auto column = createTypeFromNode(child.get());
|
||||
assert(child);
|
||||
auto column = createTypeFromNode(*child);
|
||||
tuple_elements.emplace_back(name, std::move(column));
|
||||
}
|
||||
|
||||
@ -475,13 +479,13 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node * node)
|
||||
return std::make_tuple(std::move(tuple_names), std::move(tuple_columns));
|
||||
};
|
||||
|
||||
if (node->kind == Node::SCALAR)
|
||||
if (node.kind == Node::SCALAR)
|
||||
{
|
||||
return node->data;
|
||||
return node.data;
|
||||
}
|
||||
else if (node->kind == Node::NESTED)
|
||||
else if (node.kind == Node::NESTED)
|
||||
{
|
||||
auto [tuple_names, tuple_columns] = collect_tuple_elemets(node->children);
|
||||
auto [tuple_names, tuple_columns] = collect_tuple_elemets(node.children);
|
||||
|
||||
Columns offsets_columns;
|
||||
offsets_columns.reserve(tuple_columns[0].array_dimensions + 1);
|
||||
@ -492,7 +496,7 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node * node)
|
||||
/// `k1 Array(Nested(k2 Int, k3 Int))` and k1 is marked as Nested
|
||||
/// and `k2` and `k3` has anonymous_array_level = 1 in that case.
|
||||
|
||||
const auto & current_array = assert_cast<const ColumnArray &>(*node->data.column);
|
||||
const auto & current_array = assert_cast<const ColumnArray &>(*node.data.column);
|
||||
offsets_columns.push_back(current_array.getOffsetsPtr());
|
||||
|
||||
auto first_column = tuple_columns[0].column;
|
||||
@ -529,7 +533,7 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node * node)
|
||||
}
|
||||
else
|
||||
{
|
||||
auto [tuple_names, tuple_columns] = collect_tuple_elemets(node->children);
|
||||
auto [tuple_names, tuple_columns] = collect_tuple_elemets(node.children);
|
||||
|
||||
size_t num_elements = tuple_columns.size();
|
||||
Columns tuple_elements_columns(num_elements);
|
||||
@ -587,6 +591,15 @@ std::pair<ColumnPtr, DataTypePtr> unflattenObjectToTuple(const ColumnObject & co
|
||||
{
|
||||
const auto & subcolumns = column.getSubcolumns();
|
||||
|
||||
if (subcolumns.empty())
|
||||
{
|
||||
auto type = std::make_shared<DataTypeTuple>(
|
||||
DataTypes{std::make_shared<DataTypeUInt8>()},
|
||||
Names{ColumnObject::COLUMN_NAME_DUMMY});
|
||||
|
||||
return {type->createColumn()->cloneResized(column.size()), type};
|
||||
}
|
||||
|
||||
PathsInData paths;
|
||||
DataTypes types;
|
||||
Columns columns;
|
||||
@ -613,6 +626,9 @@ std::pair<ColumnPtr, DataTypePtr> unflattenTuple(
|
||||
assert(paths.size() == tuple_types.size());
|
||||
assert(paths.size() == tuple_columns.size());
|
||||
|
||||
if (paths.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot unflatten empty Tuple");
|
||||
|
||||
/// We add all paths to the subcolumn tree and then create a type from it.
|
||||
/// The tree stores column, type and number of array dimensions
|
||||
/// for each intermediate node.
|
||||
|
@ -51,6 +51,8 @@ public:
|
||||
using NodeKind = typename Node::Kind;
|
||||
using NodePtr = std::shared_ptr<Node>;
|
||||
|
||||
SubcolumnsTree() : root(std::make_shared<Node>(Node::TUPLE)) {}
|
||||
|
||||
/// Add a leaf without any data in other nodes.
|
||||
bool add(const PathInData & path, const NodeData & leaf_data)
|
||||
{
|
||||
@ -73,13 +75,9 @@ public:
|
||||
bool add(const PathInData & path, const NodeCreator & node_creator)
|
||||
{
|
||||
const auto & parts = path.getParts();
|
||||
|
||||
if (parts.empty())
|
||||
return false;
|
||||
|
||||
if (!root)
|
||||
root = std::make_shared<Node>(Node::TUPLE);
|
||||
|
||||
Node * current_node = root.get();
|
||||
for (size_t i = 0; i < parts.size() - 1; ++i)
|
||||
{
|
||||
@ -166,13 +164,13 @@ public:
|
||||
return node;
|
||||
}
|
||||
|
||||
bool empty() const { return root == nullptr; }
|
||||
bool empty() const { return root->children.empty(); }
|
||||
size_t size() const { return leaves.size(); }
|
||||
|
||||
using Nodes = std::vector<NodePtr>;
|
||||
|
||||
const Nodes & getLeaves() const { return leaves; }
|
||||
const Node * getRoot() const { return root.get(); }
|
||||
const Node & getRoot() const { return *root; }
|
||||
|
||||
using iterator = typename Nodes::iterator;
|
||||
using const_iterator = typename Nodes::const_iterator;
|
||||
@ -186,11 +184,11 @@ public:
|
||||
private:
|
||||
const Node * findImpl(const PathInData & path, bool find_exact) const
|
||||
{
|
||||
if (!root)
|
||||
if (empty())
|
||||
return nullptr;
|
||||
|
||||
const auto & parts = path.getParts();
|
||||
const Node * current_node = root.get();
|
||||
const auto * current_node = root.get();
|
||||
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
|
@ -457,17 +457,20 @@ try
|
||||
|
||||
format->addBuffer(std::move(last_buffer));
|
||||
|
||||
auto chunk = Chunk(executor.getResultColumns(), total_rows);
|
||||
size_t total_bytes = chunk.bytes();
|
||||
if (total_rows)
|
||||
{
|
||||
auto chunk = Chunk(executor.getResultColumns(), total_rows);
|
||||
size_t total_bytes = chunk.bytes();
|
||||
|
||||
auto source = std::make_shared<SourceFromSingleChunk>(header, std::move(chunk));
|
||||
pipeline.complete(Pipe(std::move(source)));
|
||||
auto source = std::make_shared<SourceFromSingleChunk>(header, std::move(chunk));
|
||||
pipeline.complete(Pipe(std::move(source)));
|
||||
|
||||
CompletedPipelineExecutor completed_executor(pipeline);
|
||||
completed_executor.execute();
|
||||
CompletedPipelineExecutor completed_executor(pipeline);
|
||||
completed_executor.execute();
|
||||
|
||||
LOG_INFO(log, "Flushed {} rows, {} bytes for query '{}'",
|
||||
total_rows, total_bytes, queryToString(key.query));
|
||||
LOG_INFO(log, "Flushed {} rows, {} bytes for query '{}'",
|
||||
total_rows, total_bytes, queryToString(key.query));
|
||||
}
|
||||
|
||||
for (const auto & entry : data->entries)
|
||||
if (!entry->isFinished())
|
||||
|
@ -0,0 +1,5 @@
|
||||
Cannot parse object
|
||||
0
|
||||
0
|
||||
Cannot parse object
|
||||
aaa
|
21
tests/queries/0_stateless/02421_type_json_async_insert.sh
Executable file
21
tests/queries/0_stateless/02421_type_json_async_insert.sh
Executable file
@ -0,0 +1,21 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_async_insert"
|
||||
$CLICKHOUSE_CLIENT --allow_experimental_object_type=1 -q "CREATE TABLE t_json_async_insert (data JSON) ENGINE = MergeTree ORDER BY tuple()"
|
||||
|
||||
$CLICKHOUSE_CLIENT --async_insert=1 --wait_for_async_insert=1 -q 'INSERT INTO t_json_async_insert FORMAT JSONAsObject {"aaa"}' 2>&1 | grep -o -m1 "Cannot parse object"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count() FROM t_json_async_insert"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.parts WHERE database = '$CLICKHOUSE_DATABASE' AND table = 't_json_async_insert'"
|
||||
|
||||
$CLICKHOUSE_CLIENT --async_insert=1 --wait_for_async_insert=1 -q 'INSERT INTO t_json_async_insert FORMAT JSONAsObject {"aaa"}' 2>&1 | grep -o -m1 "Cannot parse object" &
|
||||
$CLICKHOUSE_CLIENT --async_insert=1 --wait_for_async_insert=1 -q 'INSERT INTO t_json_async_insert FORMAT JSONAsObject {"k1": "aaa"}' &
|
||||
|
||||
wait
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT data.k1 FROM t_json_async_insert ORDER BY data.k1"
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_json_async_insert"
|
@ -0,0 +1,26 @@
|
||||
Collapsing
|
||||
0
|
||||
0
|
||||
id UInt64
|
||||
s Int8
|
||||
data Tuple(_dummy UInt8)
|
||||
DELETE all
|
||||
2
|
||||
1
|
||||
id UInt64
|
||||
data Tuple(k1 String, k2 String)
|
||||
0
|
||||
0
|
||||
id UInt64
|
||||
data Tuple(_dummy UInt8)
|
||||
TTL
|
||||
1
|
||||
1
|
||||
id UInt64
|
||||
d Date
|
||||
data Tuple(k1 String, k2 String)
|
||||
0
|
||||
0
|
||||
id UInt64
|
||||
d Date
|
||||
data Tuple(_dummy UInt8)
|
61
tests/queries/0_stateless/02421_type_json_empty_parts.sql
Normal file
61
tests/queries/0_stateless/02421_type_json_empty_parts.sql
Normal file
@ -0,0 +1,61 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
SET allow_experimental_object_type = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t_json_empty_parts;
|
||||
|
||||
SELECT 'Collapsing';
|
||||
CREATE TABLE t_json_empty_parts (id UInt64, s Int8, data JSON) ENGINE = CollapsingMergeTree(s) ORDER BY id;
|
||||
|
||||
INSERT INTO t_json_empty_parts VALUES (1, 1, '{"k1": "aaa"}') (1, -1, '{"k2": "bbb"}');
|
||||
|
||||
SELECT count() FROM t_json_empty_parts;
|
||||
SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;
|
||||
DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;
|
||||
|
||||
DROP TABLE t_json_empty_parts;
|
||||
|
||||
DROP TABLE IF EXISTS t_json_empty_parts;
|
||||
|
||||
SELECT 'DELETE all';
|
||||
CREATE TABLE t_json_empty_parts (id UInt64, data JSON) ENGINE = MergeTree ORDER BY id;
|
||||
|
||||
INSERT INTO t_json_empty_parts VALUES (1, '{"k1": "aaa"}') (2, '{"k2": "bbb"}');
|
||||
|
||||
SELECT count() FROM t_json_empty_parts;
|
||||
SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;
|
||||
DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;
|
||||
|
||||
SET mutations_sync = 2;
|
||||
ALTER TABLE t_json_empty_parts DELETE WHERE 1;
|
||||
|
||||
DETACH TABLE t_json_empty_parts;
|
||||
ATTACH TABLE t_json_empty_parts;
|
||||
|
||||
SELECT count() FROM t_json_empty_parts;
|
||||
SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;
|
||||
DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t_json_empty_parts;
|
||||
|
||||
SELECT 'TTL';
|
||||
CREATE TABLE t_json_empty_parts (id UInt64, d Date, data JSON) ENGINE = MergeTree ORDER BY id TTL d WHERE id % 2 = 1;
|
||||
|
||||
INSERT INTO t_json_empty_parts VALUES (1, '2000-01-01', '{"k1": "aaa"}') (2, '2000-01-01', '{"k2": "bbb"}');
|
||||
OPTIMIZE TABLE t_json_empty_parts FINAL;
|
||||
|
||||
SELECT count() FROM t_json_empty_parts;
|
||||
SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;
|
||||
DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;
|
||||
|
||||
ALTER TABLE t_json_empty_parts MODIFY TTL d;
|
||||
OPTIMIZE TABLE t_json_empty_parts FINAL;
|
||||
|
||||
DETACH TABLE t_json_empty_parts;
|
||||
ATTACH TABLE t_json_empty_parts;
|
||||
|
||||
SELECT count() FROM t_json_empty_parts;
|
||||
SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;
|
||||
DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t_json_empty_parts;
|
Loading…
Reference in New Issue
Block a user