Merge remote-tracking branch 'origin/master' into functions-simplification

This commit is contained in:
Alexey Milovidov 2017-07-24 23:14:28 +03:00
commit 3080e20f66
16 changed files with 56 additions and 17 deletions

View File

@ -9,9 +9,12 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
message (FATAL_ERROR "GCC version must be at least 6! For example, if GCC 6 is available under gcc-6, g++-6 names, do the following: export CC=gcc-6 CXX=g++-6; rm -rf CMakeCache.txt CMakeFiles; and re run cmake or ./release.") message (FATAL_ERROR "GCC version must be at least 6! For example, if GCC 6 is available under gcc-6, g++-6 names, do the following: export CC=gcc-6 CXX=g++-6; rm -rf CMakeCache.txt CMakeFiles; and re run cmake or ./release.")
endif () endif ()
elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
# Require at least clang 4 # Require at least clang 3.8
if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS 3.8)
message (FATAL_ERROR "Clang version must be at least 3.8! Recommended 4+")
endif ()
if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS 4) if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS 4)
message (FATAL_ERROR "Clang version must be at least 4!") message (WARNING "Compilation has only been tested with Clang 4+")
endif () endif ()
else () else ()
message (WARNING "You are using an unsupported compiler! Compilation has only been tested with Clang 4+ and GCC 6+.") message (WARNING "You are using an unsupported compiler! Compilation has only been tested with Clang 4+ and GCC 6+.")

View File

@ -1,13 +1,13 @@
include_directories (${CMAKE_CURRENT_BINARY_DIR}) include_directories (${CMAKE_CURRENT_BINARY_DIR})
if (NOT AARCH64) # Not used. Pretty easy to port. if (HAVE_SSE42) # Not used. Pretty easy to port.
set (SOURCES_ONLY_ON_X86_64 src/metrohash128crc.cpp) set (SOURCES_SSE42_ONLY src/metrohash128crc.cpp)
endif () endif ()
add_library(metrohash add_library(metrohash
src/metrohash.h src/metrohash.h
src/testvector.h src/testvector.h
src/metrohash64.cpp src/metrohash64.cpp
src/metrohash128.cpp src/metrohash128.cpp
${SOURCES_ONLY_ON_X86_64}) ${SOURCES_SSE42_ONLY})

View File

@ -6,7 +6,8 @@
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h> #include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <IO/ReadHelpers.h>
#include <Common/typeid_cast.h>
namespace DB namespace DB
{ {

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <tuple>
#include <city.h> #include <city.h>
#include <Common/HashTable/Hash.h> #include <Common/HashTable/Hash.h>

View File

@ -2,6 +2,7 @@
#include <limits> #include <limits>
#include <Core/Types.h> #include <Core/Types.h>
#include <Common/UInt128.h>
/** Preceptually-correct number comparisons. /** Preceptually-correct number comparisons.
* Example: Int8(-1) != UInt8(255) * Example: Int8(-1) != UInt8(255)

View File

@ -1,5 +1,6 @@
#pragma once #pragma once
#include <mutex>
#include <Databases/IDatabase.h> #include <Databases/IDatabase.h>
#include <Common/UInt128.h> #include <Common/UInt128.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>

View File

@ -502,7 +502,7 @@ struct MatchImpl
const ColumnString::Offsets_t & needle_offsets, const ColumnString::Offsets_t & needle_offsets,
PaddedPODArray<UInt8> & res) PaddedPODArray<UInt8> & res)
{ {
throw Exception("Functions 'like' and 'match' doesn't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); throw Exception("Functions 'like' and 'match' don't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
} }
/// Search different needles in single haystack. /// Search different needles in single haystack.
@ -511,7 +511,7 @@ struct MatchImpl
const ColumnString::Offsets_t & needle_offsets, const ColumnString::Offsets_t & needle_offsets,
PaddedPODArray<UInt8> & res) PaddedPODArray<UInt8> & res)
{ {
throw Exception("Functions 'like' and 'match' doesn't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); throw Exception("Functions 'like' and 'match' don't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
} }
}; };

View File

@ -87,15 +87,23 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
if (!task.size_predictor) if (!task.size_predictor)
return max_block_size_rows; return max_block_size_rows;
size_t rows_to_read = std::max(index_granularity, task.size_predictor->estimateNumRows(preferred_block_size_bytes)); /// Calculates number of rows will be read using preferred_block_size_bytes.
/// Can't be less than index_granularity.
size_t rows_to_read = task.size_predictor->estimateNumRows(preferred_block_size_bytes);
if (!rows_to_read)
return rows_to_read;
rows_to_read = std::max(index_granularity, rows_to_read);
if (preferred_max_column_in_block_size_bytes) if (preferred_max_column_in_block_size_bytes)
{ {
/// Calculates number of rows will be read using preferred_max_column_in_block_size_bytes.
size_t rows_to_read_for_max_size_column size_t rows_to_read_for_max_size_column
= task.size_predictor->estimateNumRowsForMaxSizeColumn(preferred_max_column_in_block_size_bytes); = task.size_predictor->estimateNumRowsForMaxSizeColumn(preferred_max_column_in_block_size_bytes);
double filtration_ratio = std::max(min_filtration_ratio, 1.0 - task.size_predictor->filtered_rows_ratio); double filtration_ratio = std::max(min_filtration_ratio, 1.0 - task.size_predictor->filtered_rows_ratio);
size_t rows_to_read_for_max_size_column_with_filtration size_t rows_to_read_for_max_size_column_with_filtration
= static_cast<size_t>(rows_to_read_for_max_size_column / filtration_ratio); = static_cast<size_t>(rows_to_read_for_max_size_column / filtration_ratio);
/// If preferred_max_column_in_block_size_bytes is used, number of rows to read can be less than index_granularity.
rows_to_read = std::min(rows_to_read, rows_to_read_for_max_size_column_with_filtration); rows_to_read = std::min(rows_to_read, rows_to_read_for_max_size_column_with_filtration);
} }

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <memory> #include <memory>
#include <unordered_map>
namespace DB namespace DB

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Core/Defines.h>
#include <common/MultiVersion.h> #include <common/MultiVersion.h>
#include <ext/shared_ptr_helper.h> #include <ext/shared_ptr_helper.h>

View File

@ -2617,11 +2617,11 @@ String StorageReplicatedMergeTree::getFakePartNameCoveringAllPartsInPartition(co
block_number_lock.unlock(); block_number_lock.unlock();
} }
/// This should never happen. /// Empty partition.
if (right == 0) if (right == 0)
throw Exception("Logical error: newly allocated block number is zero", ErrorCodes::LOGICAL_ERROR); return {};
--right;
--right;
return getFakePartNameCoveringPartRange(month_name, left, right); return getFakePartNameCoveringPartRange(month_name, left, right);
} }
@ -2636,6 +2636,12 @@ void StorageReplicatedMergeTree::clearColumnInPartition(
String month_name = MergeTreeData::getMonthName(partition); String month_name = MergeTreeData::getMonthName(partition);
String fake_part_name = getFakePartNameCoveringAllPartsInPartition(month_name); String fake_part_name = getFakePartNameCoveringAllPartsInPartition(month_name);
if (fake_part_name.empty())
{
LOG_INFO(log, "Will not clear partition " << month_name << ", it is empty.");
return;
}
/// We allocated new block number for this part, so new merges can't merge clearing parts with new ones /// We allocated new block number for this part, so new merges can't merge clearing parts with new ones
LogEntry entry; LogEntry entry;
@ -2670,6 +2676,12 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const Field
String month_name = MergeTreeData::getMonthName(partition); String month_name = MergeTreeData::getMonthName(partition);
String fake_part_name = getFakePartNameCoveringAllPartsInPartition(month_name); String fake_part_name = getFakePartNameCoveringAllPartsInPartition(month_name);
if (fake_part_name.empty())
{
LOG_INFO(log, "Will not drop partition " << month_name << ", it is empty.");
return;
}
/** Forbid to choose the parts to be deleted for merging. /** Forbid to choose the parts to be deleted for merging.
* Invariant: after the `DROP_RANGE` entry appears in the log, merge of deleted parts will not appear in the log. * Invariant: after the `DROP_RANGE` entry appears in the log, merge of deleted parts will not appear in the log.
*/ */

View File

@ -468,6 +468,7 @@ private:
void assertNotReadonly() const; void assertNotReadonly() const;
/// The name of an imaginary part covering all parts in the specified partition (at the call moment). /// The name of an imaginary part covering all parts in the specified partition (at the call moment).
/// Returns empty string if partition is empy.
String getFakePartNameCoveringAllPartsInPartition(const String & month_name); String getFakePartNameCoveringAllPartsInPartition(const String & month_name);
/// Check for a node in ZK. If it is, remember this information, and then immediately answer true. /// Check for a node in ZK. If it is, remember this information, and then immediately answer true.

View File

@ -9,6 +9,7 @@
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Common/FileChecker.h> #include <Common/FileChecker.h>
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
#include <Core/Defines.h>
namespace DB namespace DB

View File

@ -9,6 +9,7 @@
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Common/FileChecker.h> #include <Common/FileChecker.h>
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
#include <Core/Defines.h>
namespace DB namespace DB

View File

@ -49,6 +49,12 @@ SELECT sum(data_uncompressed_bytes) FROM system.columns WHERE database='test' AN
ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200001'; ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200001';
ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200002'; ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200002';
-- clear column in empty partition should be Ok
ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200012', CLEAR COLUMN i IN PARTITION '200012';
-- Drop empty partition also Ok
ALTER TABLE test.clear_column1 DROP PARTITION '200012', DROP PARTITION '200011';
-- check optimize for non-leader replica (it is not related with CLEAR COLUMN) -- check optimize for non-leader replica (it is not related with CLEAR COLUMN)
OPTIMIZE TABLE test.clear_column1; OPTIMIZE TABLE test.clear_column1;
OPTIMIZE TABLE test.clear_column2; OPTIMIZE TABLE test.clear_column2;

View File

@ -36,7 +36,7 @@ inc="-I. \
if [ -z $1 ]; then if [ -z $1 ]; then
cd .. cd ..
find dbms libs utils -name *.h -exec sh $pwd/$0 {} \; ; find dbms libs utils \( -name *.h -and -not -name *.inl.h \) -exec sh $pwd/$0 {} \; ;
else else
echo -n "$1 " echo -n "$1 "
echo -n `grep "#include" $1| wc -l` " " echo -n `grep "#include" $1| wc -l` " "