Merge branch 'master' into fix_duplicated_code

This commit is contained in:
JaySon 2023-04-15 02:42:46 +08:00 committed by GitHub
commit 186e3c4251
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
65 changed files with 472 additions and 115 deletions

View File

@ -1,5 +1,6 @@
#pragma once
#include <base/extended_types.h>
#include <base/Decimal_fwd.h>
#if !defined(NO_SANITIZE_UNDEFINED)
#if defined(__clang__)
@ -19,23 +20,6 @@ using Decimal64 = Decimal<Int64>;
using Decimal128 = Decimal<Int128>;
using Decimal256 = Decimal<Int256>;
template <class T>
concept is_decimal =
std::is_same_v<T, Decimal32>
|| std::is_same_v<T, Decimal64>
|| std::is_same_v<T, Decimal128>
|| std::is_same_v<T, Decimal256>
|| std::is_same_v<T, DateTime64>;
template <class T>
concept is_over_big_int =
std::is_same_v<T, Int128>
|| std::is_same_v<T, UInt128>
|| std::is_same_v<T, Int256>
|| std::is_same_v<T, UInt256>
|| std::is_same_v<T, Decimal128>
|| std::is_same_v<T, Decimal256>;
template <class T> struct NativeTypeT { using Type = T; };
template <is_decimal T> struct NativeTypeT<T> { using Type = typename T::NativeType; };
template <class T> using NativeType = typename NativeTypeT<T>::Type;

46
base/base/Decimal_fwd.h Normal file
View File

@ -0,0 +1,46 @@
#pragma once
#include <base/types.h>
namespace wide
{
template <size_t Bits, typename Signed>
class integer;
}
using Int128 = wide::integer<128, signed>;
using UInt128 = wide::integer<128, unsigned>;
using Int256 = wide::integer<256, signed>;
using UInt256 = wide::integer<256, unsigned>;
namespace DB
{
template <class> struct Decimal;
using Decimal32 = Decimal<Int32>;
using Decimal64 = Decimal<Int64>;
using Decimal128 = Decimal<Int128>;
using Decimal256 = Decimal<Int256>;
class DateTime64;
template <class T>
concept is_decimal =
std::is_same_v<T, Decimal32>
|| std::is_same_v<T, Decimal64>
|| std::is_same_v<T, Decimal128>
|| std::is_same_v<T, Decimal256>
|| std::is_same_v<T, DateTime64>;
template <class T>
concept is_over_big_int =
std::is_same_v<T, Int128>
|| std::is_same_v<T, UInt128>
|| std::is_same_v<T, Int256>
|| std::is_same_v<T, UInt256>
|| std::is_same_v<T, Decimal128>
|| std::is_same_v<T, Decimal256>;
}

2
contrib/zstd vendored

@ -1 +1 @@
Subproject commit 945f27758c0fd67b636103a38dbf050266c6b90a
Subproject commit 63779c798237346c2b245c546c40b72a5a5913fe

View File

@ -6,13 +6,15 @@ sidebar_label: SHOW
# SHOW Statements
## SHOW CREATE TABLE
## SHOW CREATE TABLE | DICTIONARY | VIEW | DATABASE
``` sql
SHOW CREATE [TEMPORARY] [TABLE|DICTIONARY|VIEW] [db.]table|view [INTO OUTFILE filename] [FORMAT format]
SHOW [CREATE] [TEMPORARY] TABLE|DICTIONARY|VIEW|DATABASE [db.]table|view [INTO OUTFILE filename] [FORMAT format]
```
Returns a single `String`-type statement column, which contains a single value the `CREATE` query used for creating the specified object.
Returns a single column of type String containing the CREATE query used for creating the specified object.
`SHOW TABLE t` and `SHOW DATABASE db` have the same meaning as `SHOW CREATE TABLE|DATABASE t|db`, but `SHOW t` and `SHOW db` are not supported.
Note that if you use this statement to get `CREATE` query of system tables, you will get a *fake* query, which only declares table structure, but cannot be used to create table.

View File

@ -1,5 +1,6 @@
#include <Analyzer/ConstantNode.h>
#include <Common/assert_cast.h>
#include <Common/FieldVisitorToString.h>
#include <Common/SipHash.h>

View File

@ -3,6 +3,7 @@
#include <Core/Field.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ConstantValue.h>
namespace DB
{

View File

@ -11,7 +11,6 @@
#include <Parsers/IAST_fwd.h>
#include <Analyzer/Identifier.h>
#include <Analyzer/ConstantValue.h>
class SipHash;

View File

@ -1,5 +1,6 @@
#include <Analyzer/IdentifierNode.h>
#include <Common/assert_cast.h>
#include <Common/SipHash.h>
#include <IO/WriteBufferFromString.h>

View File

@ -1,5 +1,6 @@
#include <Analyzer/LambdaNode.h>
#include <Common/assert_cast.h>
#include <Common/SipHash.h>
#include <IO/WriteBuffer.h>

View File

@ -1,5 +1,6 @@
#include <Analyzer/MatcherNode.h>
#include <Common/assert_cast.h>
#include <Common/SipHash.h>
#include <IO/WriteBuffer.h>

View File

@ -118,7 +118,7 @@ namespace ErrorCodes
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
}
/** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before.
/** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h first.
* And additional documentation for each method, where special cases are described in detail.
*
* Each node in query must be resolved. For each query tree node resolved state is specific.
@ -159,7 +159,7 @@ namespace ErrorCodes
*
* TODO: This does not supported properly before, because matchers could not be resolved from aliases.
*
* Identifiers are resolved with following resules:
* Identifiers are resolved with following rules:
* Resolution starts with current scope.
* 1. Try to resolve identifier from expression scope arguments. Lambda expression arguments are greatest priority.
* 2. Try to resolve identifier from aliases.

View File

@ -1,6 +1,5 @@
#pragma once
#include <Parsers/IAST_fwd.h>
#include <Interpreters/Context_fwd.h>
#include <Analyzer/IQueryTreePass.h>

View File

@ -2,6 +2,7 @@
#include <fmt/core.h>
#include <Common/assert_cast.h>
#include <Common/SipHash.h>
#include <Common/FieldVisitorToString.h>

View File

@ -1,5 +1,6 @@
#include <Analyzer/SortNode.h>
#include <Common/assert_cast.h>
#include <Common/SipHash.h>
#include <IO/WriteBufferFromString.h>

View File

@ -35,6 +35,7 @@
#include <Parsers/ASTCreateFunctionQuery.h>
#include <Parsers/Access/ASTCreateUserQuery.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTUseQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>

View File

@ -5,7 +5,7 @@
#include <Common/Exception.h>
#include <Common/typeid_cast.h>
#include <base/StringRef.h>
#include <Core/Types.h>
#include <Core/TypeId.h>
#include "config.h"

View File

@ -8,6 +8,7 @@
#include <Columns/ColumnTuple.h>
#include <Common/FieldVisitorToString.h>
#include <Common/assert_cast.h>
#include <Common/randomSeed.h>
#include <fmt/core.h>
#include <pcg_random.hpp>

View File

@ -1,9 +1,55 @@
#pragma once
#include <Core/Types.h>
#include <Core/Types_fwd.h>
namespace DB
{
/// @note Except explicitly described you should not assume on TypeIndex numbers and/or their orders in this enum.
enum class TypeIndex
{
Nothing = 0,
UInt8,
UInt16,
UInt32,
UInt64,
UInt128,
UInt256,
Int8,
Int16,
Int32,
Int64,
Int128,
Int256,
Float32,
Float64,
Date,
Date32,
DateTime,
DateTime64,
String,
FixedString,
Enum8,
Enum16,
Decimal32,
Decimal64,
Decimal128,
Decimal256,
UUID,
Array,
Tuple,
Set,
Interval,
Nullable,
Function,
AggregateFunction,
LowCardinality,
Map,
Object,
IPv4,
IPv6,
};
/**
* Obtain TypeIndex value from real type if possible.
*

View File

@ -1,5 +1,6 @@
#pragma once
#include <Core/TypeId.h>
#include <cstdint>
#include <string>
#include <vector>
@ -44,52 +45,6 @@ struct Null
}
};
/// @note Except explicitly described you should not assume on TypeIndex numbers and/or their orders in this enum.
enum class TypeIndex
{
Nothing = 0,
UInt8,
UInt16,
UInt32,
UInt64,
UInt128,
UInt256,
Int8,
Int16,
Int32,
Int64,
Int128,
Int256,
Float32,
Float64,
Date,
Date32,
DateTime,
DateTime64,
String,
FixedString,
Enum8,
Enum16,
Decimal32,
Decimal64,
Decimal128,
Decimal256,
UUID,
Array,
Tuple,
Set,
Interval,
Nullable,
Function,
AggregateFunction,
LowCardinality,
Map,
Object,
IPv4,
IPv6,
};
using UInt128 = ::UInt128;
using UInt256 = ::UInt256;
using Int128 = ::Int128;

47
src/Core/Types_fwd.h Normal file
View File

@ -0,0 +1,47 @@
#pragma once
#include <cstdint>
#include <string>
#include <vector>
#include <unordered_set>
#include <base/strong_typedef.h>
#include <base/defines.h>
#include <base/types.h>
#include <base/Decimal_fwd.h>
namespace wide
{
template <size_t Bits, typename Signed>
class integer;
}
using Int128 = wide::integer<128, signed>;
using UInt128 = wide::integer<128, unsigned>;
using Int256 = wide::integer<256, signed>;
using UInt256 = wide::integer<256, unsigned>;
namespace DB
{
using UUID = StrongTypedef<UInt128, struct UUIDTag>;
using IPv4 = StrongTypedef<UInt32, struct IPv4Tag>;
struct IPv6;
struct Null;
using UInt128 = ::UInt128;
using UInt256 = ::UInt256;
using Int128 = ::Int128;
using Int256 = ::Int256;
enum class TypeIndex;
/// Not a data type in database, defined just for convenience.
using Strings = std::vector<String>;
using TypeIndexesSet = std::unordered_set<TypeIndex>;
}

View File

@ -2,7 +2,7 @@
#include <memory>
#include <cstddef>
#include <Core/Types.h>
#include <Core/Types_fwd.h>
#include <DataTypes/Serializations/ISerialization.h>
namespace DB

View File

@ -1,7 +1,8 @@
#pragma once
#include <DataTypes/DataTypeNumberBase.h>
#include <Core/Field.h>
#include <Common/DateLUT.h>
#include <DataTypes/DataTypeNumberBase.h>
namespace DB
{

View File

@ -1,5 +1,6 @@
#pragma once
#include <base/TypeName.h>
#include <Core/TypeId.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/Serializations/SerializationNumber.h>
@ -8,9 +9,6 @@
namespace DB
{
template <typename T>
class ColumnVector;
/** Implements part of the IDataType interface, common to all numbers and for Date and DateTime.
*/
template <typename T>

View File

@ -1,7 +1,7 @@
#pragma once
#include <Common/COW.h>
#include <Core/Types.h>
#include <Core/Types_fwd.h>
#include <base/demangle.h>
#include <Common/typeid_cast.h>
#include <Columns/IColumn.h>

View File

@ -1,10 +1,11 @@
#include <DataTypes/Serializations/SerializationEnum.h>
#include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <IO/WriteBufferFromString.h>
#include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <Common/assert_cast.h>
namespace DB
{

View File

@ -1,9 +1,10 @@
#pragma once
#include <base/TypeName.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnsNumber.h>
#include <base/TypeName.h>
#include <DataTypes/Serializations/SimpleTextSerialization.h>
namespace DB
{

View File

@ -1,6 +1,6 @@
#pragma once
#include <Core/Types.h>
#include <Core/Types_fwd.h>
#include <DataTypes/Serializations/ISerialization.h>
#include <Poco/JSON/Object.h>

View File

@ -1,11 +1,14 @@
#pragma once
#include <Core/Types.h>
#include <DataTypes/Serializations/SimpleTextSerialization.h>
#include <Columns/ColumnVector.h>
namespace DB
{
template <typename T>
class ColumnVector;
template <typename T>
class SerializationNumber : public SimpleTextSerialization
{

View File

@ -360,6 +360,19 @@ SyncGuardPtr DiskEncrypted::getDirectorySyncGuard(const String & path) const
return delegate->getDirectorySyncGuard(wrapped_path);
}
std::unordered_map<String, String> DiskEncrypted::getSerializedMetadata(const std::vector<String> & paths) const
{
std::vector<String> wrapped_paths;
wrapped_paths.reserve(paths.size());
for (const auto & path : paths)
wrapped_paths.emplace_back(wrappedPath(path));
auto metadata = delegate->getSerializedMetadata(wrapped_paths);
std::unordered_map<String, String> res;
for (size_t i = 0; i != paths.size(); ++i)
res.emplace(paths[i], metadata.at(wrapped_paths.at(i)));
return res;
}
void DiskEncrypted::applyNewSettings(
const Poco::Util::AbstractConfiguration & config,
ContextPtr /*context*/,

View File

@ -225,6 +225,11 @@ public:
return delegate->getUniqueId(wrapped_path);
}
bool checkUniqueId(const String & id) const override
{
return delegate->checkUniqueId(id);
}
void onFreeze(const String & path) override
{
auto wrapped_path = wrappedPath(path);
@ -276,6 +281,8 @@ public:
return delegate->getMetadataStorage();
}
std::unordered_map<String, String> getSerializedMetadata(const std::vector<String> & paths) const override;
DiskPtr getDelegateDiskIfExists() const override
{
return delegate;

View File

@ -1,13 +1,10 @@
#include <DataTypes/DataTypeDateTime.h>
#include <Functions/IFunction.h>
#include <Core/DecimalFunctions.h>
#include <Functions/FunctionFactory.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <ctime>
#include <Core/Field.h>
#include <Core/DecimalFunctions.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
namespace DB
{

View File

@ -3,6 +3,7 @@
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Columns/ColumnsDateTime.h>
#include <Columns/ColumnVector.h>
namespace DB

View File

@ -17,6 +17,7 @@
#include <Common/LocalDateTime.h>
#include <base/StringRef.h>
#include <base/arithmeticOverflow.h>
#include <base/sort.h>
#include <base/unit.h>
#include <Core/Types.h>
@ -38,8 +39,6 @@
#include <IO/PeekableReadBuffer.h>
#include <IO/VarInt.h>
#include <DataTypes/DataTypeDateTime.h>
#include <double-conversion/double-conversion.h>
static constexpr auto DEFAULT_MAX_STRING_SIZE = 1_GiB;

View File

@ -17,7 +17,6 @@
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/MergeTreeTransactionHolder.h>
#include <IO/IResourceManager.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/IAST_fwd.h>
#include <Server/HTTP/HTTPContext.h>
#include <Storages/ColumnsDescription.h>
@ -42,6 +41,8 @@ struct OvercommitTracker;
namespace DB
{
class ASTSelectQuery;
struct ContextSharedPart;
class ContextAccess;
struct User;

View File

@ -136,15 +136,24 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
std::tie(ref_dependencies, loading_dependencies) = database_catalog.removeDependencies(from_table_id, check_ref_deps, check_loading_deps);
}
database->renameTable(
getContext(),
elem.from_table_name,
*database_catalog.getDatabase(elem.to_database_name),
elem.to_table_name,
exchange_tables,
rename.dictionary);
try
{
database->renameTable(
getContext(),
elem.from_table_name,
*database_catalog.getDatabase(elem.to_database_name),
elem.to_table_name,
exchange_tables,
rename.dictionary);
DatabaseCatalog::instance().addDependencies(to_table_id, ref_dependencies, loading_dependencies);
DatabaseCatalog::instance().addDependencies(to_table_id, ref_dependencies, loading_dependencies);
}
catch (...)
{
/// Restore dependencies if RENAME fails
DatabaseCatalog::instance().addDependencies(from_table_id, ref_dependencies, loading_dependencies);
throw;
}
}
}

View File

@ -3,8 +3,6 @@
#include <Interpreters/IInterpreter.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Storages/MergeTree/RequestResponse.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Analyzer/QueryTreePassManager.h>
#include <Planner/Planner.h>
#include <Interpreters/Context_fwd.h>
@ -65,9 +63,6 @@ public:
bool ignoreQuota() const override { return select_query_options.ignore_quota; }
/// Set merge tree read task callback in context and set collaborate_with_initiator in client info
void setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback);
/// Set number_of_current_replica and count_participating_replicas in client_info
void setProperClientInfo(size_t replica_number, size_t count_participating_replicas);

View File

@ -1,5 +1,6 @@
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTExplainQuery.h>

View File

@ -13,6 +13,8 @@
namespace DB
{
class NamesAndTypesList;
namespace MySQLInterpreter
{
struct InterpreterDropImpl

View File

@ -10,6 +10,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime.h>
namespace ProfileEvents
{

View File

@ -7,13 +7,15 @@
#include <DataTypes/IDataType.h>
#include <Core/Names.h>
#include <Core/Types.h>
#include <Processors/QueryPlan/FilterStep.h>
namespace DB
{
class ASTFunction;
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
struct WindowFunctionDescription
{
std::string column_name;

View File

@ -2,6 +2,7 @@
#include <Parsers/ASTFunction.h>
#include <Common/assert_cast.h>
#include <Common/quoteString.h>
#include <Common/FieldVisitorToString.h>
#include <Common/KnownObjectNames.h>

View File

@ -1,4 +1,6 @@
#include <Parsers/ASTQueryWithOutput.h>
#include <Common/assert_cast.h>
#include <Parsers/ASTSetQuery.h>
namespace DB

View File

@ -1,3 +1,4 @@
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTFunction.h>

View File

@ -18,6 +18,7 @@
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserUnionQueryElement.h>
#include <Parsers/parseIntervalKind.h>
#include <Common/assert_cast.h>
#include <Common/StringUtils/StringUtils.h>
#include <Parsers/ParserSelectWithUnionQuery.h>

View File

@ -27,7 +27,8 @@
#include <Parsers/Access/ParserShowCreateAccessEntityQuery.h>
#include <Parsers/Access/ParserShowGrantsQuery.h>
#include <Parsers/Access/ParserShowPrivilegesQuery.h>
#include "Common/Exception.h"
#include <Common/Exception.h>
#include <Common/assert_cast.h>
namespace DB

View File

@ -61,8 +61,13 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
}
else if (s_show.ignore(pos, expected))
{
if (!s_create.ignore(pos, expected))
return false;
bool has_create = false;
if (s_create.checkWithoutMoving(pos, expected))
{
has_create = true;
s_create.ignore(pos, expected);
}
if (s_database.ignore(pos, expected))
{
@ -77,7 +82,15 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
parse_show_create_view = true;
}
else
query = std::make_shared<ASTShowCreateTableQuery>();
{
/// We support `SHOW CREATE tbl;` and `SHOW TABLE tbl`,
/// but do not support `SHOW tbl`, which is ambiguous
/// with other statement like `SHOW PRIVILEGES`.
if (has_create || s_table.checkWithoutMoving(pos, expected))
query = std::make_shared<ASTShowCreateTableQuery>();
else
return false;
}
}
else
{

View File

@ -10,6 +10,7 @@
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Common/UTF8Helpers.h>
#include <Common/PODArray.h>
namespace DB
{

View File

@ -12,6 +12,7 @@
#include <Processors/QueryPlan/CubeStep.h>
#include <Processors/QueryPlan/DistinctStep.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/FilterStep.h>
#include <Processors/QueryPlan/ITransformingStep.h>
#include <Processors/QueryPlan/JoinStep.h>
#include <Processors/QueryPlan/Optimizations/Optimizations.h>

View File

@ -3,6 +3,7 @@
#include <Processors/QueryPlan/AggregatingStep.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/FillingStep.h>
#include <Processors/QueryPlan/FilterStep.h>
#include <Processors/QueryPlan/ITransformingStep.h>
#include <Processors/QueryPlan/JoinStep.h>
#include <Processors/QueryPlan/LimitByStep.h>

View File

@ -11,6 +11,7 @@
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeDateTime.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>

View File

@ -9,6 +9,7 @@
#include <Disks/IDisk.h>
#include <Common/formatReadable.h>
#include <Common/StringUtils/StringUtils.h>
#include <Interpreters/Context.h>
#include <Common/logger_useful.h>
#include <Interpreters/Set.h>
#include <Processors/Sinks/SinkToStorage.h>

View File

@ -1,6 +1,6 @@
#pragma once
#include <Interpreters/Context.h>
#include <Interpreters/Context_fwd.h>
#include <Storages/IStorage.h>
#include <Storages/SetSettings.h>
@ -8,6 +8,9 @@
namespace DB
{
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
class Set;
using SetPtr = std::shared_ptr<Set>;

View File

@ -1,5 +1,6 @@
#include <Storages/System/StorageSystemBackups.h>
#include <Backups/BackupsWorker.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>

View File

@ -1,5 +1,6 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Storages/System/StorageSystemDistributionQueue.h>

View File

@ -1,5 +1,6 @@
#include <Storages/System/StorageSystemDroppedTables.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeUUID.h>
@ -7,7 +8,7 @@
#include <Columns/ColumnsNumber.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include "base/types.h"
#include <base/types.h>
namespace DB

View File

@ -0,0 +1,16 @@
<clickhouse>
<remote_servers>
<cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</cluster>
</remote_servers>
</clickhouse>

View File

@ -0,0 +1,49 @@
<clickhouse>
<storage_configuration>
<disks>
<disk_s3>
<type>s3</type>
<endpoint>http://minio1:9001/root/data/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</disk_s3>
<disk_s3_encrypted>
<type>encrypted</type>
<disk>disk_s3</disk>
<key>1234567812345678</key>
<path>encrypted/</path>
</disk_s3_encrypted>
<disk_s3_encrypted_with_diff_keys>
<type>encrypted</type>
<disk>disk_s3</disk>
<path>encrypted_with_diff_keys/</path>
</disk_s3_encrypted_with_diff_keys>
</disks>
<policies>
<s3_policy>
<volumes>
<main>
<disk>disk_s3</disk>
</main>
<external>
<disk>disk_s3_encrypted</disk>
</external>
</volumes>
</s3_policy>
<s3_encrypted_policy>
<volumes>
<main>
<disk>disk_s3_encrypted</disk>
</main>
</volumes>
</s3_encrypted_policy>
<s3_encrypted_policy_with_diff_keys>
<volumes>
<main>
<disk>disk_s3_encrypted_with_diff_keys</disk>
</main>
</volumes>
</s3_encrypted_policy_with_diff_keys>
</policies>
</storage_configuration>
</clickhouse>

View File

@ -0,0 +1,9 @@
<clickhouse>
<storage_configuration>
<disks>
<disk_s3_encrypted_with_diff_keys>
<key>1111111111111111</key>
</disk_s3_encrypted_with_diff_keys>
</disks>
</storage_configuration>
</clickhouse>

View File

@ -0,0 +1,9 @@
<clickhouse>
<storage_configuration>
<disks>
<disk_s3_encrypted_with_diff_keys>
<key>2222222222222222</key>
</disk_s3_encrypted_with_diff_keys>
</disks>
</storage_configuration>
</clickhouse>

View File

@ -0,0 +1,92 @@
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1",
main_configs=[
"configs/cluster.xml",
"configs/disk_s3_encrypted.xml",
"configs/disk_s3_encrypted_node1.xml",
],
macros={"replica": "node1"},
with_zookeeper=True,
with_minio=True,
)
node2 = cluster.add_instance(
"node2",
main_configs=[
"configs/cluster.xml",
"configs/disk_s3_encrypted.xml",
"configs/disk_s3_encrypted_node2.xml",
],
macros={"replica": "node2"},
with_zookeeper=True,
with_minio=True,
)
@pytest.fixture(scope="module", autouse=True)
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def cleanup_after_test():
try:
yield
finally:
node1.query("DROP TABLE IF EXISTS encrypted_test ON CLUSTER 'cluster' NO DELAY")
def create_table(
zero_copy_replication=False, storage_policy="s3_encrypted_policy_with_diff_keys"
):
engine = "ReplicatedMergeTree('/clickhouse/tables/encrypted_test/', '{replica}')"
settings = f"storage_policy='{storage_policy}'"
if zero_copy_replication:
settings += ", allow_remote_fs_zero_copy_replication=true"
node1.query(
f"""
CREATE TABLE encrypted_test ON CLUSTER 'cluster' (
id Int64,
data String
) ENGINE={engine}
ORDER BY id
SETTINGS {settings}
"""
)
def check_replication():
node1.query("INSERT INTO encrypted_test VALUES (0, 'a'), (1, 'b')")
node2.query("INSERT INTO encrypted_test VALUES (2, 'c'), (3, 'd')")
node1.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' encrypted_test")
select_query = "SELECT * FROM encrypted_test ORDER BY id"
assert node1.query(select_query) == TSV([[0, "a"], [1, "b"], [2, "c"], [3, "d"]])
assert node2.query(select_query) == TSV([[0, "a"], [1, "b"], [2, "c"], [3, "d"]])
def test_replication():
create_table(
zero_copy_replication=False, storage_policy="s3_encrypted_policy_with_diff_keys"
)
check_replication()
def test_zero_copy_replication():
create_table(zero_copy_replication=True, storage_policy="s3_encrypted_policy")
check_replication()

View File

@ -30,6 +30,14 @@ mv [] [] []
s [] ['join'] ['t']
t ['mv'] ['dict1','join','s'] []
OK
UNKNOWN_DATABASE
dict1 [] ['dict_src'] ['join','t']
dict2 [] ['join'] []
dict_src [] [] ['dict1']
join [] ['dict1'] ['dict2','s','t']
mv [] [] []
s [] ['join'] ['t']
t ['mv'] ['dict1','join','s'] []
dict1 [] ['dict_src'] ['join','t']
dict2 [] ['join'] []
dict_src [] [] ['dict1']

View File

@ -64,6 +64,10 @@ else
echo "OK"
fi
$CLICKHOUSE_CLIENT -q "rename table t to ${CLICKHOUSE_DATABASE}_2.t" |& grep -m1 -F -o UNKNOWN_DATABASE
$CLICKHOUSE_CLIENT -q "select table, arraySort(dependencies_table),
arraySort(loading_dependencies_table), arraySort(loading_dependent_table) from system.tables where database in (currentDatabase(), '$t_database') order by table"
$CLICKHOUSE_CLIENT -q "drop table mv"
$CLICKHOUSE_CLIENT -q "create database ${CLICKHOUSE_DATABASE}_1"

View File

@ -0,0 +1,5 @@
CREATE TABLE default.t_2710_show_table\n(\n `n1` UInt32,\n `s` String\n)\nENGINE = Log
CREATE TABLE default.t_2710_show_table\n(\n `n1` UInt32,\n `s` String\n)\nENGINE = Log
CREATE TABLE default.t_2710_show_table\n(\n `n1` UInt32,\n `s` String\n)\nENGINE = Log
CREATE DATABASE t_2710_db\nENGINE = Atomic
CREATE DATABASE t_2710_db\nENGINE = Atomic

View File

@ -0,0 +1,16 @@
-- Tags: no-parallel
DROP TABLE IF EXISTS t_2710_show_table;
CREATE TABLE t_2710_show_table(n1 UInt32, s String) engine=Log;
SHOW TABLE t_2710_show_table;
SHOW CREATE TABLE t_2710_show_table;
SHOW CREATE t_2710_show_table;
DROP TABLE t_2710_show_table;
DROP DATABASE IF EXISTS t_2710_db;
CREATE DATABASE t_2710_db engine=Atomic;
SHOW DATABASE t_2710_db;
SHOW CREATE DATABASE t_2710_db;
DROP DATABASE t_2710_db;