From 9996e181e1ddc7ef50f03884c7ab7a83be9281d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 6 Jan 2018 21:10:44 +0300 Subject: [PATCH 01/34] Added method "getHeader" in IBlockInputStream [#CLICKHOUSE-2] --- .../AggregateFunctionSequenceMatch.h | 3 +- .../ReservoirSamplerDeterministic.h | 3 +- dbms/src/Common/HyperLogLogCounter.h | 4 +- dbms/src/Common/OptimizedRegularExpression.h | 2 +- dbms/src/Common/SmallObjectPool.h | 3 +- dbms/src/Common/formatIPv6.cpp | 3 +- .../Common/tests/gtest_rw_lock_fifo.cpp.cpp | 9 +- .../tests/integer_hash_tables_and_hashes.cpp | 2 +- dbms/src/Common/tests/shell_command_test.cpp | 3 +- .../AddingConstColumnBlockInputStream.h | 7 + .../AggregatingBlockInputStream.cpp | 7 +- .../DataStreams/AggregatingBlockInputStream.h | 2 + .../AggregatingSortedBlockInputStream.h | 2 + .../AsynchronousBlockInputStream.h | 3 + .../DataStreams/BlockExtraInfoInputStream.h | 4 +- .../BlockInputStreamFromRowInputStream.h | 2 + .../DataStreams/CapnProtoRowInputStream.cpp | 105 +++++------ .../DataStreams/CastTypeBlockInputStream.h | 2 + .../CollapsingFinalBlockInputStream.h | 2 + .../CollapsingSortedBlockInputStream.h | 2 + dbms/src/DataStreams/ColumnGathererStream.h | 2 + dbms/src/DataStreams/ConcatBlockInputStream.h | 2 + .../CreatingSetsBlockInputStream.h | 2 + .../DataStreams/DistinctBlockInputStream.h | 2 + .../DistinctSortedBlockInputStream.h | 2 + dbms/src/DataStreams/EmptyBlockOutputStream.h | 27 --- .../ExpressionBlockInputStream.cpp | 7 + .../DataStreams/ExpressionBlockInputStream.h | 1 + .../DataStreams/FilterBlockInputStream.cpp | 65 +++---- dbms/src/DataStreams/FilterBlockInputStream.h | 5 +- .../FilterColumnsBlockInputStream.cpp | 12 ++ .../FilterColumnsBlockInputStream.h | 2 + dbms/src/DataStreams/ForkBlockInputStreams.h | 74 -------- .../GraphiteRollupSortedBlockInputStream.h | 2 + dbms/src/DataStreams/IBlockInputStream.h | 7 + .../InputStreamFromASTInsertQuery.cpp | 17 +- .../InputStreamFromASTInsertQuery.h | 20 +-- dbms/src/DataStreams/LazyBlockInputStream.h | 108 +++++++----- dbms/src/DataStreams/LimitBlockInputStream.h | 2 + .../DataStreams/LimitByBlockInputStream.cpp | 2 + .../src/DataStreams/LimitByBlockInputStream.h | 4 +- .../MaterializingBlockInputStream.cpp | 12 ++ .../MaterializingBlockInputStream.h | 1 + .../MergeSortingBlockInputStream.h | 4 + .../MergingAggregatedBlockInputStream.cpp | 5 + .../MergingAggregatedBlockInputStream.h | 2 + ...regatedMemoryEfficientBlockInputStream.cpp | 6 + ...ggregatedMemoryEfficientBlockInputStream.h | 2 + .../MergingSortedBlockInputStream.h | 2 + .../DataStreams/NativeBlockInputStream.cpp | 14 ++ dbms/src/DataStreams/NativeBlockInputStream.h | 2 + .../NullAndDoCopyBlockInputStream.h | 2 + dbms/src/DataStreams/NullBlockInputStream.h | 9 +- .../NullableAdapterBlockInputStream.cpp | 1 + .../NullableAdapterBlockInputStream.h | 3 + dbms/src/DataStreams/OneBlockInputStream.h | 8 + dbms/src/DataStreams/OwningBlockInputStream.h | 2 + .../ParallelAggregatingBlockInputStream.cpp | 6 + .../ParallelAggregatingBlockInputStream.h | 2 + .../PartialSortingBlockInputStream.h | 2 + dbms/src/DataStreams/QueueBlockIOStream.h | 69 -------- .../DataStreams/RemoteBlockInputStream.cpp | 48 ++++- dbms/src/DataStreams/RemoteBlockInputStream.h | 6 + .../RemoveColumnsBlockInputStream.h | 11 ++ .../ReplacingSortedBlockInputStream.h | 2 + .../DataStreams/SquashingBlockInputStream.h | 2 + .../SummingSortedBlockInputStream.h | 2 + .../TotalsHavingBlockInputStream.cpp | 17 +- .../TotalsHavingBlockInputStream.h | 4 +- dbms/src/DataStreams/UnionBlockInputStream.h | 2 + .../src/DataStreams/glueBlockInputStreams.cpp | 63 ------- dbms/src/DataStreams/glueBlockInputStreams.h | 17 -- dbms/src/DataStreams/tests/CMakeLists.txt | 6 - dbms/src/DataStreams/tests/fork_streams.cpp | 110 ------------ dbms/src/DataStreams/tests/glue_streams.cpp | 98 ----------- dbms/src/Databases/DatabaseMemory.cpp | 3 +- dbms/src/Databases/DatabaseOrdinary.cpp | 3 +- .../Dictionaries/ComplexKeyCacheDictionary.h | 9 +- .../ComplexKeyHashedDictionary.cpp | 6 +- .../Dictionaries/DictionaryBlockInputStream.h | 13 +- .../DictionaryBlockInputStreamBase.cpp | 11 ++ .../DictionaryBlockInputStreamBase.h | 5 +- dbms/src/Dictionaries/Embedded/RegionsNames.h | 3 +- .../ExecutableDictionarySource.cpp | 2 + dbms/src/Dictionaries/FlatDictionary.cpp | 5 +- dbms/src/Dictionaries/HashedDictionary.cpp | 4 +- .../Dictionaries/MongoDBBlockInputStream.cpp | 2 +- .../Dictionaries/MongoDBBlockInputStream.h | 2 + dbms/src/Dictionaries/MySQLBlockInputStream.h | 2 + dbms/src/Dictionaries/ODBCBlockInputStream.h | 2 + .../RangeDictionaryBlockInputStream.h | 26 +-- .../Dictionaries/RangeHashedDictionary.cpp | 3 +- dbms/src/Dictionaries/TrieDictionary.cpp | 3 +- dbms/src/Functions/FunctionsFindCluster.h | 2 +- dbms/src/Functions/FunctionsGeo.h | 3 +- dbms/src/Functions/FunctionsHashing.h | 3 +- dbms/src/Functions/FunctionsMiscellaneous.cpp | 6 +- dbms/src/Functions/Regexps.h | 4 +- dbms/src/IO/ReadBufferFromFile.cpp | 3 +- dbms/src/IO/WriteBufferFromFile.cpp | 3 +- dbms/src/Interpreters/Aggregator.cpp | 165 ++++++------------ dbms/src/Interpreters/Aggregator.h | 50 +++--- .../ClusterProxy/AlterStreamFactory.cpp | 40 ----- .../ClusterProxy/AlterStreamFactory.h | 25 --- .../ClusterProxy/executeQuery.cpp | 1 - dbms/src/Interpreters/DDLWorker.cpp | 4 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 5 +- .../src/Interpreters/ExternalDictionaries.cpp | 4 +- dbms/src/Interpreters/ExternalModels.cpp | 4 +- .../InterpreterKillQueryQuery.cpp | 4 +- .../Interpreters/InterpreterSelectQuery.cpp | 24 ++- dbms/src/Interpreters/Join.cpp | 2 + dbms/src/Interpreters/tests/aggregate.cpp | 9 +- .../Interpreters/tests/hash_map_string.cpp | 24 +-- .../Interpreters/tests/hash_map_string_3.cpp | 27 +-- dbms/src/Parsers/ASTColumnDeclaration.h | 3 +- dbms/src/Server/Client.cpp | 3 +- dbms/src/Server/InterruptListener.h | 12 +- dbms/src/Server/MetricsTransmitter.cpp | 3 +- dbms/src/Server/PerformanceTest.cpp | 9 +- dbms/src/Storages/AlterCommands.cpp | 9 +- .../MergeTree/MergeTreeBaseBlockInputStream.h | 2 + .../MergeTree/MergeTreeBlockReadUtils.cpp | 4 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 6 +- dbms/src/Storages/StorageBuffer.cpp | 15 +- dbms/src/Storages/StorageCatBoostPool.cpp | 2 + dbms/src/Storages/StorageCatBoostPool.h | 3 +- dbms/src/Storages/StorageDistributed.cpp | 2 - dbms/src/Storages/StorageFile.cpp | 2 + dbms/src/Storages/StorageKafka.cpp | 5 +- dbms/src/Storages/StorageLog.cpp | 2 + dbms/src/Storages/StorageMemory.cpp | 9 +- dbms/src/Storages/StorageMerge.cpp | 2 +- dbms/src/Storages/StorageNull.h | 2 +- dbms/src/Storages/StorageStripeLog.cpp | 4 +- dbms/src/Storages/StorageTinyLog.cpp | 2 + .../Storages/System/StorageSystemGraphite.cpp | 3 +- .../Storages/System/StorageSystemNumbers.cpp | 11 +- .../src/TableFunctions/TableFunctionFactory.h | 3 +- .../dictionary_library/dictionary_library.cpp | 3 +- 141 files changed, 747 insertions(+), 969 deletions(-) delete mode 100644 dbms/src/DataStreams/EmptyBlockOutputStream.h delete mode 100644 dbms/src/DataStreams/ForkBlockInputStreams.h delete mode 100644 dbms/src/DataStreams/QueueBlockIOStream.h delete mode 100644 dbms/src/DataStreams/glueBlockInputStreams.cpp delete mode 100644 dbms/src/DataStreams/glueBlockInputStreams.h delete mode 100644 dbms/src/DataStreams/tests/fork_streams.cpp delete mode 100644 dbms/src/DataStreams/tests/glue_streams.cpp delete mode 100644 dbms/src/Interpreters/ClusterProxy/AlterStreamFactory.cpp delete mode 100644 dbms/src/Interpreters/ClusterProxy/AlterStreamFactory.h diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index c41bd9e8010..297e74ae975 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -338,7 +338,8 @@ protected: std::stack back_stack; /// backtrack if possible - const auto do_backtrack = [&] { + const auto do_backtrack = [&] + { while (!back_stack.empty()) { auto & top = back_stack.top(); diff --git a/dbms/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/dbms/src/AggregateFunctions/ReservoirSamplerDeterministic.h index 5d90ce4ebaa..bff3fcb78ea 100644 --- a/dbms/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/dbms/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -210,7 +210,8 @@ private: if (sorted) return; sorted = true; - std::sort(samples.begin(), samples.end(), [] (const std::pair & lhs, const std::pair & rhs) { + std::sort(samples.begin(), samples.end(), [] (const std::pair & lhs, const std::pair & rhs) + { return lhs.first < rhs.first; }); } diff --git a/dbms/src/Common/HyperLogLogCounter.h b/dbms/src/Common/HyperLogLogCounter.h index b51d00fd0ff..325d853c371 100644 --- a/dbms/src/Common/HyperLogLogCounter.h +++ b/dbms/src/Common/HyperLogLogCounter.h @@ -309,7 +309,7 @@ public: { /// Normalizing factor for harmonic mean. static constexpr double alpha_m = - bucket_count == 2 ? 0.351 : + bucket_count == 2 ? 0.351 : bucket_count == 4 ? 0.532 : bucket_count == 8 ? 0.626 : bucket_count == 16 ? 0.673 : @@ -516,10 +516,8 @@ private: } private: - /// Maximum rank. static constexpr int max_rank = sizeof(HashValueType) * 8 - precision + 1; - /// Rank storage. RankStore rank_store; /// Expression's denominator for HyperLogLog algorithm. diff --git a/dbms/src/Common/OptimizedRegularExpression.h b/dbms/src/Common/OptimizedRegularExpression.h index 3dbfb27c78f..81bfc2653c9 100644 --- a/dbms/src/Common/OptimizedRegularExpression.h +++ b/dbms/src/Common/OptimizedRegularExpression.h @@ -85,7 +85,7 @@ public: unsigned getNumberOfSubpatterns() const { return number_of_subpatterns; } /// Get the regexp re2 or nullptr if the pattern is trivial (for output to the log). - const std::unique_ptr& getRE2() const { return re2; } + const std::unique_ptr & getRE2() const { return re2; } static void analyze(const std::string & regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix); diff --git a/dbms/src/Common/SmallObjectPool.h b/dbms/src/Common/SmallObjectPool.h index 9d915ddb6af..4eaf6cd26ab 100644 --- a/dbms/src/Common/SmallObjectPool.h +++ b/dbms/src/Common/SmallObjectPool.h @@ -62,7 +62,8 @@ public: void free(const void * ptr) { - union { + union + { const void * p_v; Block * block; }; diff --git a/dbms/src/Common/formatIPv6.cpp b/dbms/src/Common/formatIPv6.cpp index 575549dd5ea..a9eea719e6f 100644 --- a/dbms/src/Common/formatIPv6.cpp +++ b/dbms/src/Common/formatIPv6.cpp @@ -74,7 +74,8 @@ void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_ for (const auto i : ext::range(0, words.size())) { - if (words[i] == 0) { + if (words[i] == 0) + { if (cur.base == -1) cur.base = i, cur.len = 1; else diff --git a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp index 11e0a45789d..3334dccabc6 100644 --- a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp @@ -30,7 +30,8 @@ TEST(Common, RWLockFIFO_1) static thread_local std::random_device rd; static thread_local pcg64 gen(rd()); - auto func = [&] (size_t threads, int round) { + auto func = [&] (size_t threads, int round) + { for (int i = 0; i < cycles; ++i) { auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockFIFO::Read : RWLockFIFO::Write; @@ -92,7 +93,8 @@ TEST(Common, RWLockFIFO_Recursive) static thread_local std::random_device rd; static thread_local pcg64 gen(rd()); - std::thread t1([&] () { + std::thread t1([&] () + { for (int i = 0; i < 2 * cycles; ++i) { auto lock = fifo_lock->getLock(RWLockFIFO::Write); @@ -102,7 +104,8 @@ TEST(Common, RWLockFIFO_Recursive) } }); - std::thread t2([&] () { + std::thread t2([&] () + { for (int i = 0; i < cycles; ++i) { auto lock1 = fifo_lock->getLock(RWLockFIFO::Read); diff --git a/dbms/src/Common/tests/integer_hash_tables_and_hashes.cpp b/dbms/src/Common/tests/integer_hash_tables_and_hashes.cpp index a13df34129b..d86d77c5144 100644 --- a/dbms/src/Common/tests/integer_hash_tables_and_hashes.cpp +++ b/dbms/src/Common/tests/integer_hash_tables_and_hashes.cpp @@ -286,7 +286,7 @@ namespace Hashes template