mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #41290 from CurtizJ/dynamic-columns-20
Fix `ALTER ADD COLUMN` with columns of type `Object`
This commit is contained in:
commit
3623807527
@ -1,3 +1,4 @@
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <DataTypes/DataTypeObject.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
@ -159,6 +160,16 @@ void convertObjectsToTuples(Block & block, const NamesAndTypesList & extended_st
|
||||
}
|
||||
}
|
||||
|
||||
void deduceTypesOfObjectColumns(const StorageSnapshotPtr & storage_snapshot, Block & block)
|
||||
{
|
||||
if (!storage_snapshot->object_columns.empty())
|
||||
{
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects();
|
||||
auto storage_columns = storage_snapshot->getColumns(options);
|
||||
convertObjectsToTuples(block, storage_columns);
|
||||
}
|
||||
}
|
||||
|
||||
static bool isPrefix(const PathInData::Parts & prefix, const PathInData::Parts & parts)
|
||||
{
|
||||
if (prefix.size() > parts.size())
|
||||
|
@ -11,6 +11,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct StorageSnapshot;
|
||||
using StorageSnapshotPtr = std::shared_ptr<StorageSnapshot>;
|
||||
|
||||
/// Returns number of dimensions in Array type. 0 if type is not array.
|
||||
size_t getNumberOfDimensions(const IDataType & type);
|
||||
|
||||
@ -38,6 +41,7 @@ DataTypePtr getDataTypeByColumn(const IColumn & column);
|
||||
/// Converts Object types and columns to Tuples in @columns_list and @block
|
||||
/// and checks that types are consistent with types in @extended_storage_columns.
|
||||
void convertObjectsToTuples(Block & block, const NamesAndTypesList & extended_storage_columns);
|
||||
void deduceTypesOfObjectColumns(const StorageSnapshotPtr & storage_snapshot, Block & block);
|
||||
|
||||
/// Checks that each path is not the prefix of any other path.
|
||||
void checkObjectHasNoAmbiguosPaths(const PathsInData & paths);
|
||||
@ -164,27 +168,24 @@ ColumnsDescription getObjectColumns(
|
||||
const ColumnsDescription & storage_columns,
|
||||
EntryColumnsGetter && entry_columns_getter)
|
||||
{
|
||||
ColumnsDescription res;
|
||||
|
||||
if (begin == end)
|
||||
{
|
||||
for (const auto & column : storage_columns)
|
||||
{
|
||||
if (isObject(column.type))
|
||||
{
|
||||
auto tuple_type = std::make_shared<DataTypeTuple>(
|
||||
DataTypes{std::make_shared<DataTypeUInt8>()},
|
||||
Names{ColumnObject::COLUMN_NAME_DUMMY});
|
||||
|
||||
res.add({column.name, std::move(tuple_type)});
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
std::unordered_map<String, DataTypes> types_in_entries;
|
||||
|
||||
/// Add dummy column for all Object columns
|
||||
/// to not lose any column if it's missing
|
||||
/// in all entries. If it exists in any entry
|
||||
/// dummy column will be removed.
|
||||
for (const auto & column : storage_columns)
|
||||
{
|
||||
if (isObject(column.type))
|
||||
{
|
||||
auto tuple_type = std::make_shared<DataTypeTuple>(
|
||||
DataTypes{std::make_shared<DataTypeUInt8>()},
|
||||
Names{ColumnObject::COLUMN_NAME_DUMMY});
|
||||
|
||||
types_in_entries[column.name].push_back(std::move(tuple_type));
|
||||
}
|
||||
}
|
||||
|
||||
for (auto it = begin; it != end; ++it)
|
||||
{
|
||||
const auto & entry_columns = entry_columns_getter(*it);
|
||||
@ -196,6 +197,7 @@ ColumnsDescription getObjectColumns(
|
||||
}
|
||||
}
|
||||
|
||||
ColumnsDescription res;
|
||||
for (const auto & [name, types] : types_in_entries)
|
||||
res.add({name, getLeastCommonTypeForObject(types)});
|
||||
|
||||
|
@ -1242,6 +1242,9 @@ protected:
|
||||
/// Attaches restored parts to the storage.
|
||||
virtual void attachRestoredParts(MutableDataPartsVector && parts) = 0;
|
||||
|
||||
void resetObjectColumnsFromActiveParts(const DataPartsLock & lock);
|
||||
void updateObjectColumns(const DataPartPtr & part, const DataPartsLock & lock);
|
||||
|
||||
static void incrementInsertedPartsProfileEvent(MergeTreeDataPartType type);
|
||||
static void incrementMergedPartsProfileEvent(MergeTreeDataPartType type);
|
||||
|
||||
@ -1329,9 +1332,6 @@ private:
|
||||
DataPartsVector & duplicate_parts_to_remove,
|
||||
MutableDataPartsVector & parts_from_wal);
|
||||
|
||||
void resetObjectColumnsFromActiveParts(const DataPartsLock & lock);
|
||||
void updateObjectColumns(const DataPartPtr & part, const DataPartsLock & lock);
|
||||
|
||||
/// Create zero-copy exclusive lock for part and disk. Useful for coordination of
|
||||
/// distributed operations which can lead to data duplication. Implemented only in ReplicatedMergeTree.
|
||||
virtual std::optional<ZeroCopyLock> tryCreateZeroCopyExclusiveLock(const String &, const DiskPtr &) { return std::nullopt; }
|
||||
|
@ -483,16 +483,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart(
|
||||
return temp_part;
|
||||
}
|
||||
|
||||
void MergeTreeDataWriter::deduceTypesOfObjectColumns(const StorageSnapshotPtr & storage_snapshot, Block & block)
|
||||
{
|
||||
if (!storage_snapshot->object_columns.empty())
|
||||
{
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects();
|
||||
auto storage_columns = storage_snapshot->getColumns(options);
|
||||
convertObjectsToTuples(block, storage_columns);
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
|
||||
const String & part_name,
|
||||
MergeTreeDataPartType part_type,
|
||||
|
@ -45,8 +45,6 @@ public:
|
||||
*/
|
||||
static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context);
|
||||
|
||||
static void deduceTypesOfObjectColumns(const StorageSnapshotPtr & storage_snapshot, Block & block);
|
||||
|
||||
/// This structure contains not completely written temporary part.
|
||||
/// Some writes may happen asynchronously, e.g. for blob storages.
|
||||
/// You should call finalize() to wait until all data is written.
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeSink.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -56,7 +57,7 @@ void MergeTreeSink::consume(Chunk chunk)
|
||||
{
|
||||
auto block = getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
|
||||
storage.writer.deduceTypesOfObjectColumns(storage_snapshot, block);
|
||||
deduceTypesOfObjectColumns(storage_snapshot, block);
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context);
|
||||
|
||||
using DelayedPartitions = std::vector<MergeTreeSink::DelayedChunk::Partition>;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeSink.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
@ -161,7 +162,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk)
|
||||
*/
|
||||
size_t replicas_num = checkQuorumPrecondition(zookeeper);
|
||||
|
||||
storage.writer.deduceTypesOfObjectColumns(storage_snapshot, block);
|
||||
deduceTypesOfObjectColumns(storage_snapshot, block);
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context);
|
||||
|
||||
using DelayedPartitions = std::vector<ReplicatedMergeTreeSink::DelayedChunk::Partition>;
|
||||
|
@ -335,6 +335,13 @@ void StorageMergeTree::alter(
|
||||
mutation_version = startMutation(maybe_mutation_commands, local_context);
|
||||
}
|
||||
|
||||
{
|
||||
/// Reset Object columns, because column of type
|
||||
/// Object may be added or dropped by alter.
|
||||
auto parts_lock = lockParts();
|
||||
resetObjectColumnsFromActiveParts(parts_lock);
|
||||
}
|
||||
|
||||
/// Always execute required mutations synchronously, because alters
|
||||
/// should be executed in sequential order.
|
||||
if (!maybe_mutation_commands.empty())
|
||||
|
@ -4649,6 +4649,13 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer
|
||||
LOG_INFO(log, "Applied changes to the metadata of the table. Current metadata version: {}", metadata_version);
|
||||
}
|
||||
|
||||
{
|
||||
/// Reset Object columns, because column of type
|
||||
/// Object may be added or dropped by alter.
|
||||
auto parts_lock = lockParts();
|
||||
resetObjectColumnsFromActiveParts(parts_lock);
|
||||
}
|
||||
|
||||
/// This transaction may not happen, but it's OK, because on the next retry we will eventually create/update this node
|
||||
/// TODO Maybe do in in one transaction for Replicated database?
|
||||
zookeeper->createOrUpdate(fs::path(replica_path) / "metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent);
|
||||
|
@ -0,0 +1,6 @@
|
||||
{% for storage in ["MergeTree", "ReplicatedMergeTree('/clickhouse/tables/{database}/test_01825_add_column/', 'r1')"] -%}
|
||||
{"id":"1","s":{"k1":0}}
|
||||
{"id":"2","s":{"k1":100}}
|
||||
{"id":"1"}
|
||||
{"id":"2"}
|
||||
{% endfor -%}
|
23
tests/queries/0_stateless/01825_type_json_add_column.sql.j2
Normal file
23
tests/queries/0_stateless/01825_type_json_add_column.sql.j2
Normal file
@ -0,0 +1,23 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
{% for storage in ["MergeTree", "ReplicatedMergeTree('/clickhouse/tables/{database}/test_01825_add_column/', 'r1')"] -%}
|
||||
|
||||
DROP TABLE IF EXISTS t_json_add_column;
|
||||
SET allow_experimental_object_type = 1;
|
||||
|
||||
CREATE TABLE t_json_add_column (id UInt64) ENGINE = {{ storage }} ORDER BY tuple();
|
||||
|
||||
INSERT INTO t_json_add_column VALUES (1);
|
||||
ALTER TABLE t_json_add_column ADD COLUMN s JSON;
|
||||
|
||||
INSERT INTO t_json_add_column VALUES(2, '{"k1": 100}');
|
||||
|
||||
SELECT * FROM t_json_add_column ORDER BY id FORMAT JSONEachRow;
|
||||
|
||||
ALTER TABLE t_json_add_column DROP COLUMN s;
|
||||
|
||||
SELECT * FROM t_json_add_column ORDER BY id FORMAT JSONEachRow;
|
||||
|
||||
DROP TABLE t_json_add_column;
|
||||
|
||||
{% endfor -%}
|
Loading…
Reference in New Issue
Block a user