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}