diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index d3d7084f37f..b0b5ebdb2e2 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -61,7 +61,7 @@ function configure cp -rv right/config left ||: # Start a temporary server to rename the tables - while pkill clickhouse-serv; do echo . ; sleep 1 ; done + while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done echo all killed set -m # Spawn temporary in its own process groups @@ -88,7 +88,7 @@ function configure clickhouse-client --port $LEFT_SERVER_PORT --query "create database test" ||: clickhouse-client --port $LEFT_SERVER_PORT --query "rename table datasets.hits_v1 to test.hits" ||: - while pkill clickhouse-serv; do echo . ; sleep 1 ; done + while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done echo all killed # Make copies of the original db for both servers. Use hardlinks instead @@ -106,7 +106,7 @@ function configure function restart { - while pkill clickhouse-serv; do echo . ; sleep 1 ; done + while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done echo all killed # Change the jemalloc settings here. @@ -1400,7 +1400,7 @@ case "$stage" in while env kill -- -$watchdog_pid ; do sleep 1; done # Stop the servers to free memory for the subsequent query analysis. - while pkill clickhouse-serv; do echo . ; sleep 1 ; done + while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done echo Servers stopped. ;& "analyze_queries") diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 194778400d3..663469ef4ae 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -134,6 +134,13 @@ Example of configuration for versions later or equal to 22.8: 10000000 + + +
+ cache +
+
+ ``` @@ -151,6 +158,13 @@ Example of configuration for versions earlier than 22.8: 10000000 + + +
+ s3 +
+
+ ``` diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index a110bfbd15c..f40107aaaca 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -13,7 +13,7 @@ Creates a table from a file. This table function is similar to [url](../../sql-r **Syntax** ``` sql -file(path, format, structure) +file(path [,format] [,structure]) ``` **Parameters** diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 2df7d6e46b3..545037665bb 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -11,7 +11,7 @@ Provides table-like interface to select/insert files in [Amazon S3](https://aws. **Syntax** ``` sql -s3(path, [aws_access_key_id, aws_secret_access_key,] format, structure, [compression]) +s3(path [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) ``` **Arguments** diff --git a/docs/en/sql-reference/table-functions/s3Cluster.md b/docs/en/sql-reference/table-functions/s3Cluster.md index 9d006af9572..b81fc51fd18 100644 --- a/docs/en/sql-reference/table-functions/s3Cluster.md +++ b/docs/en/sql-reference/table-functions/s3Cluster.md @@ -10,7 +10,7 @@ Allows processing files from [Amazon S3](https://aws.amazon.com/s3/) in parallel **Syntax** ``` sql -s3Cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure) +s3Cluster(cluster_name, source, [,access_key_id, secret_access_key] [,format] [,structure]) ``` **Arguments** diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index f1ed7b4dfe4..014dc3ae853 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -13,7 +13,7 @@ sidebar_label: url **Syntax** ``` sql -url(URL, format, structure) +url(URL [,format] [,structure]) ``` **Parameters** diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 1f262c9403a..df35a1c4ac0 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -13,7 +13,7 @@ sidebar_label: file **Синтаксис** ``` sql -file(path, format, structure) +file(path [,format] [,structure]) ``` **Параметры** diff --git a/docs/ru/sql-reference/table-functions/s3.md b/docs/ru/sql-reference/table-functions/s3.md index ae0419a4b84..14c8204fd1d 100644 --- a/docs/ru/sql-reference/table-functions/s3.md +++ b/docs/ru/sql-reference/table-functions/s3.md @@ -11,7 +11,7 @@ sidebar_label: s3 **Синтаксис** ``` sql -s3(path, [aws_access_key_id, aws_secret_access_key,] format, structure, [compression]) +s3(path [,aws_access_key_id, aws_secret_access_key] [,format] [,structure] [,compression]) ``` **Aргументы** diff --git a/docs/ru/sql-reference/table-functions/s3Cluster.md b/docs/ru/sql-reference/table-functions/s3Cluster.md index e6b317253c0..1c12913fabe 100644 --- a/docs/ru/sql-reference/table-functions/s3Cluster.md +++ b/docs/ru/sql-reference/table-functions/s3Cluster.md @@ -11,7 +11,7 @@ sidebar_label: s3Cluster **Синтаксис** ``` sql -s3Cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure) +s3Cluster(cluster_name, source, [,access_key_id, secret_access_key] [,format] [,structure]) ``` **Аргументы** diff --git a/docs/ru/sql-reference/table-functions/url.md b/docs/ru/sql-reference/table-functions/url.md index d4fb11b0de7..e5d9faeec00 100644 --- a/docs/ru/sql-reference/table-functions/url.md +++ b/docs/ru/sql-reference/table-functions/url.md @@ -13,7 +13,7 @@ sidebar_label: url **Синтаксис** ``` sql -url(URL, format, structure) +url(URL [,format] [,structure]) ``` **Параметры** diff --git a/src/Core/Settings.h b/src/Core/Settings.h index da420079766..8793bbb3011 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -481,7 +481,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_multiif_to_if, true, "Replace 'multiIf' with only one condition to 'if'.", 0) \ M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \ - M(Bool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \ + M(Bool, optimize_monotonous_functions_in_order_by, false, "Replace monotonous function with its argument in ORDER BY", 0) \ M(Bool, optimize_functions_to_subcolumns, false, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ M(Bool, optimize_using_constraints, false, "Use constraints for query optimization", 0) \ M(Bool, optimize_substitute_columns, false, "Use constraints for column substitution", 0) \ diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index d97b669ab10..da7f8c871cb 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -143,9 +143,11 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) } CachedOnDiskReadBufferFromFile::ImplementationBufferPtr -CachedOnDiskReadBufferFromFile::getCacheReadBuffer(size_t offset) const +CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segment) const { - auto path = cache->getPathInLocalCache(cache_key, offset, is_persistent); + /// Use is_persistent flag from in-memory state of the filesegment, + /// because it is consistent with what is written on disk. + auto path = file_segment.getPathInLocalCache(); ReadSettings local_read_settings{settings}; /// Do not allow to use asynchronous version of LocalFSReadMethod. @@ -237,8 +239,6 @@ bool CachedOnDiskReadBufferFromFile::canStartFromCache(size_t current_offset, co CachedOnDiskReadBufferFromFile::ImplementationBufferPtr CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & file_segment) { - auto range = file_segment->range(); - auto download_state = file_segment->state(); LOG_TEST(log, "getReadBufferForFileSegment: {}", file_segment->getInfoForLog()); @@ -247,7 +247,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil if (download_state == FileSegment::State::DOWNLOADED) { read_type = ReadType::CACHED; - return getCacheReadBuffer(range.left); + return getCacheReadBuffer(*file_segment); } else { @@ -280,7 +280,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil /// file_offset_of_buffer_end read_type = ReadType::CACHED; - return getCacheReadBuffer(range.left); + return getCacheReadBuffer(*file_segment); } download_state = file_segment->wait(); @@ -289,7 +289,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil case FileSegment::State::DOWNLOADED: { read_type = ReadType::CACHED; - return getCacheReadBuffer(range.left); + return getCacheReadBuffer(*file_segment); } case FileSegment::State::EMPTY: case FileSegment::State::PARTIALLY_DOWNLOADED: @@ -305,7 +305,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil /// file_offset_of_buffer_end read_type = ReadType::CACHED; - return getCacheReadBuffer(range.left); + return getCacheReadBuffer(*file_segment); } auto downloader_id = file_segment->getOrSetDownloader(); @@ -323,7 +323,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil read_type = ReadType::CACHED; file_segment->resetDownloader(); - return getCacheReadBuffer(range.left); + return getCacheReadBuffer(*file_segment); } if (file_segment->getCurrentWriteOffset() < file_offset_of_buffer_end) @@ -339,7 +339,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil LOG_TEST(log, "Predownload. File segment info: {}", file_segment->getInfoForLog()); chassert(file_offset_of_buffer_end > file_segment->getCurrentWriteOffset()); bytes_to_predownload = file_offset_of_buffer_end - file_segment->getCurrentWriteOffset(); - chassert(bytes_to_predownload < range.size()); + chassert(bytes_to_predownload < file_segment->range().size()); } read_type = ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE; @@ -354,7 +354,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil if (canStartFromCache(file_offset_of_buffer_end, *file_segment)) { read_type = ReadType::CACHED; - return getCacheReadBuffer(range.left); + return getCacheReadBuffer(*file_segment); } else { diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index b86e53ec160..535d01f3a8c 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -68,7 +68,7 @@ private: ImplementationBufferPtr getReadBufferForFileSegment(FileSegmentPtr & file_segment); - ImplementationBufferPtr getCacheReadBuffer(size_t offset) const; + ImplementationBufferPtr getCacheReadBuffer(const FileSegment & file_segment) const; std::optional getLastNonDownloadedOffset() const; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index f18debe8a8b..56cc20098ba 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_FORMAT; - extern const int LOGICAL_ERROR; } void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) @@ -131,9 +130,6 @@ DiskObjectStorageMetadata::DiskObjectStorageMetadata( void DiskObjectStorageMetadata::addObject(const String & path, size_t size) { - if (!object_storage_root_path.empty() && path.starts_with(object_storage_root_path)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected relative path"); - total_size += size; storage_objects.emplace_back(path, size); } diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 7ddcc80c02a..e3c1fa4c271 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -61,18 +61,23 @@ public: return host_fqdn_id; } + std::string getQueueDir() const + { + return queue_dir; + } + void startup(); virtual void shutdown(); bool isCurrentlyActive() const { return initialized && !stop_flag; } -protected: /// Returns cached ZooKeeper session (possibly expired). ZooKeeperPtr tryGetZooKeeper() const; /// If necessary, creates a new session and caches it. ZooKeeperPtr getAndSetZooKeeper(); +protected: /// Iterates through queue tasks in ZooKeeper, runs execution of new tasks void scheduleTasks(bool reinitialized); diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index b3d237fecfd..c9978de3ab2 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -235,8 +235,10 @@ static void insertNull(IColumn & column, DataTypePtr type) assert_cast(column).insertDefault(); } -static void insertUUID(IColumn & column, DataTypePtr /*type*/, const char * value, size_t size) +static void insertUUID(IColumn & column, DataTypePtr type, const char * value, size_t size) { + if (!isUUID(type)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack UUID into column with type {}.", type->getName()); ReadBufferFromMemory buf(value, size); UUID uuid; readBinaryBigEndian(uuid.toUnderType().items[0], buf); diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp index 111ea343398..67867b6c577 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp @@ -205,9 +205,9 @@ static void fillStatusColumns(MutableColumns & res_columns, size_t & col, void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); - fs::path ddl_zookeeper_path = context->getConfigRef().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/"); - + auto& ddl_worker = context->getDDLWorker(); + fs::path ddl_zookeeper_path = ddl_worker.getQueueDir(); + zkutil::ZooKeeperPtr zookeeper = ddl_worker.getAndSetZooKeeper(); Strings ddl_task_paths = zookeeper->getChildren(ddl_zookeeper_path); GetResponseFutures ddl_task_futures; diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index f58c7a74dfe..d668dbe0498 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -291,7 +291,9 @@ def main(): logging.info("Will try to fetch cache for our build") try: - get_ccache_if_not_exists(ccache_path, s3_helper, pr_info.number, TEMP_PATH) + get_ccache_if_not_exists( + ccache_path, s3_helper, pr_info.number, TEMP_PATH, pr_info.release_pr + ) except Exception as e: # In case there are issues with ccache, remove the path and do not fail a build logging.info("Failed to get ccache, building without it. Error: %s", e) diff --git a/tests/ci/ccache_utils.py b/tests/ci/ccache_utils.py index cfe07363589..864b3a8f9b6 100644 --- a/tests/ci/ccache_utils.py +++ b/tests/ci/ccache_utils.py @@ -11,6 +11,7 @@ import requests # type: ignore from compress_files import decompress_fast, compress_fast from env_helper import S3_DOWNLOAD, S3_BUILDS_BUCKET +from s3_helper import S3Helper DOWNLOAD_RETRIES_COUNT = 5 @@ -57,12 +58,19 @@ def dowload_file_with_progress(url, path): def get_ccache_if_not_exists( - path_to_ccache_dir, s3_helper, current_pr_number, temp_path + path_to_ccache_dir: str, + s3_helper: S3Helper, + current_pr_number: int, + temp_path: str, + release_pr: int, ) -> int: """returns: number of PR for downloaded PR. -1 if ccache not found""" ccache_name = os.path.basename(path_to_ccache_dir) cache_found = False prs_to_check = [current_pr_number] + # Release PR is either 0 or defined + if release_pr: + prs_to_check.append(release_pr) ccache_pr = -1 if current_pr_number != 0: prs_to_check.append(0) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 038289406de..03e42726808 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -125,7 +125,7 @@ if __name__ == "__main__": logging.info("Will try to fetch cache for our build") ccache_for_pr = get_ccache_if_not_exists( - cache_path, s3_helper, pr_info.number, temp_path + cache_path, s3_helper, pr_info.number, temp_path, pr_info.release_pr ) upload_master_ccache = ccache_for_pr in (-1, 0) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 2acd0e4c811..77421ddac32 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -86,7 +86,7 @@ class PRInfo: self.changed_files = set() # type: Set[str] self.body = "" self.diff_urls = [] - self.release_pr = "" + self.release_pr = 0 ref = github_event.get("ref", "refs/head/master") if ref and ref.startswith("refs/heads/"): ref = ref[11:] diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index 3ee49744a61..f3505f53339 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -38,6 +38,20 @@ /jbod1/ 1000000000 + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + + + cache + s3_r + /s3_cache_r/ + 1000000000 + 1 + @@ -78,6 +92,13 @@ + + +
+ s3_cache_r +
+
+
diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 9b254b71826..4ce5fd5a069 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -6,7 +6,6 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.utility import generate_values, replace_config, SafeThread - SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -36,6 +35,7 @@ def cluster(): "/jbod1:size=2M", ], ) + logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -742,3 +742,79 @@ def test_store_cleanup_disk_s3(cluster, node_name): "CREATE TABLE s3_test UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" ) node.query("INSERT INTO s3_test SELECT 1") + + +@pytest.mark.parametrize("node_name", ["node"]) +def test_cache_setting_compatibility(cluster, node_name): + node = cluster.instances[node_name] + + node.query("DROP TABLE IF EXISTS s3_test NO DELAY") + + node.query( + "CREATE TABLE s3_test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_r';" + ) + node.query( + "INSERT INTO s3_test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 500" + ) + + result = node.query("SYSTEM DROP FILESYSTEM CACHE") + + result = node.query( + "SELECT count() FROM system.filesystem_cache WHERE cache_path LIKE '%persistent'" + ) + assert int(result) == 0 + + node.query("SELECT * FROM s3_test") + + result = node.query( + "SELECT count() FROM system.filesystem_cache WHERE cache_path LIKE '%persistent'" + ) + assert int(result) > 0 + + config_path = os.path.join( + SCRIPT_DIR, + f"./{cluster.instances_dir_name}/node/configs/config.d/storage_conf.xml", + ) + + replace_config( + config_path, + "1", + "0", + ) + + result = node.query("DESCRIBE CACHE 's3_cache_r'") + assert result.strip().endswith("1") + + node.restart_clickhouse() + + result = node.query("DESCRIBE CACHE 's3_cache_r'") + assert result.strip().endswith("0") + + result = node.query( + "SELECT count() FROM system.filesystem_cache WHERE cache_path LIKE '%persistent'" + ) + assert int(result) > 0 + + node.query("SELECT * FROM s3_test FORMAT Null") + + assert not node.contains_in_log("No such file or directory: Cache info:") + + replace_config( + config_path, + "0", + "1", + ) + + result = node.query( + "SELECT count() FROM system.filesystem_cache WHERE cache_path LIKE '%persistent'" + ) + assert int(result) > 0 + + node.restart_clickhouse() + + result = node.query("DESCRIBE CACHE 's3_cache_r'") + assert result.strip().endswith("1") + + node.query("SELECT * FROM s3_test FORMAT Null") + + assert not node.contains_in_log("No such file or directory: Cache info:") diff --git a/tests/queries/0_stateless/01321_monotonous_functions_in_order_by_bug.reference b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by_bug.reference new file mode 100644 index 00000000000..0c720206065 --- /dev/null +++ b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by_bug.reference @@ -0,0 +1,2 @@ +2020-01-01 01:00:00 1 +2020-01-01 01:00:00 999 diff --git a/tests/queries/0_stateless/01321_monotonous_functions_in_order_by_bug.sql b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by_bug.sql new file mode 100644 index 00000000000..4aa52fe6ae8 --- /dev/null +++ b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by_bug.sql @@ -0,0 +1,7 @@ +SELECT + toStartOfHour(c1) AS _c1, + c2 +FROM values((toDateTime('2020-01-01 01:01:01'), 999), (toDateTime('2020-01-01 01:01:59'), 1)) +ORDER BY + _c1 ASC, + c2 ASC diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.sql b/tests/queries/0_stateless/01576_alias_column_rewrite.sql index 8424eb11f9b..1f28225bef8 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.sql +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.sql @@ -17,7 +17,7 @@ INSERT INTO test_table(timestamp, value) SELECT toDateTime('2020-01-01 12:00:00' INSERT INTO test_table(timestamp, value) SELECT toDateTime('2020-01-02 12:00:00'), 1 FROM numbers(10); INSERT INTO test_table(timestamp, value) SELECT toDateTime('2020-01-03 12:00:00'), 1 FROM numbers(10); -set optimize_respect_aliases = 1; +set optimize_respect_aliases = 1, optimize_monotonous_functions_in_order_by = 1; SELECT 'test-partition-prune'; SELECT COUNT() = 10 FROM test_table WHERE day = '2020-01-01' SETTINGS max_rows_to_read = 10; diff --git a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql index 4dfcbb9bf80..44c1c12be35 100644 --- a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql +++ b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql @@ -56,7 +56,13 @@ ENGINE = MergeTree ORDER BY (toStartOfDay(dt), d); INSERT INTO t_read_in_order SELECT toDateTime('2020-10-10 00:00:00') + number, 1 / (number % 100 + 1), number FROM numbers(1000); EXPLAIN PIPELINE SELECT toStartOfDay(dt) as date, d FROM t_read_in_order ORDER BY date, round(d) LIMIT 5; -SELECT toStartOfDay(dt) as date, d FROM t_read_in_order ORDER BY date, round(d) LIMIT 5; +SELECT * from ( + SELECT toStartOfDay(dt) as date, d FROM t_read_in_order ORDER BY date, round(d) LIMIT 50000000000 + -- subquery with limit 50000000 to stabilize a test result and prevent order by d pushdown +) order by d limit 5; EXPLAIN PIPELINE SELECT toStartOfDay(dt) as date, d FROM t_read_in_order ORDER BY date, round(d) LIMIT 5; -SELECT toStartOfDay(dt) as date, d FROM t_read_in_order WHERE date = '2020-10-10' ORDER BY round(d) LIMIT 5; +SELECT * from ( + SELECT toStartOfDay(dt) as date, d FROM t_read_in_order WHERE date = '2020-10-10' ORDER BY round(d) LIMIT 50000000000 + -- subquery with limit 50000000 to stabilize a test result and prevent order by d pushdown +) order by d limit 5; diff --git a/tests/queries/0_stateless/02422_msgpack_uuid_wrong_column.reference b/tests/queries/0_stateless/02422_msgpack_uuid_wrong_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02422_msgpack_uuid_wrong_column.sql b/tests/queries/0_stateless/02422_msgpack_uuid_wrong_column.sql new file mode 100644 index 00000000000..4d790354d51 --- /dev/null +++ b/tests/queries/0_stateless/02422_msgpack_uuid_wrong_column.sql @@ -0,0 +1,4 @@ +-- Tags: no-parallel, no-fasttest + +insert into function file(02422_data.msgpack) select toUUID('f4cdd80d-5d15-4bdc-9527-adcca635ec1f') as uuid settings output_format_msgpack_uuid_representation='ext'; +select * from file(02422_data.msgpack, auto, 'x Int32'); -- {serverError ILLEGAL_COLUMN}