..
Cache
Remove superfluous includes of logger_userful.h from headers
2023-04-10 17:59:30 +02:00
DataLakes
Switch Block::NameMap to google::dense_hash_map over HashMap
2023-05-12 05:52:57 +02:00
Distributed
Fix processing pending batch for Distributed async INSERT after restart
2023-05-15 15:57:30 +02:00
examples
Fix build
2023-05-03 00:09:52 +02:00
FileLog
Highly questionable refactoring (getInputMultistream() nonsense)
2023-04-17 04:58:32 +00:00
fuzzers
HDFS
Fixes for clang-17
2023-05-13 02:57:31 +02:00
Hive
Remove dependency from DB::Context in readers
2023-05-02 21:45:27 +02:00
Kafka
Allow using Alias column type for KafkaEngine
2023-05-15 15:39:58 +02:00
LiveView
Remove -Wshadow suppression which leaked into global namespace
2023-04-13 08:46:40 +00:00
MeiliSearch
fix stdlib compatibility issues
2023-03-29 12:20:27 +03:00
MergeTree
Merge branch 'master' into fix_another_zero_copy_bug
2023-05-19 12:40:48 +02:00
MySQL
Fox
2023-04-13 19:36:25 +02:00
NATS
Highly questionable refactoring (getInputMultistream() nonsense)
2023-04-17 04:58:32 +00:00
PostgreSQL
replace NO DELAY with SYNC in tests
2023-05-03 20:08:49 +02:00
RabbitMQ
fix convertation
2023-05-10 17:50:42 +00:00
RocksDB
Merge pull request #48435 from ClickHouse/kv-update-only-affected-rows
2023-04-06 12:33:54 +02:00
System
New system table zookeeper connection ( #45245 )
2023-05-19 17:06:43 +03:00
tests
Remove unused mockSystemDatabase from gtest_transform_query_for_external_database
2023-03-29 11:02:50 +00:00
WindowView
use Poco::Timestamp() instead of std::time
2023-04-20 14:36:54 +00:00
addColumnsStructureToQueryWithClusterEngine.cpp
Correctly append arguments
2023-05-04 08:30:45 +00:00
addColumnsStructureToQueryWithClusterEngine.h
Correctly append arguments
2023-05-04 08:30:45 +00:00
AlterCommands.cpp
Merge pull request #49563 from evillique/object-column-alter
2023-05-06 18:17:16 +03:00
AlterCommands.h
remove useless code
2023-01-25 16:11:15 +00:00
checkAndGetLiteralArgument.cpp
s390x build support
2023-01-20 21:16:55 +00:00
checkAndGetLiteralArgument.h
CheckResults.h
CMakeLists.txt
ColumnDefault.cpp
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
ColumnDefault.h
ColumnDependency.h
ColumnsDescription.cpp
Allow using Alias column type for KafkaEngine
2023-05-15 15:39:58 +02:00
ColumnsDescription.h
Allow using Alias column type for KafkaEngine
2023-05-15 15:39:58 +02:00
CompressionCodecSelector.h
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
ConstraintsDescription.cpp
Add support for substitute column
2023-03-17 13:38:01 +00:00
ConstraintsDescription.h
Add support for substitute column
2023-03-17 13:38:01 +00:00
DataDestinationType.h
ExecutableSettings.cpp
ExecutableSettings.h
Better alias definition
2023-02-01 13:54:03 +00:00
ExternalDataSourceConfiguration.cpp
Remove redundant
2023-02-27 12:32:13 +01:00
ExternalDataSourceConfiguration.h
Finish for streaming engines
2023-02-21 14:50:55 +01:00
extractKeyExpressionList.cpp
extractKeyExpressionList.h
Freeze.cpp
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
Freeze.h
getStructureOfRemoteTable.cpp
Parallel replicas. Part [2] ( #43772 )
2023-02-03 14:34:18 +01:00
getStructureOfRemoteTable.h
getVirtualsForStorage.cpp
getVirtualsForStorage.h
IMessageProducer.cpp
Fix race
2023-02-13 16:02:41 +01:00
IMessageProducer.h
Fix race
2023-02-13 16:02:41 +01:00
IndicesDescription.cpp
fix skip indexes
2023-03-06 15:29:13 +00:00
IndicesDescription.h
IStorage_fwd.h
IStorage.cpp
Propagate input_format_parquet_preserve_order to parallelizeOutputAfterReading()
2023-05-05 04:20:27 +00:00
IStorage.h
Merge pull request #49434 from ClickHouse/ins
2023-05-09 08:10:15 +03:00
IStorageCluster.h
add skip_unavailable_shards setting for table function cluste
2023-02-23 17:05:51 +08:00
KeyDescription.cpp
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
KeyDescription.h
KVStorageUtils.cpp
KVStorageUtils.h
Remove superfluous includes of logger_userful.h from headers
2023-04-10 17:59:30 +02:00
LightweightDeleteDescription.cpp
LightweightDeleteDescription.h
MarkCache.h
Reduce inter-header dependencies
2023-05-02 21:15:18 +02:00
MemorySettings.cpp
MemorySettings.h
Better alias definition
2023-02-01 13:54:03 +00:00
MessageQueueSink.cpp
Add logging
2023-02-10 18:14:49 +01:00
MessageQueueSink.h
MutationCommands.cpp
Revert "Revert "Fix several RENAME COLUMN
bugs.""
2023-02-27 12:27:57 +01:00
MutationCommands.h
Revert "Revert "Fix several RENAME COLUMN
bugs.""
2023-02-27 12:27:57 +01:00
NamedCollectionsHelpers.cpp
Review fixes
2023-03-17 13:56:02 +01:00
NamedCollectionsHelpers.h
Fix clang-tidy
2023-03-05 22:12:51 +01:00
PartitionCommands.cpp
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
PartitionCommands.h
Fix typos with new codespell
2022-09-02 08:54:48 +00:00
PartitionedSink.cpp
PartitionedSink.h
Fix double whitespace in exception message
2023-04-15 23:54:10 +02:00
ProjectionsDescription.cpp
forbid old ctor of Exception
2023-01-23 22:18:05 +01:00
ProjectionsDescription.h
ReadFinalForExternalReplicaStorage.cpp
ReadFinalForExternalReplicaStorage.h
ReadFromStorageProgress.cpp
ReadFromStorageProgress.h
Show progress bar while reading from s3 table function
2022-11-21 17:56:02 +01:00
ReadInOrderOptimizer.cpp
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
ReadInOrderOptimizer.h
registerStorages.cpp
fix build without parquet
2023-04-17 21:37:32 -07:00
registerStorages.h
removeGroupingFunctionSpecializations.cpp
Resolve as FunctionGrouping
2023-03-14 03:33:31 +00:00
removeGroupingFunctionSpecializations.h
Move visitor
2023-03-10 14:36:56 +00:00
RenamingRestrictions.h
SelectQueryDescription.cpp
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
SelectQueryDescription.h
Analyzer support LiveView
2023-02-16 12:17:03 +01:00
SelectQueryInfo.h
Support transformQueryForExternalDatabase for analyzer
2023-03-22 08:59:04 +01:00
SetSettings.cpp
SetSettings.h
Better alias definition
2023-02-01 13:54:03 +00:00
StorageBuffer.cpp
Only check MV on ALTER when necessary
2023-03-27 17:45:15 +02:00
StorageBuffer.h
[RFC] Replacing merge tree new engine ( #41005 )
2023-02-16 16:03:16 +03:00
StorageConfiguration.h
Simplify code around storage s3 configuration
2023-02-06 16:23:17 +01:00
StorageDictionary.cpp
Apply some CTAD
2023-03-02 13:36:47 +00:00
StorageDictionary.h
Propagate input_format_parquet_preserve_order to parallelizeOutputAfterReading()
2023-05-05 04:20:27 +00:00
StorageDistributed.cpp
Fixes for clang-17
2023-05-13 02:57:31 +02:00
StorageDistributed.h
Remove superfluous includes of logger_userful.h from headers
2023-04-10 17:59:30 +02:00
StorageDummy.cpp
Fixed tests
2023-03-01 18:05:07 +01:00
StorageDummy.h
Fixed tests
2023-03-01 18:05:07 +01:00
StorageExecutable.cpp
StorageExecutable.h
Remove superfluous includes of logger_userful.h from headers
2023-04-10 17:59:30 +02:00
StorageExternalDistributed.cpp
Review fixes
2023-03-17 13:56:02 +01:00
StorageExternalDistributed.h
Replace for table function remote, and external storage
2023-02-21 14:33:37 +01:00
StorageFactory.cpp
Remove PVS-Studio
2023-02-19 23:30:05 +01:00
StorageFactory.h
StorageFile.cpp
Disable mmap for server
2023-05-10 03:16:52 +02:00
StorageFile.h
Propagate input_format_parquet_preserve_order to parallelizeOutputAfterReading()
2023-05-05 04:20:27 +00:00
StorageGenerateRandom.cpp
Fix IBM
2023-04-21 12:38:45 +02:00
StorageGenerateRandom.h
Fix SipHash integer hashing and byte order issue in GenerateRandom for s390x
2023-03-14 11:54:17 -04:00
StorageInMemoryMetadata.cpp
Revert "Revert "Fix several RENAME COLUMN
bugs.""
2023-02-27 12:27:57 +01:00
StorageInMemoryMetadata.h
Revert "Revert "Fix several RENAME COLUMN
bugs.""
2023-02-27 12:27:57 +01:00
StorageInput.cpp
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
StorageInput.h
StorageJoin.cpp
Fix key not found error for queries with multiple StorageJoin
2023-04-25 11:28:54 +00:00
StorageJoin.h
Fix build
2023-03-13 10:49:51 +00:00
StorageKeeperMap.cpp
Merge pull request #48435 from ClickHouse/kv-update-only-affected-rows
2023-04-06 12:33:54 +02:00
StorageKeeperMap.h
Remove superfluous includes of logger_userful.h from headers
2023-04-10 17:59:30 +02:00
StorageLog.cpp
Add backup setting "decrypt_files_from_encrypted_disks"
2023-05-16 14:27:27 +02:00
StorageLog.h
StorageLogSettings.cpp
fix
2023-02-11 13:58:19 +00:00
StorageLogSettings.h
Storage Log faminy support settings storage policy
2023-02-04 14:28:31 +00:00
StorageMaterializedMySQL.cpp
StorageMaterializedMySQL.h
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
StorageMaterializedView.cpp
Fix race between DROP MatView and RESTART REPLICAS ( #47863 )
2023-04-01 15:26:00 +03:00
StorageMaterializedView.h
[RFC] Replacing merge tree new engine ( #41005 )
2023-02-16 16:03:16 +03:00
StorageMemory.cpp
Corrections after review.
2023-05-17 03:23:16 +02:00
StorageMemory.h
move pipe compute into initializePipeline
2023-05-02 14:59:41 +02:00
StorageMerge.cpp
Merge pull request #48062 from Algunenano/unnecessary_alter_checks
2023-04-03 17:23:11 -04:00
StorageMerge.h
Fix crash in EXPLAIN PIPELINE for Merge over Distributed
2023-04-02 10:48:23 +02:00
StorageMergeTree.cpp
Add backup setting "decrypt_files_from_encrypted_disks"
2023-05-16 14:27:27 +02:00
StorageMergeTree.h
add lock
2023-05-10 23:00:49 +00:00
StorageMongoDB.cpp
Add support for {server_uuid} macro
2023-04-09 03:04:26 +02:00
StorageMongoDB.h
fix typo
2023-05-15 16:51:20 +08:00
StorageMongoDBSocketFactory.cpp
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
StorageMongoDBSocketFactory.h
StorageMySQL.cpp
Fox
2023-04-13 19:36:25 +02:00
StorageMySQL.h
Fix test_storage_mysql/test.py::test_settings_connection_wait_timeout
2023-04-14 20:10:57 +00:00
StorageNull.cpp
Only check MV on ALTER when necessary
2023-03-27 17:45:15 +02:00
StorageNull.h
Propagate input_format_parquet_preserve_order to parallelizeOutputAfterReading()
2023-05-05 04:20:27 +00:00
StoragePostgreSQL.cpp
s/use_tables_cache/use_table_cache
2023-05-03 23:40:12 +00:00
StoragePostgreSQL.h
Finish for mysql
2023-02-20 21:37:38 +01:00
StorageProxy.h
[RFC] Replacing merge tree new engine ( #41005 )
2023-02-16 16:03:16 +03:00
StorageReplicatedMergeTree.cpp
Merge branch 'master' into fix_another_zero_copy_bug
2023-05-19 12:40:48 +02:00
StorageReplicatedMergeTree.h
use flag has_exclusive_blobs to track the part's originate
2023-05-06 03:07:29 +02:00
StorageS3.cpp
rework WriteBufferFromS3, squashed
2023-05-10 18:31:47 +00:00
StorageS3.h
Switch Block::NameMap to google::dense_hash_map over HashMap
2023-05-12 05:52:57 +02:00
StorageS3Cluster.cpp
Correctly append arguments
2023-05-04 08:30:45 +00:00
StorageS3Cluster.h
Correctly append arguments
2023-05-04 08:30:45 +00:00
StorageS3Settings.cpp
Add ability to use strict parts size for S3 (compatibility with R2)
2023-04-28 11:01:56 +02:00
StorageS3Settings.h
Add ability to use strict parts size for S3 (compatibility with R2)
2023-04-28 11:01:56 +02:00
StorageSet.cpp
Merge remote-tracking branch 'origin/master' into optimize-compilation
2023-04-13 16:04:09 +02:00
StorageSet.h
Improve file includes
2023-03-24 03:44:52 +01:00
StorageSnapshot.cpp
StorageSnapshot.h
StorageSQLite.cpp
Support transformQueryForExternalDatabase for analyzer
2023-03-22 08:59:04 +01:00
StorageSQLite.h
StorageStripeLog.cpp
Add backup setting "decrypt_files_from_encrypted_disks"
2023-05-16 14:27:27 +02:00
StorageStripeLog.h
StorageTableFunction.h
Analyzer added distributed table functions support
2023-02-16 12:17:03 +01:00
StorageURL.cpp
Propagate input_format_parquet_preserve_order to parallelizeOutputAfterReading()
2023-05-05 04:20:27 +00:00
StorageURL.h
Propagate input_format_parquet_preserve_order to parallelizeOutputAfterReading()
2023-05-05 04:20:27 +00:00
StorageValues.cpp
StorageValues.h
StorageView.cpp
Added space to if expression of replaceQueryParameterWithValue function
2023-03-14 09:26:53 +01:00
StorageView.h
Addressed review comments for parameterized view bug fix
2023-03-14 09:23:12 +01:00
StorageXDBC.cpp
Support transformQueryForExternalDatabase for analyzer
2023-03-22 08:59:04 +01:00
StorageXDBC.h
TableLockHolder.h
transformQueryForExternalDatabase.cpp
Better transformQueryForExternalDatabase for analyzer
2023-03-22 16:58:34 +00:00
transformQueryForExternalDatabase.h
Support transformQueryForExternalDatabase for analyzer
2023-03-22 08:59:04 +01:00
transformQueryForExternalDatabaseAnalyzer.cpp
Better transformQueryForExternalDatabase for analyzer
2023-03-22 16:58:34 +00:00
transformQueryForExternalDatabaseAnalyzer.h
Support transformQueryForExternalDatabase for analyzer
2023-03-22 08:59:04 +01:00
TTLDescription.cpp
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
TTLDescription.h
support TTL TO [DISK|VOLUME] [IF EXISTS]
2022-02-10 19:26:23 +03:00
TTLMode.h
UVLoop.h
Better formatting for exception messages ( #45449 )
2023-01-24 00:13:58 +03:00
VirtualColumnUtils.cpp
Do not skip building set even when reading from remote
2023-05-02 21:31:56 +02:00
VirtualColumnUtils.h