mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into fix_duplicated_code
This commit is contained in:
commit
186e3c4251
@ -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
46
base/base/Decimal_fwd.h
Normal 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
2
contrib/zstd
vendored
@ -1 +1 @@
|
||||
Subproject commit 945f27758c0fd67b636103a38dbf050266c6b90a
|
||||
Subproject commit 63779c798237346c2b245c546c40b72a5a5913fe
|
@ -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.
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/ConstantNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Field.h>
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/ConstantValue.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
#include <Analyzer/Identifier.h>
|
||||
#include <Analyzer/ConstantValue.h>
|
||||
|
||||
class SipHash;
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/IdentifierNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/LambdaNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/MatcherNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
@ -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.
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Analyzer/IQueryTreePass.h>
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <fmt/core.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/SortNode.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
@ -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>
|
||||
|
@ -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"
|
||||
|
||||
|
@ -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>
|
||||
|
@ -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.
|
||||
*
|
||||
|
@ -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
47
src/Core/Types_fwd.h
Normal 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>;
|
||||
|
||||
}
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <memory>
|
||||
#include <cstddef>
|
||||
#include <Core/Types.h>
|
||||
#include <Core/Types_fwd.h>
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,7 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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>
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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*/,
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Columns/ColumnsDateTime.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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>
|
||||
|
@ -13,6 +13,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class NamesAndTypesList;
|
||||
|
||||
namespace MySQLInterpreter
|
||||
{
|
||||
struct InterpreterDropImpl
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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>
|
||||
|
@ -1,4 +1,6 @@
|
||||
#include <Parsers/ASTQueryWithOutput.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Common/PODArray.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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>;
|
||||
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
|
@ -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>
|
@ -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>
|
@ -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>
|
@ -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>
|
92
tests/integration/test_encrypted_disk_replication/test.py
Normal file
92
tests/integration/test_encrypted_disk_replication/test.py
Normal 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()
|
@ -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']
|
||||
|
@ -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"
|
||||
|
||||
|
5
tests/queries/0_stateless/02710_show_table.reference
Normal file
5
tests/queries/0_stateless/02710_show_table.reference
Normal 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
|
16
tests/queries/0_stateless/02710_show_table.sql
Normal file
16
tests/queries/0_stateless/02710_show_table.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user