diff --git a/docs/en/operations/system-tables/views.md b/docs/en/operations/system-tables/views.md new file mode 100644 index 00000000000..8edebf00a91 --- /dev/null +++ b/docs/en/operations/system-tables/views.md @@ -0,0 +1,44 @@ +# system.views {#system-views} + +Contains the dependencies of all views and the type to which the view belongs. The metadata of the view comes from the [system.tables](tables.md). + +Columns: + +- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the view is in. + +- `name` ([String](../../sql-reference/data-types/string.md)) — Name of the view. + +- `main_dependency_database` ([String](../../sql-reference/data-types/string.md)) — The name of the database on which the view depends. + +- `main_dependency_table` ([String](../../sql-reference/data-types/string.md)) - The name of the table on which the view depends. + +- `view_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Type of the view. Values: + - `'Default' = 1` — [Default views](../../sql-reference/statements/create/view.md#normal). Should not appear in this log. + - `'Materialized' = 2` — [Materialized views](../../sql-reference/statements/create/view.md#materialized). + - `'Live' = 3` — [Live views](../../sql-reference/statements/create/view.md#live-view). + +**Example** + +```sql +SELECT * FROM system.views LIMIT 2 FORMAT Vertical; +``` + +```text +Row 1: +────── +database: default +name: live_view +main_dependency_database: default +main_dependency_table: view_source_tb +view_type: Live + +Row 2: +────── +database: default +name: materialized_view +main_dependency_database: default +main_dependency_table: view_source_tb +view_type: Materialized +``` + +[Original article](https://clickhouse.tech/docs/en/operations/system-tables/views) diff --git a/src/Common/Fiber.h b/src/Common/Fiber.h index 10310b57678..e3ad8af12cd 100644 --- a/src/Common/Fiber.h +++ b/src/Common/Fiber.h @@ -1,5 +1,7 @@ #pragma once - +/// defines.h should be included before fiber.hpp +/// BOOST_USE_ASAN, BOOST_USE_TSAN and BOOST_USE_UCONTEXT should be correctly defined for sanitizers. +#include #include using Fiber = boost::context::fiber; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 9b3c3191b5d..220b52104b5 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -130,8 +130,27 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ throw DB::Exception("Unknown implementation of coordination service: " + implementation, DB::ErrorCodes::NOT_IMPLEMENTED); } - if (!chroot.empty() && !exists("/")) - throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start.", Coordination::Error::ZNONODE); + if (!chroot.empty()) + { + /// Here we check that zk root exists. + /// This check is clumsy. The reason is we do this request under common mutex, and never want to hung here. + /// Otherwise, all threads which need zk will wait for this mutex eternally. + /// + /// Usually, this was possible in case of memory limit exception happened inside zk implementation. + /// This should not happen now, when memory tracker is disabled. + /// But let's keep it just in case (it is also easy to backport). + auto future = asyncExists("/"); + auto res = future.wait_for(std::chrono::milliseconds(operation_timeout_ms)); + if (res != std::future_status::ready) + throw KeeperException("Cannot check if zookeeper root exists.", Coordination::Error::ZOPERATIONTIMEOUT); + + auto code = future.get().error; + if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) + throw KeeperException(code, "/"); + + if (code == Coordination::Error::ZNONODE) + throw KeeperException("ZooKeeper root doesn't exist. You should create root node " + chroot + " before start.", Coordination::Error::ZNONODE); + } } ZooKeeper::ZooKeeper(const std::string & hosts_string, const std::string & identity_, int32_t session_timeout_ms_, diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 1f780a206dd..c5684b797bf 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -632,19 +632,23 @@ ColumnPtr ConstantExpressionTemplate::evaluateAll(BlockMissingValues & nulls, si void ConstantExpressionTemplate::TemplateStructure::addNodesToCastResult(const IDataType & result_column_type, ASTPtr & expr, bool null_as_default) { /// Replace "expr" with "CAST(expr, 'TypeName')" - /// or with "(CAST(assumeNotNull(expr as _expression), 'TypeName'), isNull(_expression))" if null_as_default is true + /// or with "(if(isNull(_dummy_0 AS _expression), defaultValueOfTypeName('TypeName'), _CAST(_expression, 'TypeName')), isNull(_expression))" if null_as_default is true if (null_as_default) { expr->setAlias("_expression"); - expr = makeASTFunction("assumeNotNull", std::move(expr)); - } - expr = makeASTFunction("_CAST", std::move(expr), std::make_shared(result_column_type.getName())); - - if (null_as_default) - { auto is_null = makeASTFunction("isNull", std::make_shared("_expression")); - expr = makeASTFunction("tuple", std::move(expr), std::move(is_null)); + is_null->setAlias("_is_expression_nullable"); + + auto default_value = makeASTFunction("defaultValueOfTypeName", std::make_shared(result_column_type.getName())); + auto cast = makeASTFunction("_CAST", std::move(expr), std::make_shared(result_column_type.getName())); + + auto cond = makeASTFunction("if", std::move(is_null), std::move(default_value), std::move(cast)); + expr = makeASTFunction("tuple", std::move(cond), std::make_shared("_is_expression_nullable")); + } + else + { + expr = makeASTFunction("_CAST", std::move(expr), std::make_shared(result_column_type.getName())); } } diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 7d9906ae663..040d49b992f 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -240,15 +240,27 @@ class _NetworkManager: # Approximately mesure network I/O speed for interface class NetThroughput(object): - def __init__(self, node, interface="eth0"): - self.interface = interface + def __init__(self, node): self.node = node + # trying to get default interface and check it in /proc/net/dev + self.interface = self.node.exec_in_container(["bash", "-c", "awk '{print $1 \" \" $2}' /proc/net/route | grep 00000000 | awk '{print $1}'"]).strip() + check = self.node.exec_in_container(["bash", "-c", f'grep "^ *{self.interface}:" /proc/net/dev']).strip() + if not check: # if check is not successful just try eth{1-10} + for i in range(10): + try: + self.interface = self.node.exec_in_container(["bash", "-c", f"awk '{{print $1}}' /proc/net/route | grep 'eth{i}'"]).strip() + break + except Exception as ex: + print(f"No interface eth{i}") + else: + raise Exception("No interface eth{1-10} and default interface not specified in /proc/net/route, maybe some special network configuration") + try: - check = subprocess.check_output(f'grep "^ *{self.interface}:" /proc/net/dev', shell=True) + check = self.node.exec_in_container(["bash", "-c", f'grep "^ *{self.interface}:" /proc/net/dev']).strip() if not check: raise Exception(f"No such interface {self.interface} found in /proc/net/dev") except: - logging.error("All available interfaces %s", subprocess.check_output("cat /proc/net/dev", shell=True)) + logging.error("All available interfaces %s", self.node.exec_in_container(["bash", "-c", "cat /proc/net/dev"])) raise Exception(f"No such interface {self.interface} found in /proc/net/dev") self.current_in = self._get_in_bytes() diff --git a/tests/queries/0_stateless/02013_emptystring_cast.reference b/tests/queries/0_stateless/02013_emptystring_cast.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02013_emptystring_cast.sql b/tests/queries/0_stateless/02013_emptystring_cast.sql new file mode 100644 index 00000000000..b91c523aeb2 --- /dev/null +++ b/tests/queries/0_stateless/02013_emptystring_cast.sql @@ -0,0 +1,19 @@ +drop table if exists test_uint64; +create table test_uint64 (`data` UInt64 Default 0) engine = MergeTree order by tuple(); +insert into test_uint64 values ('0'), (NULL), (1), ('2'); +drop table if exists test_uint64; + +drop table if exists test_float64; +create table test_float64 (`data` Float64 Default 0.0) engine = MergeTree order by tuple(); +insert into test_float64 values ('0.1'), (NULL), (1.1), ('2.2'); +drop table if exists test_float64; + +drop table if exists test_date; +create table test_date (`data` Date) engine = MergeTree order by tuple(); +insert into test_date values ('2021-01-01'), (NULL), ('2021-02-01'), ('2021-03-01'); +drop table if exists test_date; + +drop table if exists test_datetime; +create table test_datetime (`data` DateTime) engine = MergeTree order by tuple(); +insert into test_datetime values ('2021-01-01 00:00:00'), (NULL), ('2021-02-01 01:00:00'), ('2021-03-01 02:00:00'); +drop table if exists test_datetime;