mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
fixed alter
This commit is contained in:
parent
f937048872
commit
2de2b6f32e
@ -317,9 +317,15 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde
|
||||
}
|
||||
else if (type == ADD_INDEX)
|
||||
{
|
||||
ASTPtr new_indexes_decl_ast;
|
||||
if (indexes_decl_ast)
|
||||
new_indexes_decl_ast = indexes_decl_ast->clone();
|
||||
else
|
||||
new_indexes_decl_ast = ASTExpressionList().ptr();
|
||||
|
||||
if (std::any_of(
|
||||
indexes_decl_ast->children.cbegin(),
|
||||
indexes_decl_ast->children.cend(),
|
||||
new_indexes_decl_ast->children.cbegin(),
|
||||
new_indexes_decl_ast->children.cend(),
|
||||
[this](const ASTPtr & index_ast) {
|
||||
return typeid_cast<const ASTIndexDeclaration &>(*index_ast).name == index_name;
|
||||
}))
|
||||
@ -331,7 +337,6 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
auto new_indexes_decl_ast = indexes_decl_ast->clone();
|
||||
auto insert_it = new_indexes_decl_ast->children.end();
|
||||
|
||||
if (!after_index_name.empty())
|
||||
@ -354,7 +359,11 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde
|
||||
}
|
||||
else if (type == DROP_INDEX)
|
||||
{
|
||||
auto new_indexes_decl_ast = indexes_decl_ast->clone();
|
||||
ASTPtr new_indexes_decl_ast;
|
||||
if (indexes_decl_ast)
|
||||
new_indexes_decl_ast = indexes_decl_ast->clone();
|
||||
else
|
||||
new_indexes_decl_ast = ASTExpressionList().ptr();
|
||||
|
||||
auto erase_it = std::find_if(
|
||||
new_indexes_decl_ast->children.begin(),
|
||||
|
@ -1131,7 +1131,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands)
|
||||
bool unused_bool;
|
||||
|
||||
createConvertExpression(nullptr, getColumns().getAllPhysical(), new_columns.getAllPhysical(),
|
||||
skip_indexes_ast, new_indexes_ast,unused_expression, unused_map, unused_bool);
|
||||
skip_indexes_ast, new_indexes_ast, unused_expression, unused_map, unused_bool);
|
||||
}
|
||||
|
||||
void MergeTreeData::createConvertExpression(const DataPartPtr & part, const NamesAndTypesList & old_columns, const NamesAndTypesList & new_columns,
|
||||
@ -1153,18 +1153,19 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name
|
||||
|
||||
/// Remove old indices
|
||||
std::set<String> new_indices;
|
||||
for (const auto & index_decl : new_indices_ast->children)
|
||||
new_indices.emplace(dynamic_cast<const ASTIndexDeclaration &>(*index_decl.get()).name);
|
||||
|
||||
for (const auto & index_decl : old_indices_ast->children)
|
||||
{
|
||||
const auto & index = dynamic_cast<const ASTIndexDeclaration &>(*index_decl.get());
|
||||
if (!new_indices.count(index.name))
|
||||
if (new_indices_ast)
|
||||
for (const auto & index_decl : new_indices_ast->children)
|
||||
new_indices.emplace(dynamic_cast<const ASTIndexDeclaration &>(*index_decl.get()).name);
|
||||
if (old_indices_ast)
|
||||
for (const auto & index_decl : old_indices_ast->children)
|
||||
{
|
||||
out_rename_map["skp_idx_" + index.name + ".idx"] = "";
|
||||
out_rename_map["skp_idx_" + index.name + ".mrk"] = "";
|
||||
const auto & index = dynamic_cast<const ASTIndexDeclaration &>(*index_decl.get());
|
||||
if (!new_indices.count(index.name))
|
||||
{
|
||||
out_rename_map["skp_idx_" + index.name + ".idx"] = "";
|
||||
out_rename_map["skp_idx_" + index.name + ".mrk"] = "";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes.
|
||||
std::map<String, size_t> stream_counts;
|
||||
|
@ -476,7 +476,7 @@ public:
|
||||
/// Check if the ALTER can be performed:
|
||||
/// - all needed columns are present.
|
||||
/// - all type conversions can be done.
|
||||
/// - columns corresponding to primary key, sign, sampling expression and date are not affected.
|
||||
/// - columns corresponding to primary key, indices, sign, sampling expression and date are not affected.
|
||||
/// If something is wrong, throws an exception.
|
||||
void checkAlter(const AlterCommands & commands);
|
||||
|
||||
|
@ -184,10 +184,19 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
|
||||
}
|
||||
|
||||
if (skip_indexes != from_zk.skip_indexes)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in skip indexes."
|
||||
" Stored in ZooKeeper: " + from_zk.skip_indexes +
|
||||
", local: " + skip_indexes,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
{
|
||||
if (allow_alter)
|
||||
{
|
||||
diff.skip_indices_changed = true;
|
||||
diff.new_skip_indices = from_zk.skip_indexes;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Existing table metadata in ZooKeeper differs in skip indexes."
|
||||
" Stored in ZooKeeper: " + from_zk.skip_indexes +
|
||||
", local: " + skip_indexes,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
|
||||
return diff;
|
||||
}
|
||||
|
@ -41,7 +41,10 @@ struct ReplicatedMergeTreeTableMetadata
|
||||
bool sorting_key_changed = false;
|
||||
String new_sorting_key;
|
||||
|
||||
bool empty() const { return !sorting_key_changed; }
|
||||
bool skip_indices_changed = false;
|
||||
String new_skip_indices;
|
||||
|
||||
bool empty() const { return !sorting_key_changed && !skip_indices_changed; }
|
||||
};
|
||||
|
||||
Diff checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const;
|
||||
|
@ -418,26 +418,36 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
{
|
||||
ASTPtr new_primary_key_ast = data.primary_key_ast;
|
||||
ASTPtr new_order_by_ast = data.order_by_ast;
|
||||
ASTPtr new_indices_ast = data.skip_indexes_ast;
|
||||
IDatabase::ASTModifier storage_modifier;
|
||||
if (!metadata_diff.empty())
|
||||
{
|
||||
ParserNotEmptyExpressionList parser(false);
|
||||
auto new_sorting_key_expr_list = parseQuery(parser, metadata_diff.new_sorting_key, 0);
|
||||
|
||||
if (new_sorting_key_expr_list->children.size() == 1)
|
||||
new_order_by_ast = new_sorting_key_expr_list->children[0];
|
||||
else
|
||||
if (metadata_diff.sorting_key_changed)
|
||||
{
|
||||
auto tuple = makeASTFunction("tuple");
|
||||
tuple->arguments->children = new_sorting_key_expr_list->children;
|
||||
new_order_by_ast = tuple;
|
||||
ParserNotEmptyExpressionList parser(false);
|
||||
auto new_sorting_key_expr_list = parseQuery(parser, metadata_diff.new_sorting_key, 0);
|
||||
|
||||
if (new_sorting_key_expr_list->children.size() == 1)
|
||||
new_order_by_ast = new_sorting_key_expr_list->children[0];
|
||||
else
|
||||
{
|
||||
auto tuple = makeASTFunction("tuple");
|
||||
tuple->arguments->children = new_sorting_key_expr_list->children;
|
||||
new_order_by_ast = tuple;
|
||||
}
|
||||
|
||||
if (!data.primary_key_ast)
|
||||
{
|
||||
/// Primary and sorting key become independent after this ALTER so we have to
|
||||
/// save the old ORDER BY expression as the new primary key.
|
||||
new_primary_key_ast = data.order_by_ast->clone();
|
||||
}
|
||||
}
|
||||
|
||||
if (!data.primary_key_ast)
|
||||
if (metadata_diff.skip_indices_changed)
|
||||
{
|
||||
/// Primary and sorting key become independent after this ALTER so we have to
|
||||
/// save the old ORDER BY expression as the new primary key.
|
||||
new_primary_key_ast = data.order_by_ast->clone();
|
||||
ParserIndexDeclaration parser;
|
||||
new_indices_ast = parseQuery(parser, metadata_diff.new_skip_indices, 0);
|
||||
}
|
||||
|
||||
storage_modifier = [&](IAST & ast)
|
||||
@ -453,6 +463,8 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
storage_ast.set(storage_ast.primary_key, new_primary_key_ast);
|
||||
|
||||
storage_ast.set(storage_ast.order_by, new_order_by_ast);
|
||||
|
||||
storage_ast.set(storage_ast.indexes, new_indices_ast);
|
||||
};
|
||||
}
|
||||
|
||||
@ -461,7 +473,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
/// Even if the primary/sorting keys didn't change we must reinitialize it
|
||||
/// because primary key column types might have changed.
|
||||
data.setPrimaryKeyAndColumns(new_order_by_ast, new_primary_key_ast, new_columns);
|
||||
data.setSkipIndexes(data.skip_indexes_ast);
|
||||
data.setSkipIndexes(new_indices_ast);
|
||||
}
|
||||
|
||||
|
||||
|
558
dbms/tests/queries/clickhouse-functions
Normal file
558
dbms/tests/queries/clickhouse-functions
Normal file
@ -0,0 +1,558 @@
|
||||
convertCharset
|
||||
transform
|
||||
findClusterValue
|
||||
findClusterIndex
|
||||
toNullable
|
||||
coalesce
|
||||
isNotNull
|
||||
pointInEllipses
|
||||
pow
|
||||
acos
|
||||
asin
|
||||
tan
|
||||
cos
|
||||
lgamma
|
||||
erfc
|
||||
erf
|
||||
sqrt
|
||||
log10
|
||||
exp10
|
||||
e
|
||||
visitParamExtractFloat
|
||||
visitParamExtractUInt
|
||||
decodeURLComponent
|
||||
cutURLParameter
|
||||
cutQueryStringAndFragment
|
||||
cutFragment
|
||||
cutWWW
|
||||
URLPathHierarchy
|
||||
URLHierarchy
|
||||
extractURLParameterNames
|
||||
extractURLParameter
|
||||
queryStringAndFragment
|
||||
pathFull
|
||||
sin
|
||||
topLevelDomain
|
||||
domainWithoutWWW
|
||||
domain
|
||||
protocol
|
||||
greatCircleDistance
|
||||
extract
|
||||
match
|
||||
positionCaseInsensitiveUTF8
|
||||
positionUTF8
|
||||
position
|
||||
replaceRegexpAll
|
||||
replaceRegexpOne
|
||||
arrayStringConcat
|
||||
splitByString
|
||||
alphaTokens
|
||||
tgamma
|
||||
base64Decode
|
||||
regexpQuoteMeta
|
||||
positionCaseInsensitive
|
||||
trimBoth
|
||||
trimRight
|
||||
endsWith
|
||||
startsWith
|
||||
appendTrailingCharIfAbsent
|
||||
substringUTF8
|
||||
concatAssumeInjective
|
||||
reverseUTF8
|
||||
upperUTF8
|
||||
upper
|
||||
lower
|
||||
length
|
||||
notEmpty
|
||||
trunc
|
||||
round
|
||||
reinterpretAsString
|
||||
reinterpretAsDateTime
|
||||
reinterpretAsDate
|
||||
reinterpretAsFloat64
|
||||
reinterpretAsFloat32
|
||||
reinterpretAsInt64
|
||||
reinterpretAsInt8
|
||||
reinterpretAsUInt32
|
||||
generateUUIDv4
|
||||
rand
|
||||
toISOYear
|
||||
toISOWeek
|
||||
concat
|
||||
toDecimal64
|
||||
ifNull
|
||||
toStartOfDay
|
||||
toSecond
|
||||
addSeconds
|
||||
sleepEachRow
|
||||
toDayOfWeek
|
||||
toDayOfMonth
|
||||
reinterpretAsUInt8
|
||||
UUIDNumToString
|
||||
bitShiftLeft
|
||||
toDate
|
||||
sleep
|
||||
emptyArrayUInt8
|
||||
parseDateTimeBestEffort
|
||||
toFloat32OrNull
|
||||
toInt16
|
||||
IPv6NumToString
|
||||
atan
|
||||
substring
|
||||
arrayIntersect
|
||||
isInfinite
|
||||
visitParamExtractString
|
||||
globalNotIn
|
||||
toDateTimeOrZero
|
||||
toRelativeHourNum
|
||||
toIntervalYear
|
||||
toFloat64OrZero
|
||||
dateDiff
|
||||
hex
|
||||
arrayEnumerateDense
|
||||
toUInt8OrZero
|
||||
toRelativeSecondNum
|
||||
toUInt64OrNull
|
||||
MACNumToString
|
||||
toInt32OrNull
|
||||
toDayOfYear
|
||||
toUnixTimestamp
|
||||
toString
|
||||
toDateOrZero
|
||||
subtractDays
|
||||
murmurHash2_32
|
||||
toUInt64
|
||||
toUInt8
|
||||
dictGetDateTime
|
||||
empty
|
||||
isFinite
|
||||
caseWithExpression
|
||||
caseWithoutExpression
|
||||
caseWithoutExpr
|
||||
visitParamExtractRaw
|
||||
queryString
|
||||
dictGetInt32OrDefault
|
||||
tryBase64Decode
|
||||
toInt8OrZero
|
||||
multiIf
|
||||
if
|
||||
intExp10
|
||||
bitShiftRight
|
||||
less
|
||||
roundToExp2
|
||||
toUInt8OrNull
|
||||
dictGetUInt16
|
||||
like
|
||||
regionToPopulation
|
||||
MACStringToOUI
|
||||
parseDateTimeBestEffortOrNull
|
||||
not
|
||||
toInt32OrZero
|
||||
arrayFilter
|
||||
toInt16OrZero
|
||||
range
|
||||
equals
|
||||
now
|
||||
toInt8OrNull
|
||||
bitmaskToArray
|
||||
roundAge
|
||||
toIntervalMonth
|
||||
toUUID
|
||||
notEquals
|
||||
toInt16OrNull
|
||||
murmurHash2_64
|
||||
hasAny
|
||||
joinGet
|
||||
toIntervalMinute
|
||||
notLike
|
||||
lcm
|
||||
toStringCutToZero
|
||||
isNull
|
||||
tupleElement
|
||||
replaceAll
|
||||
parseDateTimeBestEffortOrZero
|
||||
toFloat32OrZero
|
||||
lowerUTF8
|
||||
notIn
|
||||
gcd
|
||||
murmurHash3_64
|
||||
toMinute
|
||||
toDateTimeOrNull
|
||||
toTime
|
||||
materialize
|
||||
roundDuration
|
||||
gccMurmurHash
|
||||
ceil
|
||||
toStartOfQuarter
|
||||
dictGetInt8OrDefault
|
||||
MACStringToNum
|
||||
toTypeName
|
||||
toUInt32OrNull
|
||||
emptyArrayString
|
||||
dictGetDateTimeOrDefault
|
||||
bitRotateRight
|
||||
cutIPv6
|
||||
toUInt32OrZero
|
||||
timezone
|
||||
arrayPopFront
|
||||
toInt32
|
||||
intHash64
|
||||
extractURLParameters
|
||||
lowCardinalityIndices
|
||||
toStartOfMonth
|
||||
toYear
|
||||
hasAll
|
||||
rowNumberInAllBlocks
|
||||
bitTestAll
|
||||
arrayCount
|
||||
arraySort
|
||||
abs
|
||||
bitNot
|
||||
intDiv
|
||||
intDivOrZero
|
||||
firstSignificantSubdomain
|
||||
reinterpretAsUInt16
|
||||
dictGetFloat32OrDefault
|
||||
toHour
|
||||
minus
|
||||
regionToArea
|
||||
unhex
|
||||
IPv4StringToNum
|
||||
toIntervalHour
|
||||
toInt8
|
||||
dictGetFloat32
|
||||
log
|
||||
IPv4NumToString
|
||||
modulo
|
||||
arrayEnumerate
|
||||
reinterpretAsUInt64
|
||||
toMonth
|
||||
visitParamExtractBool
|
||||
emptyArrayUInt64
|
||||
replaceOne
|
||||
arrayReverseSort
|
||||
toFloat32
|
||||
trimLeft
|
||||
toRelativeMonthNum
|
||||
emptyArrayInt32
|
||||
randConstant
|
||||
CAST
|
||||
bitRotateLeft
|
||||
toRelativeYearNum
|
||||
negate
|
||||
toUInt64OrZero
|
||||
emptyArrayFloat64
|
||||
bitTest
|
||||
toDecimal128
|
||||
plus
|
||||
or
|
||||
cutQueryString
|
||||
reinterpretAsFixedString
|
||||
countEqual
|
||||
arrayPopBack
|
||||
arrayElement
|
||||
log2
|
||||
array
|
||||
arrayReverse
|
||||
arrayPushBack
|
||||
subtractWeeks
|
||||
bitTestAny
|
||||
bitAnd
|
||||
base64Encode
|
||||
toDecimal32
|
||||
arrayPushFront
|
||||
lessOrEquals
|
||||
intExp2
|
||||
toUInt16OrZero
|
||||
arrayConcat
|
||||
arrayCumSum
|
||||
arraySlice
|
||||
addDays
|
||||
dictGetUInt8
|
||||
toUInt32
|
||||
bitOr
|
||||
caseWithExpr
|
||||
UUIDStringToNum
|
||||
emptyArrayUInt16
|
||||
toIntervalDay
|
||||
MD5
|
||||
emptyArrayUInt32
|
||||
emptyArrayInt8
|
||||
toMonday
|
||||
addMonths
|
||||
CHAR_LENGTH
|
||||
least
|
||||
divide
|
||||
arrayUniq
|
||||
SHA256
|
||||
arrayExists
|
||||
multiply
|
||||
toUInt16OrNull
|
||||
dictGetInt8
|
||||
visitParamHas
|
||||
emptyArrayInt64
|
||||
toIntervalSecond
|
||||
emptyArrayToSingle
|
||||
path
|
||||
toInt64OrZero
|
||||
SHA1
|
||||
extractAll
|
||||
roundDown
|
||||
emptyArrayDate
|
||||
dumpColumnStructure
|
||||
lengthUTF8
|
||||
greatest
|
||||
arrayEnumerateUniq
|
||||
arrayDistinct
|
||||
javaHash
|
||||
arrayFirst
|
||||
toFixedString
|
||||
IPv4NumToStringClassC
|
||||
toFloat64OrNull
|
||||
IPv4ToIPv6
|
||||
identity
|
||||
toStartOfYear
|
||||
visitParamExtractInt
|
||||
toStartOfMinute
|
||||
toStartOfFiveMinute
|
||||
toStartOfFifteenMinutes
|
||||
toStartOfHour
|
||||
sumburConsistentHash
|
||||
toStartOfISOYear
|
||||
toRelativeQuarterNum
|
||||
toRelativeWeekNum
|
||||
toRelativeDayNum
|
||||
cbrt
|
||||
yesterday
|
||||
bitXor
|
||||
timeSlot
|
||||
timeSlots
|
||||
emptyArrayInt16
|
||||
dictGetInt16
|
||||
toYYYYMM
|
||||
toYYYYMMDDhhmmss
|
||||
toUInt16
|
||||
addMinutes
|
||||
addHours
|
||||
addWeeks
|
||||
nullIf
|
||||
subtractSeconds
|
||||
subtractMinutes
|
||||
toIntervalWeek
|
||||
subtractHours
|
||||
isNaN
|
||||
subtractMonths
|
||||
reverse
|
||||
runningDifferenceStartingWithFirstValue
|
||||
toDateTime
|
||||
subtractQuarters
|
||||
tuple
|
||||
arrayCumSumNonNegative
|
||||
rowNumberInBlock
|
||||
toDateOrNull
|
||||
subtractYears
|
||||
toTimeZone
|
||||
formatDateTime
|
||||
has
|
||||
cityHash64
|
||||
intHash32
|
||||
fragment
|
||||
regionToCity
|
||||
dictGetOrDefault
|
||||
indexOf
|
||||
regionToDistrict
|
||||
regionToCountry
|
||||
visibleWidth
|
||||
regionToContinent
|
||||
regionToTopContinent
|
||||
toColumnTypeName
|
||||
regionHierarchy
|
||||
dictGetDate
|
||||
dictHas
|
||||
dictGetUInt64
|
||||
cutToFirstSignificantSubdomain
|
||||
dictGetInt32
|
||||
pointInPolygon
|
||||
dictGetInt64
|
||||
blockNumber
|
||||
IPv6StringToNum
|
||||
dictGetString
|
||||
dictGetFloat64
|
||||
dictGetUUID
|
||||
CHARACTER_LENGTH
|
||||
toQuarter
|
||||
dictGetHierarchy
|
||||
toFloat64
|
||||
arraySum
|
||||
toInt64OrNull
|
||||
dictIsIn
|
||||
exp
|
||||
floor
|
||||
dictGetUInt8OrDefault
|
||||
dictGetUInt16OrDefault
|
||||
dictGetUInt32OrDefault
|
||||
emptyArrayDateTime
|
||||
greater
|
||||
jumpConsistentHash
|
||||
dictGetUInt64OrDefault
|
||||
dictGetInt16OrDefault
|
||||
dictGetInt64OrDefault
|
||||
reinterpretAsInt32
|
||||
dictGetUInt32
|
||||
murmurHash3_32
|
||||
rand64
|
||||
dictGetDateOrDefault
|
||||
bar
|
||||
dictGetUUIDOrDefault
|
||||
regionToName
|
||||
dictGetStringOrDefault
|
||||
splitByChar
|
||||
dictGet
|
||||
modelEvaluate
|
||||
arrayReduce
|
||||
farmHash64
|
||||
bitmaskToList
|
||||
formatReadableSize
|
||||
halfMD5
|
||||
SHA224
|
||||
arrayMap
|
||||
sipHash64
|
||||
dictGetFloat64OrDefault
|
||||
sipHash128
|
||||
metroHash64
|
||||
hiveHash
|
||||
murmurHash3_128
|
||||
toInt64
|
||||
xxHash32
|
||||
xxHash64
|
||||
yandexConsistentHash
|
||||
emptyArrayFloat32
|
||||
arrayAll
|
||||
toYYYYMMDD
|
||||
today
|
||||
arrayFirstIndex
|
||||
greaterOrEquals
|
||||
arrayDifference
|
||||
toIntervalQuarter
|
||||
throwIf
|
||||
and
|
||||
xor
|
||||
addQuarters
|
||||
currentDatabase
|
||||
hostName
|
||||
URLHash
|
||||
getSizeOfEnumType
|
||||
defaultValueOfArgumentType
|
||||
blockSize
|
||||
arrayResize
|
||||
ignore
|
||||
toRelativeMinuteNum
|
||||
indexHint
|
||||
reinterpretAsInt16
|
||||
addYears
|
||||
arrayJoin
|
||||
replicate
|
||||
hasColumnInTable
|
||||
version
|
||||
regionIn
|
||||
uptime
|
||||
runningAccumulate
|
||||
runningDifference
|
||||
assumeNotNull
|
||||
pi
|
||||
finalizeAggregation
|
||||
toLowCardinality
|
||||
exp2
|
||||
lowCardinalityKeys
|
||||
in
|
||||
globalIn
|
||||
power
|
||||
ln
|
||||
replace
|
||||
locate
|
||||
ceiling
|
||||
truncate
|
||||
lcase
|
||||
ucase
|
||||
substr
|
||||
mid
|
||||
retention
|
||||
maxIntersections
|
||||
groupBitXor
|
||||
groupBitOr
|
||||
uniqUpTo
|
||||
uniqExact
|
||||
uniq
|
||||
covarPop
|
||||
stddevPop
|
||||
varPop
|
||||
covarSamp
|
||||
varSamp
|
||||
sumMap
|
||||
corrStable
|
||||
corr
|
||||
quantileTiming
|
||||
quantileDeterministic
|
||||
quantilesExact
|
||||
uniqHLL12
|
||||
quantilesTiming
|
||||
covarPopStable
|
||||
stddevSampStable
|
||||
quantilesExactWeighted
|
||||
quantileExactWeighted
|
||||
quantileTimingWeighted
|
||||
quantileExact
|
||||
quantilesDeterministic
|
||||
quantiles
|
||||
topK
|
||||
sumWithOverflow
|
||||
count
|
||||
groupArray
|
||||
stddevSamp
|
||||
groupArrayInsertAt
|
||||
quantile
|
||||
quantilesTimingWeighted
|
||||
sum
|
||||
covarSampStable
|
||||
anyLast
|
||||
quantileTDigest
|
||||
quantilesTDigest
|
||||
windowFunnel
|
||||
min
|
||||
argMax
|
||||
varSampStable
|
||||
maxIntersectionsPosition
|
||||
quantilesTDigestWeighted
|
||||
groupUniqArray
|
||||
sequenceCount
|
||||
uniqCombined
|
||||
boundingRatio
|
||||
sumKahan
|
||||
any
|
||||
anyHeavy
|
||||
histogram
|
||||
quantileTDigestWeighted
|
||||
max
|
||||
groupBitAnd
|
||||
argMin
|
||||
varPopStable
|
||||
avg
|
||||
sequenceMatch
|
||||
stddevPopStable
|
||||
BIT_XOR
|
||||
medianExactWeighted
|
||||
medianTiming
|
||||
medianExact
|
||||
median
|
||||
medianDeterministic
|
||||
VAR_SAMP
|
||||
STDDEV_POP
|
||||
medianTDigest
|
||||
VAR_POP
|
||||
medianTDigestWeighted
|
||||
BIT_OR
|
||||
STDDEV_SAMP
|
||||
medianTimingWeighted
|
||||
COVAR_SAMP
|
||||
COVAR_POP
|
||||
BIT_AND
|
10
dbms/tests/queries/clickhouse-table_functions
Normal file
10
dbms/tests/queries/clickhouse-table_functions
Normal file
@ -0,0 +1,10 @@
|
||||
jdbc
|
||||
odbc
|
||||
hdfs
|
||||
remote
|
||||
catBoostPool
|
||||
merge
|
||||
file
|
||||
cluster
|
||||
url
|
||||
numbers
|
Loading…
Reference in New Issue
Block a user