Merge branch 'master' into fix25

This commit is contained in:
alexey-milovidov 2019-08-04 03:16:10 +03:00 committed by GitHub
commit 65699d0bc5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 397 additions and 18 deletions

View File

@ -194,10 +194,13 @@ endif ()
option(WITH_COVERAGE "Build with coverage." 0)
if(WITH_COVERAGE AND COMPILER_CLANG)
set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-instr-generate -fcoverage-mapping")
# If we want to disable coverage for specific translation units
set(WITHOUT_COVERAGE "-fno-profile-instr-generate -fno-coverage-mapping")
endif()
if(WITH_COVERAGE AND COMPILER_GCC)
set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-arcs -ftest-coverage")
set(COVERAGE_OPTION "-lgcov")
set(WITHOUT_COVERAGE "-fno-profile-arcs -fno-test-coverage")
endif()
set (CMAKE_BUILD_COLOR_MAKEFILE ON)

View File

@ -201,6 +201,13 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELW
PROPERTIES COMPILE_FLAGS -g0)
endif ()
# Otherwise it will slow down stack traces printing too much.
set_source_files_properties(
src/Common/Elf.cpp
src/Common/Dwarf.cpp
src/Common/SymbolIndex.cpp
PROPERTIES COMPILE_FLAGS "-O3 ${WITHOUT_COVERAGE}")
target_link_libraries (clickhouse_common_io
PUBLIC
common

View File

@ -15,6 +15,7 @@
#include <ext/scope_guard.h>
#include <common/logger_useful.h>
#include <common/phdr_cache.h>
#include <common/config_common.h>
#include <common/ErrorHandlers.h>
#include <common/getMemoryAmount.h>
#include <Common/ClickHouseRevision.h>
@ -510,8 +511,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
LOG_DEBUG(log, "Loaded metadata.");
/// Init trace collector only after trace_log system table was created
/// Disable it if we collect test coverage information, because it will work extremely slow.
#if USE_INTERNAL_UNWIND_LIBRARY && !WITH_COVERAGE
/// QueryProfiler cannot work reliably with any other libunwind or without PHDR cache.
if (hasPHDRCache())
global_context->initializeTraceCollector();
#endif
global_context->setCurrentDatabase(default_database);

View File

@ -190,6 +190,7 @@ QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const Int32 thread_id, const
UNUSED(clock_type);
UNUSED(period);
UNUSED(pause_signal);
throw Exception("QueryProfiler cannot work with stock libunwind", ErrorCodes::NOT_IMPLEMENTED);
#endif
}

View File

@ -1,12 +1,16 @@
#include <DataStreams/AddingDefaultBlockOutputStream.h>
#include <DataStreams/PushingToViewsBlockOutputStream.h>
#include <DataStreams/SquashingBlockInputStream.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Parsers/ASTInsertQuery.h>
#include <Common/CurrentThread.h>
#include <Common/setThreadName.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/ThreadPool.h>
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
#include <Storages/StorageBlockInserted.h>
namespace DB
{
@ -44,13 +48,15 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
auto dependent_table = context.getTable(database_table.first, database_table.second);
auto & materialized_view = dynamic_cast<const StorageMaterializedView &>(*dependent_table);
if (StoragePtr inner_table = materialized_view.tryGetTargetTable())
addTableLock(inner_table->lockStructureForShare(true, context.getCurrentQueryId()));
StoragePtr inner_table = materialized_view.getTargetTable();
auto query = materialized_view.getInnerQuery();
BlockOutputStreamPtr out = std::make_shared<PushingToViewsBlockOutputStream>(
database_table.first, database_table.second, dependent_table, *views_context, ASTPtr());
views.emplace_back(ViewInfo{std::move(query), database_table.first, database_table.second, std::move(out)});
std::unique_ptr<ASTInsertQuery> insert = std::make_unique<ASTInsertQuery>();
insert->database = inner_table->getDatabaseName();
insert->table = inner_table->getTableName();
ASTPtr insert_query_ptr(insert.release());
InterpreterInsertQuery interpreter(insert_query_ptr, *views_context);
BlockIO io = interpreter.execute();
views.emplace_back(ViewInfo{query, database_table.first, database_table.second, io.out});
}
}
@ -173,8 +179,13 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n
try
{
BlockInputStreamPtr from = std::make_shared<OneBlockInputStream>(block);
InterpreterSelectQuery select(view.query, *views_context, from);
/// We create a table with the same name as original table and the same alias columns,
/// but it will contain single block (that is INSERT-ed into main table).
/// InterpreterSelectQuery will do processing of alias columns.
Context local_context = *views_context;
local_context.addViewSource(StorageBlockInserted::create(block, storage));
InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions());
BlockInputStreamPtr in = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
/// Squashing is needed here because the materialized view query can generate a lot of blocks
/// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY

View File

@ -0,0 +1,35 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionBinaryArithmetic.h>
#include <DataTypes/NumberTraits.h>
namespace DB
{
template <typename A, typename B>
struct BitBoolMaskAndImpl
{
using ResultType = UInt8;
template <typename Result = ResultType>
static inline Result apply(A left, B right)
{
return static_cast<ResultType>(
((static_cast<ResultType>(left) & static_cast<ResultType>(right)) & 1)
| ((((static_cast<ResultType>(left) >> 1) | (static_cast<ResultType>(right) >> 1)) & 1) << 1));
}
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = false;
#endif
};
struct NameBitBoolMaskAnd { static constexpr auto name = "__bitBoolMaskAnd"; };
using FunctionBitBoolMaskAnd = FunctionBinaryArithmetic<BitBoolMaskAndImpl, NameBitBoolMaskAnd>;
void registerFunctionBitBoolMaskAnd(FunctionFactory & factory)
{
factory.registerFunction<FunctionBitBoolMaskAnd>();
}
}

View File

@ -0,0 +1,35 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionBinaryArithmetic.h>
#include <DataTypes/NumberTraits.h>
namespace DB
{
template <typename A, typename B>
struct BitBoolMaskOrImpl
{
using ResultType = UInt8;
template <typename Result = ResultType>
static inline Result apply(A left, B right)
{
return static_cast<ResultType>(
((static_cast<ResultType>(left) & 1) | (static_cast<ResultType>(right) & 1))
| ((((static_cast<ResultType>(left) >> 1) & (static_cast<ResultType>(right) >> 1)) & 1) << 1));
}
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = false;
#endif
};
struct NameBitBoolMaskOr { static constexpr auto name = "__bitBoolMaskOr"; };
using FunctionBitBoolMaskOr = FunctionBinaryArithmetic<BitBoolMaskOrImpl, NameBitBoolMaskOr>;
void registerFunctionBitBoolMaskOr(FunctionFactory & factory)
{
factory.registerFunction<FunctionBitBoolMaskOr>();
}
}

View File

@ -0,0 +1,41 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionUnaryArithmetic.h>
#include <DataTypes/NumberTraits.h>
namespace DB
{
template <typename A>
struct BitWrapperFuncImpl
{
using ResultType = UInt8;
static inline ResultType NO_SANITIZE_UNDEFINED apply(A a)
{
return a == static_cast<UInt8>(0) ? static_cast<ResultType>(0b10) : static_cast<ResultType >(0b1);
}
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = false;
#endif
};
struct NameBitWrapperFunc { static constexpr auto name = "__bitWrapperFunc"; };
using FunctionBitWrapperFunc = FunctionUnaryArithmetic<BitWrapperFuncImpl, NameBitWrapperFunc, true>;
template <> struct FunctionUnaryArithmeticMonotonicity<NameBitWrapperFunc>
{
static bool has() { return false; }
static IFunction::Monotonicity get(const Field &, const Field &)
{
return {};
}
};
void registerFunctionBitWrapperFunc(FunctionFactory & factory)
{
factory.registerFunction<FunctionBitWrapperFunc>();
}
}

View File

@ -33,6 +33,9 @@ void registerFunctionRoundToExp2(FunctionFactory & factory);
void registerFunctionRoundDuration(FunctionFactory & factory);
void registerFunctionRoundAge(FunctionFactory & factory);
void registerFunctionBitBoolMaskOr(FunctionFactory & factory);
void registerFunctionBitBoolMaskAnd(FunctionFactory & factory);
void registerFunctionBitWrapperFunc(FunctionFactory & factory);
void registerFunctionBitSwapLastTwo(FunctionFactory & factory);
void registerFunctionsArithmetic(FunctionFactory & factory)
@ -68,6 +71,9 @@ void registerFunctionsArithmetic(FunctionFactory & factory)
registerFunctionRoundAge(factory);
/// Not for external use.
registerFunctionBitBoolMaskOr(factory);
registerFunctionBitBoolMaskAnd(factory);
registerFunctionBitWrapperFunc(factory);
registerFunctionBitSwapLastTwo(factory);
}

View File

@ -236,6 +236,9 @@ void Compiler::compile(
" -fuse-ld=" << compiler_executable_root << INTERNAL_LINKER_EXECUTABLE
" -fdiagnostics-color=never"
/// Do not use libgcc and startup files. The library will work nevertheless and we avoid extra dependency.
" -nodefaultlibs -nostartfiles"
#if INTERNAL_COMPILER_CUSTOM_ROOT
/// To get correct order merge this results carefully:
/// echo | clang -x c++ -E -Wp,-v -

View File

@ -987,6 +987,21 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
}
void Context::addViewSource(const StoragePtr & storage)
{
if (view_source)
throw Exception(
"Temporary view source storage " + backQuoteIfNeed(view_source->getName()) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
view_source = storage;
}
StoragePtr Context::getViewSource()
{
return view_source;
}
DDLGuard::DDLGuard(Map & map_, std::unique_lock<std::mutex> guards_lock_, const String & elem)
: map(map_), guards_lock(std::move(guards_lock_))
{

View File

@ -133,6 +133,7 @@ private:
String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification.
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
TableAndCreateASTs external_tables; /// Temporary tables.
StoragePtr view_source; /// Temporary StorageBlockInserted used to generate alias columns for materialized views
Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id.
Context * query_context = nullptr;
Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this.
@ -245,6 +246,9 @@ public:
StoragePtr executeTableFunction(const ASTPtr & table_expression);
void addViewSource(const StoragePtr & storage);
StoragePtr getViewSource();
void addDatabase(const String & database_name, const DatabasePtr & database);
DatabasePtr detachDatabase(const String & database_name);

View File

@ -45,9 +45,12 @@
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
#include <Interpreters/CrossToInnerJoinVisitor.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/StorageBlockInserted.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
@ -264,13 +267,26 @@ InterpreterSelectQuery::InterpreterSelectQuery(
}
else
{
/// Read from table. Even without table expression (implicit SELECT ... FROM system.one).
String database_name;
String table_name;
getDatabaseAndTableNames(database_name, table_name);
storage = context.getTable(database_name, table_name);
if (auto view_source = context.getViewSource())
{
auto & storage_block = static_cast<const StorageBlockInserted &>(*view_source);
if (storage_block.getDatabaseName() == database_name && storage_block.getTableName() == table_name)
{
/// Read from view source.
storage = context.getViewSource();
}
}
if (!storage)
{
/// Read from table. Even without table expression (implicit SELECT ... FROM system.one).
storage = context.getTable(database_name, table_name);
}
}
}

View File

@ -305,7 +305,12 @@ void MergeTreeIndexConditionSet::traverseAST(ASTPtr & node) const
return;
}
if (!atomFromAST(node))
if (atomFromAST(node))
{
if (node->as<ASTIdentifier>() || node->as<ASTFunction>())
node = makeASTFunction("__bitWrapperFunc", node);
}
else
node = std::make_shared<ASTLiteral>(UNKNOWN_FIELD);
}
@ -364,12 +369,12 @@ bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node) const
ASTPtr new_func;
if (args.size() > 1)
new_func = makeASTFunction(
"bitAnd",
"__bitBoolMaskAnd",
node,
last_arg);
else
new_func = makeASTFunction(
"bitAnd",
"__bitBoolMaskAnd",
args.back(),
last_arg);
@ -383,12 +388,12 @@ bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node) const
ASTPtr new_func;
if (args.size() > 1)
new_func = makeASTFunction(
"bitOr",
"__bitBoolMaskOr",
node,
last_arg);
else
new_func = makeASTFunction(
"bitOr",
"__bitBoolMaskOr",
args.back(),
last_arg);

View File

@ -0,0 +1,59 @@
#pragma once
#include <ext/shared_ptr_helper.h>
#include <Core/NamesAndTypes.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Storages/IStorage.h>
namespace DB
{
/// The table has all the properties of another storage,
/// but will read single prepared block instead.
/// Used in PushingToViewsBlockOutputStream for generating alias columns
/// NOTE: Some of the properties seems redundant.
class StorageBlock : public ext::shared_ptr_helper<StorageBlock>, public IStorage
{
public:
std::string getName() const override { return storage->getName(); }
std::string getTableName() const override { return storage->getTableName(); }
std::string getDatabaseName() const override { return storage->getDatabaseName(); }
bool isRemote() const override { return storage->isRemote(); }
bool supportsSampling() const override { return storage->supportsSampling(); }
bool supportsFinal() const override { return storage->supportsFinal(); }
bool supportsPrewhere() const override { return storage->supportsPrewhere(); }
bool supportsReplication() const override { return storage->supportsReplication(); }
bool supportsDeduplication() const override { return storage->supportsDeduplication(); }
bool supportsIndexForIn() const override { return storage->supportsIndexForIn(); }
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override
{
return storage->mayBenefitFromIndexForIn(left_in_operand, query_context);
}
ASTPtr getPartitionKeyAST() const override { return storage->getPartitionKeyAST(); }
ASTPtr getSortingKeyAST() const override { return storage->getSortingKeyAST(); }
ASTPtr getPrimaryKeyAST() const override { return storage->getPrimaryKeyAST(); }
ASTPtr getSamplingKeyAST() const override { return storage->getSamplingKeyAST(); }
Names getColumnsRequiredForPartitionKey() const override { return storage->getColumnsRequiredForPartitionKey(); }
Names getColumnsRequiredForSortingKey() const override { return storage->getColumnsRequiredForSortingKey(); }
Names getColumnsRequiredForPrimaryKey() const override { return storage->getColumnsRequiredForPrimaryKey(); }
Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); }
Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); }
BlockInputStreams read(const Names &, const SelectQueryInfo &, const Context &, QueryProcessingStage::Enum, size_t, unsigned) override
{
return {std::make_shared<OneBlockInputStream>(std::move(block))};
}
private:
Block block;
StoragePtr storage;
protected:
StorageBlock(Block block_, StoragePtr storage_)
: IStorage{storage_->getColumns()}, block(std::move(block_)), storage(storage_)
{
}
};
}

View File

@ -0,0 +1,63 @@
#pragma once
#include <ext/shared_ptr_helper.h>
#include <Core/NamesAndTypes.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Storages/IStorage.h>
namespace DB
{
/// Allows to look at the Block that was inserted or is going to be inserted
/// to a table as if we read this block from that table.
/// The table has all the properties of another storage,
/// but will read single prepared block instead.
/// Used in PushingToViewsBlockOutputStream for generating alias columns
/// NOTE: Some of the properties seems redundant.
class StorageBlockInserted : public ext::shared_ptr_helper<StorageBlockInserted>, public IStorage
{
public:
std::string getName() const override { return storage->getName(); }
std::string getTableName() const override { return storage->getTableName(); }
std::string getDatabaseName() const override { return storage->getDatabaseName(); }
bool isRemote() const override { return storage->isRemote(); }
bool supportsSampling() const override { return storage->supportsSampling(); }
bool supportsFinal() const override { return storage->supportsFinal(); }
bool supportsPrewhere() const override { return storage->supportsPrewhere(); }
bool supportsReplication() const override { return storage->supportsReplication(); }
bool supportsDeduplication() const override { return storage->supportsDeduplication(); }
bool supportsIndexForIn() const override { return storage->supportsIndexForIn(); }
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override
{
return storage->mayBenefitFromIndexForIn(left_in_operand, query_context);
}
ASTPtr getPartitionKeyAST() const override { return storage->getPartitionKeyAST(); }
ASTPtr getSortingKeyAST() const override { return storage->getSortingKeyAST(); }
ASTPtr getPrimaryKeyAST() const override { return storage->getPrimaryKeyAST(); }
ASTPtr getSamplingKeyAST() const override { return storage->getSamplingKeyAST(); }
Names getColumnsRequiredForPartitionKey() const override { return storage->getColumnsRequiredForPartitionKey(); }
Names getColumnsRequiredForSortingKey() const override { return storage->getColumnsRequiredForSortingKey(); }
Names getColumnsRequiredForPrimaryKey() const override { return storage->getColumnsRequiredForPrimaryKey(); }
Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); }
Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); }
BlockInputStreams read(const Names &, const SelectQueryInfo &, const Context &, QueryProcessingStage::Enum, size_t, unsigned) override
{
return {std::make_shared<OneBlockInputStream>(std::move(block))};
}
private:
Block block;
StoragePtr storage;
protected:
StorageBlockInserted(Block block_, StoragePtr storage_)
: IStorage{storage_->getColumns()}, block(std::move(block_)), storage(storage_)
{
}
};
}

View File

@ -26,7 +26,7 @@
<values>
<value>2</value>
<value>500</value>
<value>65535</value>
<value>32768</value>
</values>
</substitution>
</substitutions>

View File

@ -0,0 +1,25 @@
USE test;
DROP TABLE IF EXISTS test1;
CREATE TABLE test1 (
date Date,
datetime DateTime,
zoneId UInt64,
test ALIAS zoneId == 1
) ENGINE = MergeTree(date, (date, zoneId), 8192);
CREATE MATERIALIZED VIEW test1_view
ENGINE = MergeTree(date, (date, zoneId), 8192)
AS SELECT
date,
zoneId,
test
FROM test.test1;
INSERT INTO test1 VALUES ('2018-12-10', '2018-12-10 23:59:59', 1);
SELECT * from test1_view;
DROP TABLE test1_view;
DROP TABLE test1;

View File

@ -3,4 +3,22 @@
12 5 4.7 6.50 cba b 2014-06-11
13 5 4.7 6.50 cba b 2015-01-01
"rows_read": 4,
2 2 4.5 2.50 abc a 2014-01-01
6 2 4.5 2.50 abc a 2014-02-11
7 5 6.9 1.57 bac c 2014-04-11
8 2 4.5 2.50 abc a 2014-05-11
9 5 6.9 1.57 bac c 2014-07-11
5 5 6.9 1.57 bac c 2014-11-11
4 2 4.5 2.50 abc a 2016-01-01
3 5 6.9 1.57 bac c 2017-01-01
"rows_read": 8,
"rows_read": 2,
2 2 4.5 2.50 abc a 2014-01-01
6 2 4.5 2.50 abc a 2014-02-11
7 5 6.9 1.57 bac c 2014-04-11
8 2 4.5 2.50 abc a 2014-05-11
9 5 6.9 1.57 bac c 2014-07-11
5 5 6.9 1.57 bac c 2014-11-11
4 2 4.5 2.50 abc a 2016-01-01
3 5 6.9 1.57 bac c 2017-01-01
"rows_read": 8,

View File

@ -43,8 +43,15 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO set_idx VALUES
$CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE NOT (i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba')) ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE NOT (i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba')) ORDER BY dt FORMAT JSON" | grep "rows_read"
# select with hole made by primary key
$CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE (u64 < 2 OR u64 > 10) AND s != 'cba' ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE (u64 < 2 OR u64 > 10) AND s != 'cba' ORDER BY dt FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="DROP TABLE set_idx;"
$CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE (u64 < 2 OR NOT u64 > 10) AND NOT (s = 'cba') ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE (u64 < 2 OR NOT u64 > 10) AND NOT (s = 'cba') ORDER BY dt FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="DROP TABLE set_idx;"

View File

@ -0,0 +1,2 @@
Jon alive
Jon alive

View File

@ -0,0 +1,16 @@
SET allow_experimental_data_skipping_indices = 1;
DROP TABLE IF EXISTS test.set_index_not;
CREATE TABLE test.set_index_not
( name String, status Enum8('alive' = 0, 'rip' = 1),
INDEX idx_status status TYPE set(2) GRANULARITY 1
)
ENGINE = MergeTree() ORDER BY name SETTINGS index_granularity = 8192;
insert into test.set_index_not values ('Jon','alive'),('Ramsey','rip');
select * from test.set_index_not where status!='rip';
select * from test.set_index_not where NOT (status ='rip');
DROP TABLE test.set_index_not;

View File

@ -9,3 +9,4 @@
#cmakedefine01 HAVE_READLINE_HISTORY
#cmakedefine01 UNBUNDLED
#cmakedefine01 USE_INTERNAL_UNWIND_LIBRARY
#cmakedefine01 WITH_COVERAGE