Merge branch 'master' of https://github.com/yandex/ClickHouse into CLICKHOUSE-2720

This commit is contained in:
Ivan Blinkov 2017-09-25 15:09:34 +03:00
commit ade81ce275
50 changed files with 1727 additions and 355 deletions

View File

@ -1,3 +1,5 @@
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
[Read more...](https://clickhouse.yandex/)
[ClickHouse Meetup in Berlin on October 5, 2017](https://events.yandex.com/events/meetings/05-10-2017/)

View File

@ -334,7 +334,7 @@ else()
add_feature_info(SSE2 1 "Support the SSE2 instruction set, using \"${SSE2FLAG}\"")
endif()
if(WITH_OPTIM)
if(NOT CMAKE_SYSTEM_PROCESSOR MATCHES "arm")
if(NOT CMAKE_SYSTEM_PROCESSOR MATCHES "arm" AND NOT CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64")
set(ZLIB_ARCH_SRCS ${ZLIB_ARCH_SRCS} ${ARCHDIR}/x86.c)
endif()
if(HAVE_SSE42_INTRIN)
@ -489,15 +489,39 @@ if(MINGW OR MSYS)
set(ZLIB_DLL_SRCS ${CMAKE_CURRENT_BINARY_DIR}/zlib1rc.obj)
endif(MINGW OR MSYS)
add_library(zlib SHARED ${ZLIB_SRCS} ${ZLIB_GZFILE_SRCS} ${ZLIB_ARCH_SRCS} ${ZLIB_ASMS} ${ZLIB_DLL_SRCS} ${ZLIB_PUBLIC_HDRS} ${ZLIB_PRIVATE_HDRS})
add_library(zlibstatic STATIC ${ZLIB_SRCS} ${ZLIB_GZFILE_SRCS} ${ZLIB_ARCH_SRCS} ${ZLIB_ASMS} ${ZLIB_PUBLIC_HDRS} ${ZLIB_PRIVATE_HDRS})
set_target_properties(zlib PROPERTIES DEFINE_SYMBOL ZLIB_DLL)
set_target_properties(zlib PROPERTIES SOVERSION 1)
if(NOT CYGWIN)
# This property causes shared libraries on Linux to have the full version
# encoded into their final filename. We disable this on Cygwin because
# it causes cygz-${ZLIB_FULL_VERSION}.dll to be created when cygz.dll
# seems to be the default.
#
# This has no effect with MSVC, on that platform the version info for
# the DLL comes from the resource file win32/zlib1.rc
set_target_properties(zlib PROPERTIES VERSION ${ZLIB_FULL_VERSION})
endif()
if(UNIX)
# On unix-like platforms the library is almost always called libz
set_target_properties(zlibstatic PROPERTIES OUTPUT_NAME z)
set_target_properties(zlib zlibstatic PROPERTIES OUTPUT_NAME z)
if(NOT APPLE)
set_target_properties(zlib PROPERTIES LINK_FLAGS "-Wl,--version-script,\"${CMAKE_CURRENT_SOURCE_DIR}/zlib.map\"")
endif()
elseif(MSYS)
# Suppress version number from shared library name
set(CMAKE_SHARED_LIBRARY_NAME_WITH_VERSION 0)
elseif(BUILD_SHARED_LIBS AND WIN32)
# Creates zlib1.dll when building shared library version
set_target_properties(zlib PROPERTIES SUFFIX "1.dll")
endif()
if(NOT SKIP_INSTALL_LIBRARIES AND NOT SKIP_INSTALL_ALL )
install(TARGETS zlibstatic
install(TARGETS zlib zlibstatic
RUNTIME DESTINATION "${INSTALL_BIN_DIR}"
ARCHIVE DESTINATION "${INSTALL_LIB_DIR}"
LIBRARY DESTINATION "${INSTALL_LIB_DIR}" )
@ -505,6 +529,35 @@ endif()
if(NOT SKIP_INSTALL_HEADERS AND NOT SKIP_INSTALL_ALL )
install(FILES ${ZLIB_PUBLIC_HDRS} DESTINATION "${INSTALL_INC_DIR}")
endif()
if(NOT SKIP_INSTALL_FILES AND NOT SKIP_INSTALL_ALL )
install(FILES zlib.3 DESTINATION "${INSTALL_MAN_DIR}/man3")
endif()
if(NOT SKIP_INSTALL_FILES AND NOT SKIP_INSTALL_ALL )
install(FILES ${ZLIB_PC} DESTINATION "${INSTALL_PKGCONFIG_DIR}")
endif()
#============================================================================
# Example binaries
#============================================================================
if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
set (CMAKE_EXE_LINKER_FLAGS "")
endif ()
add_executable(example test/example.c)
target_link_libraries(example zlib)
add_test(example example)
add_executable(minigzip test/minigzip.c)
target_link_libraries(minigzip zlib)
if(HAVE_OFF64_T)
add_executable(example64 test/example.c)
target_link_libraries(example64 zlib)
set_target_properties(example64 PROPERTIES COMPILE_FLAGS "-D_FILE_OFFSET_BITS=64")
add_test(example64 example64)
add_executable(minigzip64 test/minigzip.c)
target_link_libraries(minigzip64 zlib)
set_target_properties(minigzip64 PROPERTIES COMPILE_FLAGS "-D_FILE_OFFSET_BITS=64")
endif()

View File

@ -1,6 +1,6 @@
# This strings autochanged from release_lib.sh:
set(VERSION_DESCRIBE v1.1.54288-testing)
set(VERSION_REVISION 54288)
set(VERSION_DESCRIBE v1.1.54292-testing)
set(VERSION_REVISION 54292)
# end of autochange
set (VERSION_MAJOR 1)

View File

@ -0,0 +1,27 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionSumMap.h>
#include <AggregateFunctions/Helpers.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return std::make_shared<AggregateFunctionSumMap>();
}
}
void registerAggregateFunctionSumMap(AggregateFunctionFactory & factory)
{
factory.registerFunction("sumMap", createAggregateFunctionSumMap);
}
}

View File

@ -0,0 +1,192 @@
#pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Core/FieldVisitors.h>
#include <AggregateFunctions/IBinaryAggregateFunction.h>
#include <Functions/FunctionHelpers.h>
#include <map>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
struct AggregateFunctionSumMapData
{
std::map<Field, Field> merged_maps;
};
/** Aggregate function, that takes two arguments: keys and values, and as a result, builds an array of 2 arrays -
* ordered keys and values summed up by corresponding keys.
*
* This function is the most useful when using SummingMergeTree to sum Nested columns, which name ends in "Map".
*
* Example: sumMap(k, v) of:
* k v
* [1,2,3] [10,10,10]
* [3,4,5] [10,10,10]
* [4,5,6] [10,10,10]
* [6,7,8] [10,10,10]
* [7,5,3] [5,15,25]
* [8,9,10] [20,20,20]
* will return:
* ([1,2,3,4,5,6,7,8,9,10],[10,10,45,20,35,20,15,30,20,20])
*/
class AggregateFunctionSumMap final : public IBinaryAggregateFunction<struct AggregateFunctionSumMapData, AggregateFunctionSumMap>
{
private:
DataTypePtr keys_type;
DataTypePtr values_type;
public:
String getName() const override { return "sumMap"; }
DataTypePtr getReturnType() const override
{
DataTypes types;
types.emplace_back(std::make_shared<DataTypeArray>(keys_type));
types.emplace_back(std::make_shared<DataTypeArray>(values_type));
return std::make_shared<DataTypeTuple>(types);
}
void setArgumentsImpl(const DataTypes & arguments)
{
if (2 != arguments.size())
throw Exception("Aggregate function " + getName() + "require exactly two arguments of array type.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const auto * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
if (!array_type)
throw Exception("First argument for function " + getName() + " must be an array.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
keys_type = array_type->getNestedType();
array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (!array_type)
throw Exception("Second argument for function " + getName() + " must be an array.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
values_type = array_type->getNestedType();
}
void setParameters(const Array & params) override
{
if (!params.empty())
throw Exception("This instantiation of " + getName() + "aggregate function doesn't accept any parameters.",
ErrorCodes::LOGICAL_ERROR);
}
void addImpl(AggregateDataPtr place, const IColumn & column_keys, const IColumn & column_values, size_t row_num, Arena *) const
{
Field field_keys;
column_keys.get(row_num, field_keys);
const auto & keys = field_keys.get<Array &>();
Field field_values;
column_values.get(row_num, field_values);
const auto & values = field_values.get<Array &>();
auto & merged_maps = this->data(place).merged_maps;
if (keys.size() != values.size())
throw Exception("Sizes of keys and values arrays do not match", ErrorCodes::LOGICAL_ERROR);
size_t size = keys.size();
for (size_t i = 0; i < size; ++i)
{
if (merged_maps.find(keys[i]) != merged_maps.end())
applyVisitor(FieldVisitorSum(values[i]), merged_maps[keys[i]]);
else
merged_maps[keys[i]] = values[i];
}
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & merged_maps = this->data(place).merged_maps;
const auto & rhs_maps = this->data(rhs).merged_maps;
for (const auto &rhs_map : rhs_maps)
{
if (merged_maps.find(rhs_map.first) != merged_maps.end())
applyVisitor(FieldVisitorSum(rhs_map.second), merged_maps[rhs_map.first]);
else
merged_maps[rhs_map.first] = rhs_map.second;
}
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const auto & merged_maps = this->data(place).merged_maps;
size_t size = merged_maps.size();
writeVarUInt(size, buf);
for (const auto &v : merged_maps)
{
keys_type->serializeBinary(v.first, buf);
values_type->serializeBinary(v.second, buf);
}
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
{
auto & merged_maps = this->data(place).merged_maps;
size_t size = 0;
readVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
{
Field key, value;
keys_type->deserializeBinary(key, buf);
values_type->deserializeBinary(value, buf);
merged_maps[key] = value;
}
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
auto & to_cols = static_cast<ColumnTuple &>(to).getColumns();
auto & to_keys_arr = static_cast<ColumnArray &>(*to_cols[0]);
auto & to_values_arr = static_cast<ColumnArray &>(*to_cols[1]);
auto & to_keys_col = to_keys_arr.getData();
auto & to_keys_offsets = to_keys_arr.getOffsets();
auto & to_values_col = to_values_arr.getData();
auto & to_values_offsets = to_values_arr.getOffsets();
const auto & merged_maps = this->data(place).merged_maps;
size_t size = merged_maps.size();
to_keys_col.reserve(size);
to_values_col.reserve(size);
for (const auto &v : merged_maps)
{
to_keys_col.insert(v.first);
to_values_col.insert(v.second);
}
to_keys_offsets.push_back((to_keys_offsets.empty() ? 0 : to_keys_offsets.back()) + size);
to_values_offsets.push_back((to_values_offsets.empty() ? 0 : to_values_offsets.back()) + size);
}
const char * getHeaderFilePath() const override { return __FILE__; }
};
}

View File

@ -20,6 +20,7 @@ void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory & factory)
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory & factory);
void registerAggregateFunctionsStatistics(AggregateFunctionFactory & factory);
void registerAggregateFunctionSum(AggregateFunctionFactory & factory);
void registerAggregateFunctionSumMap(AggregateFunctionFactory & factory);
void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory);
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory & factory);
void registerAggregateFunctionTopK(AggregateFunctionFactory & factory);
@ -45,6 +46,7 @@ void registerAggregateFunctions()
registerAggregateFunctionsMinMaxAny(factory);
registerAggregateFunctionsStatistics(factory);
registerAggregateFunctionSum(factory);
registerAggregateFunctionSumMap(factory);
registerAggregateFunctionsUniq(factory);
registerAggregateFunctionUniqUpTo(factory);
registerAggregateFunctionTopK(factory);

View File

@ -124,8 +124,9 @@
M(DictCacheLockReadNs) \
\
M(DistributedSyncInsertionTimeoutExceeded) \
M(DataAfterMergeDiffersFromReplica)
M(DataAfterMergeDiffersFromReplica) \
M(PolygonsAddedToPool) \
M(PolygonsInPoolAllocatedBytes) \
namespace ProfileEvents
{

View File

@ -1,5 +1,6 @@
#pragma once
#include <common/Types.h>
namespace Poco
{

View File

@ -14,6 +14,7 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_CONVERT_TYPE;
extern const int LOGICAL_ERROR;
}
@ -299,4 +300,23 @@ public:
bool operator() (const Tuple & l, const Tuple & r) const { return l < r; }
};
/** Implements `+=` operation.
* Returns false if the result is zero.
*/
class FieldVisitorSum : public StaticVisitor<bool>
{
private:
const Field & rhs;
public:
explicit FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {}
bool operator() (UInt64 & x) const { x += get<UInt64>(rhs); return x != 0; }
bool operator() (Int64 & x) const { x += get<Int64>(rhs); return x != 0; }
bool operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }
bool operator() (Null & x) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
bool operator() (String & x) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Array & x) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
};
}

View File

@ -248,27 +248,6 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std:
finished = true;
}
/** Implements `+=` operation.
* Returns false if the result is zero.
*/
class FieldVisitorSum : public StaticVisitor<bool>
{
private:
const Field & rhs;
public:
explicit FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {}
bool operator() (UInt64 & x) const { x += get<UInt64>(rhs); return x != 0; }
bool operator() (Int64 & x) const { x += get<Int64>(rhs); return x != 0; }
bool operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }
bool operator() (Null & x) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
bool operator() (String & x) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Array & x) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
};
template <typename TSortCursor>
bool SummingSortedBlockInputStream::mergeMaps(Row & row, TSortCursor & cursor)
{

View File

@ -1,5 +1,7 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsGeo.h>
#include <Functions/GeoUtils.h>
#include <Functions/ObjectPool.h>
#include <boost/geometry.hpp>
#include <boost/geometry/geometries/point_xy.hpp>
@ -9,36 +11,82 @@
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnTuple.h>
#include <IO/WriteHelpers.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnArray.h>
#include <Common/ProfileEvents.h>
namespace ProfileEvents
{
extern const Event PolygonsAddedToPool;
extern const Event PolygonsInPoolAllocatedBytes;
}
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace FunctionPointInPolygonDetail
{
template <typename... Args>
using PointInPolygonCrossing = boost::geometry::strategy::within::crossings_multiply<Args...>;
template <typename... Args>
using PointInPolygonWinding = boost::geometry::strategy::within::winding<Args...>;
template <typename... Args>
using PointInPolygonFranklin = boost::geometry::strategy::within::franklin<Args...>;
template <typename Polygon, typename PointInPolygonImpl>
ColumnPtr callPointInPolygonImplWithPool(const IColumn & x, const IColumn & y, Polygon & polygon)
{
using Pool = ObjectPoolMap<PointInPolygonImpl, std::string>;
/// C++11 has thread-safe function-local statics on most modern compilers.
static Pool known_polygons;
template <template <typename...> typename Strategy>
auto factory = [& polygon]()
{
GeoUtils::normalizePolygon(polygon);
auto ptr = std::make_unique<PointInPolygonImpl>(polygon);
/// To allocate memory.
ptr->init();
ProfileEvents::increment(ProfileEvents::PolygonsAddedToPool);
ProfileEvents::increment(ProfileEvents::PolygonsInPoolAllocatedBytes, ptr->getAllocatedBytes());
return ptr.release();
};
std::string serialized_polygon = GeoUtils::serialize(polygon);
auto impl = known_polygons.get(serialized_polygon, factory);
return GeoUtils::pointInPolygon(x, y, *impl);
}
template <typename Polygon, typename PointInPolygonImpl>
ColumnPtr callPointInPolygonImpl(const IColumn & x, const IColumn & y, Polygon & polygon)
{
PointInPolygonImpl impl(polygon);
return GeoUtils::pointInPolygon(x, y, impl);
}
}
template <template <typename> typename PointInPolygonImpl, bool use_object_pool = false>
class FunctionPointInPolygon : public IFunction
{
private:
using CoordinateType = Float64;
public:
template <typename Type>
using Point = boost::geometry::model::d2::point_xy<Type>;
template <typename Type>
using Polygon = boost::geometry::model::polygon<Point<Type>, false>;
template <typename Type>
using Box = boost::geometry::model::box<Point<Type>>;
static const char * name;
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionPointInPolygon<Strategy>>();
return std::make_shared<FunctionPointInPolygon<PointInPolygonImpl, use_object_pool>>();
}
String getName() const override
@ -56,132 +104,173 @@ public:
return 0;
}
void getReturnTypeAndPrerequisitesImpl(
const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, ExpressionActions::Actions & out_prerequisites) override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
{
throw Exception("Too few arguments", ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION);
}
for (size_t i = 0; i < arguments.size(); ++i)
{
const DataTypeTuple * tuple = checkAndGetDataType<DataTypeTuple>(&*arguments[i].type);
if (tuple == nullptr)
{
throw Exception("Argument " + toString(i + 1) + " for function " + getName() + " must be tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
const DataTypes & elems = tuple->getElements();
if (elems.size() != 2)
{
throw Exception("Tuple in argument " + toString(i + 1) + " must have exactly two elements.", ErrorCodes::BAD_ARGUMENTS);
}
for (size_t elems_index = 0; elems_index < elems.size(); ++elems_index)
{
if (!checkDataType<DataTypeNumber<CoordinateType>>(&*elems[elems_index]))
{
throw Exception("Tuple element " + toString(elems_index + 1) + " in argument " + toString(i + 1)
+ " must be " + TypeName<CoordinateType>::get() + ".", ErrorCodes::BAD_ARGUMENTS);
}
}
}
out_return_type = std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
using PointType = boost::geometry::model::d2::point_xy<CoordinateType>;
using PolygonType = boost::geometry::model::polygon<PointType>;
std::pair<CoordinateType, CoordinateType> min, max;
std::vector<PointType> polygon_points(arguments.size() - 1);
auto getMsgPrefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
for (size_t i = 1; i < arguments.size(); ++i)
{
auto const_tuple_col = checkAndGetColumnConst<ColumnTuple>(block.getByPosition(arguments[i]).column.get());
if (!const_tuple_col)
auto * array = checkAndGetDataType<DataTypeArray>(arguments[i].get());
if (array == nullptr && i != 1)
throw Exception(getMsgPrefix(i) + " must be array of tuples.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto * tuple = checkAndGetDataType<DataTypeTuple>(array ? array->getNestedType().get() : arguments[i].get());
if (tuple == nullptr)
throw Exception(getMsgPrefix(i) + " must contains tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypes & elements = tuple->getElements();
if (elements.size() != 2)
throw Exception(getMsgPrefix(i) + " must have exactly two elements.", ErrorCodes::BAD_ARGUMENTS);
for (auto j : ext::range(0, elements.size()))
{
throw Exception("Argument " + toString(i + 1) + " for function " + getName() + " must be constant tuple.", ErrorCodes::ILLEGAL_COLUMN);
if (!elements[j]->isNumeric())
{
throw Exception(getMsgPrefix(i) + " must contains numeric tuple at position " + toString(j + 1),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
}
TupleBackend data = const_tuple_col->getValue<Tuple>();
const CoordinateType x = data[0].get<Float64>();
const CoordinateType y = data[1].get<Float64>();
polygon_points[i - 1] = PointType(x, y);
if (i == 1)
{
min.first = x;
min.second = y;
max.first = x;
max.second = y;
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const IColumn * point_col = block.getByPosition(arguments[0]).column.get();
auto const_tuple_col = checkAndGetColumn<ColumnConst>(point_col);
if (const_tuple_col)
point_col = &const_tuple_col->getDataColumn();
auto tuple_col = checkAndGetColumn<ColumnTuple>(point_col);
if (!tuple_col)
{
throw Exception("First argument for function " + getName() + " must be constant array of tuples.",
ErrorCodes::ILLEGAL_COLUMN);
}
const auto & tuple_block = tuple_col->getData();
const auto & x = tuple_block.safeGetByPosition(0);
const auto & y = tuple_block.safeGetByPosition(1);
bool use_float64 = checkDataType<DataTypeFloat64>(x.type.get()) || checkDataType<DataTypeFloat64>(y.type.get());
auto & result_column = block.safeGetByPosition(result).column;
if (use_float64)
result_column = executeForType<Float64>(*x.column, *y.column, block, arguments);
else
result_column = executeForType<Float32>(*x.column, *y.column, block, arguments);
if (const_tuple_col)
result_column = std::make_shared<ColumnConst>(result_column, const_tuple_col->size());
}
private:
Float64 getCoordinateFromField(const Field & field)
{
min.first = std::min(min.first, x);
max.first = std::max(max.first, x);
min.second = std::min(min.second, y);
max.second = std::max(max.second, y);
}
}
PolygonType polygon;
boost::geometry::assign_points(polygon, polygon_points);
Strategy<PointType> strategy;
auto point_checker = [&](CoordinateType x, CoordinateType y) -> bool
switch (field.getType())
{
if (x < min.first || x > max.first || y < min.second || y > max.second)
return false;
PointType point(x, y);
return boost::geometry::covered_by(point, polygon, strategy);
};
size_t rows = block.rows();
auto point_column_const = checkAndGetColumnConst<ColumnTuple>(block.getByPosition(arguments[0]).column.get());
if (point_column_const)
case Field::Types::Float64:
return field.get<Float64>();
case Field::Types::Int64:
return field.get<Int64>();
case Field::Types::UInt64:
return field.get<UInt64>();
default:
{
TupleBackend data = point_column_const->getValue<Tuple>();
const CoordinateType point_x = data[0].get<Float64>();
const CoordinateType point_y = data[1].get<Float64>();
UInt8 value = point_checker(point_x, point_y);
block.getByPosition(result).column = DataTypeUInt8().createConstColumn(rows, UInt64(value));
return;
std::string msg = "Expected numeric field, but got ";
throw Exception(msg + Field::Types::toString(field.getType()), ErrorCodes::LOGICAL_ERROR);
}
}
}
auto & res = block.getByPosition(result);
res.column = std::make_shared<ColumnUInt8>(rows);
IColumn & result_column = *res.column;
auto & result_data = static_cast<ColumnUInt8 &>(result_column).getData();
auto point_column = checkAndGetColumn<ColumnTuple>(block.getByPosition(arguments[0]).column.get());
auto column_x = checkAndGetColumn<ColumnVector<CoordinateType>>(point_column->getData().getByPosition(0).column.get());
auto column_y = checkAndGetColumn<ColumnVector<CoordinateType>>(point_column->getData().getByPosition(1).column.get());
for (size_t i = 0; i < rows; ++i)
template <typename Type>
ColumnPtr executeForType(const IColumn & x, const IColumn & y, Block & block, const ColumnNumbers & arguments)
{
const CoordinateType point_x = column_x->getElement(i);
const CoordinateType point_y = column_y->getElement(i);
result_data[i] = point_checker(point_x, point_y);
Polygon<Type> polygon;
auto getMsgPrefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); };
for (size_t i = 1; i < arguments.size(); ++i)
{
auto const_col = checkAndGetColumn<ColumnConst>(block.getByPosition(arguments[i]).column.get());
auto array_col = const_col ? checkAndGetColumn<ColumnArray>(&const_col->getDataColumn()) : nullptr;
auto tuple_col = array_col ? checkAndGetColumn<ColumnTuple>(&array_col->getData()) : nullptr;
if (!tuple_col)
throw Exception(getMsgPrefix(i) + " must be constant array of tuples.", ErrorCodes::ILLEGAL_COLUMN);
const auto & tuple_block = tuple_col->getData();
const auto & column_x = tuple_block.safeGetByPosition(0).column;
const auto & column_y = tuple_block.safeGetByPosition(1).column;
if (!polygon.outer().empty())
polygon.inners().emplace_back();
auto & container = polygon.outer().empty() ? polygon.outer() : polygon.inners().back();
auto size = column_x->size();
if (size == 0)
throw Exception(getMsgPrefix(i) + " shouldn't be empty.", ErrorCodes::ILLEGAL_COLUMN);
for (auto j : ext::range(0, size))
{
Type x_coord = getCoordinateFromField((*column_x)[j]);
Type y_coord = getCoordinateFromField((*column_y)[j]);
container.push_back(Point<Type>(x_coord, y_coord));
}
/// Polygon assumed to be closed. Allow user to escape repeating of first point.
if (!boost::geometry::equals(container.front(), container.back()))
container.push_back(container.front());
}
auto callImpl = use_object_pool
? FunctionPointInPolygonDetail::callPointInPolygonImplWithPool<Polygon<Type>, PointInPolygonImpl<Type>>
: FunctionPointInPolygonDetail::callPointInPolygonImpl<Polygon<Type>, PointInPolygonImpl<Type>>;
return callImpl(x, y, polygon);
}
};
template <typename Type>
using Point = boost::geometry::model::d2::point_xy<Type>;
template <typename Type>
using PointInPolygonCrossingStrategy = boost::geometry::strategy::within::crossings_multiply<Point<Type>>;
template <typename Type>
using PointInPolygonWindingStrategy = boost::geometry::strategy::within::winding<Point<Type>>;
template <typename Type>
using PointInPolygonFranklinStrategy = boost::geometry::strategy::within::franklin<Point<Type>>;
template <typename Type>
using PointInPolygonCrossing = GeoUtils::PointInPolygon<PointInPolygonCrossingStrategy<Type>, Type>;
template <typename Type>
using PointInPolygonWinding = GeoUtils::PointInPolygon<PointInPolygonWindingStrategy<Type>, Type>;
template <typename Type>
using PointInPolygonFranklin = GeoUtils::PointInPolygon<PointInPolygonFranklinStrategy<Type>, Type>;
template <typename Type>
using PointInPolygonWithGrid = GeoUtils::PointInPolygonWithGrid<Type>;
template <>
const char * FunctionPointInPolygon<PointInPolygonCrossing>::name = "pointInPolygon";
const char * FunctionPointInPolygon<PointInPolygonCrossing>::name = "pointInPolygonCrossing";
template <>
const char * FunctionPointInPolygon<PointInPolygonWinding>::name = "pointInPolygonWinding";
template <>
const char * FunctionPointInPolygon<PointInPolygonFranklin>::name = "pointInPolygonFranklin";
template <>
const char * FunctionPointInPolygon<PointInPolygonWithGrid, true>::name = "pointInPolygon";
void registerFunctionsGeo(FunctionFactory & factory)
@ -192,5 +281,6 @@ void registerFunctionsGeo(FunctionFactory & factory)
factory.registerFunction<FunctionPointInPolygon<PointInPolygonFranklin>>();
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWinding>>();
factory.registerFunction<FunctionPointInPolygon<PointInPolygonCrossing>>();
factory.registerFunction<FunctionPointInPolygon<PointInPolygonWithGrid, true>>();
}
}

View File

@ -290,6 +290,8 @@ inline float roundWithMode(float x, RoundingMode mode)
case RoundingMode::Floor: return floorf(x);
case RoundingMode::Ceil: return ceilf(x);
case RoundingMode::Trunc: return truncf(x);
default:
throw Exception("Logical error: unexpected 'mode' parameter passed to function roundWithMode", ErrorCodes::LOGICAL_ERROR);
}
}
@ -301,6 +303,8 @@ inline double roundWithMode(double x, RoundingMode mode)
case RoundingMode::Floor: return floor(x);
case RoundingMode::Ceil: return ceil(x);
case RoundingMode::Trunc: return trunc(x);
default:
throw Exception("Logical error: unexpected 'mode' parameter passed to function roundWithMode", ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -0,0 +1,670 @@
#pragma once
#include <Core/Types.h>
#include <Core/Defines.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnVector.h>
#include <Core/TypeListNumber.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
/// Warning in boost::geometry during template strategy substitution.
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
#include <boost/geometry.hpp>
#if !__clang__
#pragma GCC diagnostic pop
#endif
#include <boost/geometry/geometries/point_xy.hpp>
#include <boost/geometry/geometries/polygon.hpp>
#include <boost/geometry/geometries/multi_polygon.hpp>
#include <boost/geometry/geometries/segment.hpp>
#include <boost/geometry/algorithms/comparable_distance.hpp>
#include <boost/geometry/strategies/cartesian/distance_pythagoras.hpp>
#include <array>
#include <vector>
#include <iterator>
#include <cmath>
#include <algorithm>
#include <IO/WriteBufferFromString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace GeoUtils
{
template <typename Polygon>
UInt64 getPolygonAllocatedBytes(const Polygon & polygon)
{
UInt64 size = 0;
using RingType = typename Polygon::ring_type;
using ValueType = typename RingType::value_type;
auto sizeOfRing = [](const RingType & ring) { return sizeof(ring) + ring.capacity() * sizeof(ValueType); };
size += sizeOfRing(polygon.outer());
const auto & inners = polygon.inners();
size += sizeof(inners) + inners.capacity() * sizeof(RingType);
for (auto & inner : inners)
size += sizeOfRing(inner);
return size;
}
template <typename MultiPolygon>
UInt64 getMultiPolygonAllocatedBytes(const MultiPolygon & multi_polygon)
{
using ValueType = typename MultiPolygon::value_type;
UInt64 size = multi_polygon.capacity() * sizeof(ValueType);
for (const auto & polygon : multi_polygon)
size += getPolygonAllocatedBytes(polygon);
return size;
}
template <typename CoordinateType = Float32>
class PointInPolygonWithGrid
{
public:
using Point = boost::geometry::model::d2::point_xy<CoordinateType>;
/// Counter-Clockwise ordering.
using Polygon = boost::geometry::model::polygon<Point, false>;
using MultiPolygon = boost::geometry::model::multi_polygon<Polygon>;
using Box = boost::geometry::model::box<Point>;
using Segment = boost::geometry::model::segment<Point>;
explicit PointInPolygonWithGrid(const Polygon & polygon, UInt16 grid_size = 8)
: grid_size(std::max<UInt16>(1, grid_size)), polygon(polygon) {}
void init();
/// True if bound box is empty.
bool hasEmptyBound() const { return has_empty_bound; }
UInt64 getAllocatedBytes() const;
inline bool ALWAYS_INLINE contains(CoordinateType x, CoordinateType y);
private:
enum class CellType
{
inner,
outer,
singleLine,
pairOfLinesSinglePolygon,
pairOfLinesDifferentPolygons,
complexPolygon
};
struct HalfPlane
{
/// Line, a * x + b * y + c = 0. Vector (a, b) points inside half-plane.
CoordinateType a;
CoordinateType b;
CoordinateType c;
/// Take left half-plane.
void fill(const Point & from, const Point & to)
{
a = -(to.y() - from.y());
b = to.x() - from.x();
c = -from.x() * a - from.y() * b;
}
/// Inner part of the HalfPlane is the left side of initialized vector.
bool ALWAYS_INLINE contains(CoordinateType x, CoordinateType y) const { return a * x + b * y + c >= 0; }
};
struct Cell
{
static const int max_stored_half_planes = 2;
HalfPlane half_planes[max_stored_half_planes];
size_t index_of_inner_polygon;
CellType type;
};
const UInt16 grid_size;
Polygon polygon;
std::vector<Cell> cells;
std::vector<MultiPolygon> polygons;
CoordinateType cell_width;
CoordinateType cell_height;
CoordinateType x_shift;
CoordinateType y_shift;
CoordinateType x_scale;
CoordinateType y_scale;
bool has_empty_bound = false;
bool was_grid_built = false;
void buildGrid();
void calcGridAttributes(Box & box);
template <typename T>
T ALWAYS_INLINE getCellIndex(T row, T col) const { return row * grid_size + col; }
/// Complex case. Will check intersection directly.
inline void addComplexPolygonCell(size_t index, const Box & box);
/// Empty intersection or intersection == box.
inline void addCell(size_t index, const Box & empty_box);
/// Intersection is a single polygon.
inline void addCell(size_t index, const Box & box, const Polygon & intersection);
/// Intersection is a pair of polygons.
inline void addCell(size_t index, const Box & box, const Polygon & first, const Polygon & second);
/// Returns a list of half-planes were formed from intersection edges without box edges.
inline std::vector<HalfPlane> findHalfPlanes(const Box & box, const Polygon & intersection);
using Distance = typename boost::geometry::default_comparable_distance_result<Point, Segment>::type;
/// min(distance(point, edge) : edge in polygon)
inline Distance distance(const Point & point, const Polygon & polygon);
};
template <typename CoordinateType>
UInt64 PointInPolygonWithGrid<CoordinateType>::getAllocatedBytes() const
{
UInt64 size = sizeof(*this);
size += cells.capacity() * sizeof(Cell);
size += polygons.capacity() * sizeof(MultiPolygon);
size += getPolygonAllocatedBytes(polygon);
for (const auto & polygon : polygons)
size += getMultiPolygonAllocatedBytes(polygon);
return size;
}
template <typename CoordinateType>
void PointInPolygonWithGrid<CoordinateType>::init()
{
if (!was_grid_built)
buildGrid();
was_grid_built = true;
}
template <typename CoordinateType>
void PointInPolygonWithGrid<CoordinateType>::calcGridAttributes(
PointInPolygonWithGrid<CoordinateType>::Box & box)
{
boost::geometry::envelope(polygon, box);
const Point & min_corner = box.min_corner();
const Point & max_corner = box.max_corner();
cell_width = (max_corner.x() - min_corner.x()) / grid_size;
cell_height = (max_corner.y() - min_corner.y()) / grid_size;
if (cell_width == 0 || cell_height == 0)
{
has_empty_bound = true;
return;
}
x_scale = 1 / cell_width;
y_scale = 1 / cell_height;
x_shift = -min_corner.x();
y_shift = -min_corner.y();
}
template <typename CoordinateType>
void PointInPolygonWithGrid<CoordinateType>::buildGrid()
{
Box box;
calcGridAttributes(box);
if (has_empty_bound)
return;
cells.assign(grid_size * grid_size, {});
const Point & min_corner = box.min_corner();
for (size_t row = 0; row < grid_size; ++row)
{
CoordinateType y_min = min_corner.y() + row * cell_height;
CoordinateType y_max = min_corner.y() + (row + 1) * cell_height;
for (size_t col = 0; col < grid_size; ++col)
{
CoordinateType x_min = min_corner.x() + col * cell_width;
CoordinateType x_max = min_corner.x() + (col + 1) * cell_width;
Box cell_box(Point(x_min, y_min), Point(x_max, y_max));
Polygon cell_bound;
boost::geometry::convert(cell_box, cell_bound);
MultiPolygon intersection;
boost::geometry::intersection(polygon, cell_bound, intersection);
size_t cellIndex = getCellIndex(row, col);
if (intersection.empty())
addCell(cellIndex, cell_box);
else if (intersection.size() == 1)
addCell(cellIndex, cell_box, intersection.front());
else if (intersection.size() == 2)
addCell(cellIndex, cell_box, intersection.front(), intersection.back());
else
addComplexPolygonCell(cellIndex, cell_box);
}
}
}
template <typename CoordinateType>
bool PointInPolygonWithGrid<CoordinateType>::contains(CoordinateType x, CoordinateType y)
{
if (has_empty_bound)
return false;
CoordinateType float_row = (y + y_shift) * y_scale;
CoordinateType float_col = (x + x_shift) * x_scale;
if (float_row < 0 || float_row > grid_size)
return false;
if (float_col < 0 || float_col > grid_size)
return false;
int row = std::min<int>(float_row, grid_size - 1);
int col = std::min<int>(float_col, grid_size - 1);
int index = getCellIndex(row, col);
const auto & cell = cells[index];
switch (cell.type)
{
case CellType::inner:
return true;
case CellType::outer:
return false;
case CellType::singleLine:
return cell.half_planes[0].contains(x, y);
case CellType::pairOfLinesSinglePolygon:
return cell.half_planes[0].contains(x, y) && cell.half_planes[1].contains(x, y);
case CellType::pairOfLinesDifferentPolygons:
return cell.half_planes[0].contains(x, y) || cell.half_planes[1].contains(x, y);
case CellType::complexPolygon:
return boost::geometry::within(Point(x, y), polygons[cell.index_of_inner_polygon]);
default:
return false;
}
}
template <typename CoordinateType>
typename PointInPolygonWithGrid<CoordinateType>::Distance
PointInPolygonWithGrid<CoordinateType>::distance(
const PointInPolygonWithGrid<CoordinateType>::Point & point,
const PointInPolygonWithGrid<CoordinateType>::Polygon & polygon)
{
const auto & outer = polygon.outer();
Distance distance = 0;
for (auto i : ext::range(0, outer.size() - 1))
{
Segment segment(outer[i], outer[i + 1]);
Distance current = boost::geometry::comparable_distance(point, segment);
distance = i ? std::min(current, distance) : current;
}
return distance;
}
template <typename CoordinateType>
std::vector<typename PointInPolygonWithGrid<CoordinateType>::HalfPlane>
PointInPolygonWithGrid<CoordinateType>::findHalfPlanes(
const PointInPolygonWithGrid<CoordinateType>::Box & box,
const PointInPolygonWithGrid<CoordinateType>::Polygon & intersection)
{
std::vector<HalfPlane> half_planes;
Polygon bound;
boost::geometry::convert(box, bound);
const auto & outer = intersection.outer();
for (auto i : ext::range(0, outer.size() - 1))
{
/// Want to detect is intersection edge was formed from box edge or from polygon edge.
/// If center of the edge closer to box, than don't form the half-plane.
Segment segment(outer[i], outer[i + 1]);
Point center((segment.first.x() + segment.second.x()) / 2, (segment.first.y() + segment.second.y()) / 2);
if (distance(center, polygon) < distance(center, bound))
{
half_planes.push_back({});
half_planes.back().fill(segment.first, segment.second);
}
}
return half_planes;
}
template <typename CoordinateType>
void PointInPolygonWithGrid<CoordinateType>::addComplexPolygonCell(
size_t index, const PointInPolygonWithGrid<CoordinateType>::Box & box)
{
cells[index].type = CellType::complexPolygon;
cells[index].index_of_inner_polygon = polygons.size();
/// Expand box in (1 + eps_factor) times to eliminate errors for points on box bound.
static constexpr float eps_factor = 0.01;
float x_eps = eps_factor * (box.max_corner().x() - box.min_corner().x());
float y_eps = eps_factor * (box.max_corner().y() - box.min_corner().y());
Point min_corner(box.min_corner().x() - x_eps, box.min_corner().y() - y_eps);
Point max_corner(box.max_corner().x() + x_eps, box.max_corner().y() + y_eps);
Box box_with_eps_bound(min_corner, max_corner);
Polygon bound;
boost::geometry::convert(box_with_eps_bound, bound);
MultiPolygon intersection;
boost::geometry::intersection(polygon, bound, intersection);
polygons.push_back(intersection);
}
template <typename CoordinateType>
void PointInPolygonWithGrid<CoordinateType>::addCell(
size_t index, const PointInPolygonWithGrid<CoordinateType>::Box & empty_box)
{
const auto & min_corner = empty_box.min_corner();
const auto & max_corner = empty_box.max_corner();
Point center((min_corner.x() + max_corner.x()) / 2, (min_corner.y() + max_corner.y()) / 2);
if (boost::geometry::within(center, polygon))
cells[index].type = CellType::inner;
else
cells[index].type = CellType::outer;
}
template <typename CoordinateType>
void PointInPolygonWithGrid<CoordinateType>::addCell(
size_t index,
const PointInPolygonWithGrid<CoordinateType>::Box & box,
const PointInPolygonWithGrid<CoordinateType>::Polygon & intersection)
{
if (!intersection.inners().empty())
addComplexPolygonCell(index, box);
auto half_planes = findHalfPlanes(box, intersection);
if (half_planes.empty())
addCell(index, box);
else if (half_planes.size() == 1)
{
cells[index].type = CellType::singleLine;
cells[index].half_planes[0] = half_planes[0];
}
else if (half_planes.size() == 2)
{
cells[index].type = CellType::pairOfLinesSinglePolygon;
cells[index].half_planes[0] = half_planes[0];
cells[index].half_planes[1] = half_planes[1];
}
else
addComplexPolygonCell(index, box);
}
template <typename CoordinateType>
void PointInPolygonWithGrid<CoordinateType>::addCell(
size_t index,
const PointInPolygonWithGrid<CoordinateType>::Box & box,
const PointInPolygonWithGrid<CoordinateType>::Polygon & first,
const PointInPolygonWithGrid<CoordinateType>::Polygon & second)
{
if (!first.inners().empty() || !second.inners().empty())
addComplexPolygonCell(index, box);
auto first_half_planes = findHalfPlanes(box, first);
auto second_half_planes = findHalfPlanes(box, second);
if (first_half_planes.empty())
addCell(index, box, first);
else if (second_half_planes.empty())
addCell(index, box, second);
else if (first_half_planes.size() == 1 && second_half_planes.size() == 1)
{
cells[index].type = CellType::pairOfLinesDifferentPolygons;
cells[index].half_planes[0] = first_half_planes[0];
cells[index].half_planes[1] = second_half_planes[0];
}
else
addComplexPolygonCell(index, box);
}
template <typename Strategy, typename CoordinateType = Float32>
class PointInPolygon
{
public:
using Point = boost::geometry::model::d2::point_xy<CoordinateType>;
/// Counter-Clockwise ordering.
using Polygon = boost::geometry::model::polygon<Point, false>;
using Box = boost::geometry::model::box<Point>;
explicit PointInPolygon(const Polygon & polygon) : polygon(polygon) {}
void init()
{
boost::geometry::envelope(polygon, box);
const Point & min_corner = box.min_corner();
const Point & max_corner = box.max_corner();
if (min_corner.x() == max_corner.x() || min_corner.y() == max_corner.y())
has_empty_bound = true;
}
bool hasEmptyBound() const { return has_empty_bound; }
inline bool ALWAYS_INLINE contains(CoordinateType x, CoordinateType y)
{
Point point(x, y);
if (!boost::geometry::within(point, box))
return false;
return boost::geometry::covered_by(point, polygon, strategy);
}
UInt64 getAllocatedBytes() const { return sizeof(*this); }
private:
const Polygon & polygon;
Box box;
bool has_empty_bound = false;
Strategy strategy;
};
/// Algorithms.
template <typename T, typename U, typename PointInPolygonImpl>
ColumnPtr pointInPolygon(const ColumnVector<T> & x, const ColumnVector<U> & y, PointInPolygonImpl && impl)
{
auto size = x.size();
impl.init();
if (impl.hasEmptyBound())
{
return std::make_shared<ColumnVector<UInt8>>(size, 0);
}
auto result = std::make_shared<ColumnVector<UInt8>>(size);
auto & data = result->getData();
const auto & x_data = x.getData();
const auto & y_data = y.getData();
for (auto i : ext::range(0, size))
{
data[i] = static_cast<UInt8>(impl.contains(x_data[i], y_data[i]));
}
return result;
}
template <typename ... Types>
struct CallPointInPolygon;
template <typename Type, typename ... Types>
struct CallPointInPolygon<Type, Types ...>
{
template <typename T, typename PointInPolygonImpl>
static ColumnPtr call(const ColumnVector<T> & x, const IColumn & y, PointInPolygonImpl && impl)
{
if (auto column = typeid_cast<const ColumnVector<Type> *>(&y))
return pointInPolygon(x, *column, impl);
return CallPointInPolygon<Types ...>::template call<T>(x, y, impl);
}
template <typename PointInPolygonImpl>
static ColumnPtr call(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
using Impl = typename ApplyTypeListForClass<CallPointInPolygon, TypeListNumbers>::Type;
if (auto column = typeid_cast<const ColumnVector<Type> *>(&x))
return Impl::template call<Type>(*column, y, impl);
return CallPointInPolygon<Types ...>::call(x, y, impl);
}
};
template <>
struct CallPointInPolygon<>
{
template <typename T, typename PointInPolygonImpl>
static ColumnPtr call(const ColumnVector<T> & x, const IColumn & y, PointInPolygonImpl && impl)
{
throw Exception(std::string("Unknown numeric column type: ") + typeid(y).name(), ErrorCodes::LOGICAL_ERROR);
}
template <typename PointInPolygonImpl>
static ColumnPtr call(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
throw Exception(std::string("Unknown numeric column type: ") + typeid(x).name(), ErrorCodes::LOGICAL_ERROR);
}
};
template <typename PointInPolygonImpl>
ColumnPtr pointInPolygon(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
using Impl = typename ApplyTypeListForClass<CallPointInPolygon, TypeListNumbers>::Type;
return Impl::call(x, y, impl);
}
/// Total angle (signed) between neighbor vectors in linestring. Zero if linestring.size() < 2.
template <typename Linestring>
float calcLinestringRotation(const Linestring & points)
{
using Point = typename std::decay<decltype(*points.begin())>::type;
float rotation = 0;
auto sqrLength = [](const Point & point) { return point.x() * point.x() + point.y() * point.y(); };
auto vecProduct = [](const Point & from, const Point & to) { return from.x() * to.y() - from.y() * to.x(); };
auto getVector = [](const Point & from, const Point & to) -> Point
{
return Point(to.x() - from.x(), to.y() - from.y());
};
for (auto it = points.begin(); std::next(it) != points.end(); ++it)
{
if (it != points.begin())
{
auto prev = std::prev(it);
auto next = std::next(it);
Point from = getVector(*prev, *it);
Point to = getVector(*it, *next);
float sqr_from_len = sqrLength(from);
float sqr_to_len = sqrLength(to);
float sqr_len_product = (sqr_from_len * sqr_to_len);
if (std::isfinite(sqr_len_product))
{
float vec_prod = vecProduct(from, to);
float sin_ang = vec_prod * std::fabs(vec_prod) / sqr_len_product;
sin_ang = std::max(-1.f, std::min(1.f, sin_ang));
rotation += std::asin(sin_ang);
}
}
}
return rotation;
}
/// Make inner linestring counter-clockwise and outers clockwise oriented.
template <typename Polygon>
void normalizePolygon(Polygon && polygon)
{
auto & outer = polygon.outer();
if (calcLinestringRotation(outer) < 0)
std::reverse(outer.begin(), outer.end());
auto & inners = polygon.inners();
for (auto & inner : inners)
if (calcLinestringRotation(inner) > 0)
std::reverse(inner.begin(), inner.end());
}
template <typename Polygon>
std::string serialize(Polygon && polygon)
{
std::string result;
{
WriteBufferFromString buffer(result);
using RingType = typename std::decay<Polygon>::type::ring_type;
auto serializeFloat = [&buffer](float value) { buffer.write(reinterpret_cast<char *>(&value), sizeof(value)); };
auto serializeSize = [&buffer](size_t size) { buffer.write(reinterpret_cast<char *>(&size), sizeof(size)); };
auto serializeRing = [& buffer, & serializeFloat, & serializeSize](const RingType & ring)
{
serializeSize(ring.size());
for (const auto & point : ring)
{
serializeFloat(point.x());
serializeFloat(point.y());
}
};
serializeRing(polygon.outer());
const auto & inners = polygon.inners();
serializeSize(inners.size());
for (auto & inner : inners)
serializeRing(inner);
}
return result;
}
} /// GeoUtils
} /// DB

View File

@ -1,6 +1,7 @@
#include <Common/TypeList.h>
#include <Common/typeid_cast.h>
#include <Interpreters/Aggregator.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
namespace DB

View File

@ -37,6 +37,9 @@ add_library (clickhouse-performance-test PerformanceTest.cpp)
target_link_libraries (clickhouse-performance-test dbms ${Boost_PROGRAM_OPTIONS_LIBRARY})
target_include_directories (clickhouse-performance-test PRIVATE ${PCG_RANDOM_INCLUDE_DIR})
add_library (clickhouse-compressor-lib Compressor.cpp)
target_link_libraries (clickhouse-compressor-lib dbms ${Boost_PROGRAM_OPTIONS_LIBRARY})
add_executable(clickhouse main.cpp)
target_include_directories(clickhouse PRIVATE ${COMMON_INCLUDE_DIR})
target_link_libraries(clickhouse
@ -45,7 +48,8 @@ target_link_libraries(clickhouse
clickhouse-local
clickhouse-benchmark
clickhouse-performance-test
clickhouse-extract-from-config)
clickhouse-extract-from-config
clickhouse-compressor-lib)
INSTALL(TARGETS clickhouse RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse)
# make symbolic links to concrete clickhouse applications
macro(install_symlink_to_clickhouse app)

View File

@ -0,0 +1,124 @@
#include <iostream>
#include <boost/program_options.hpp>
#include <Common/Exception.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/CompressedWriteBuffer.h>
#include <IO/CompressedReadBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_SIZE_COMPRESSED;
}
}
namespace
{
/// Outputs sizes of uncompressed and compressed blocks for compressed file.
void checkAndWriteHeader(DB::ReadBuffer & in, DB::WriteBuffer & out)
{
while (!in.eof())
{
in.ignore(16); /// checksum
char header[COMPRESSED_BLOCK_HEADER_SIZE];
in.readStrict(header, COMPRESSED_BLOCK_HEADER_SIZE);
UInt32 size_compressed = unalignedLoad<UInt32>(&header[1]);
if (size_compressed > DBMS_MAX_COMPRESSED_SIZE)
throw DB::Exception("Too large size_compressed. Most likely corrupted data.", DB::ErrorCodes::TOO_LARGE_SIZE_COMPRESSED);
UInt32 size_decompressed = unalignedLoad<UInt32>(&header[5]);
DB::writeText(size_decompressed, out);
DB::writeChar('\t', out);
DB::writeText(size_compressed, out);
DB::writeChar('\n', out);
in.ignore(size_compressed - COMPRESSED_BLOCK_HEADER_SIZE);
}
}
}
int mainEntryClickHouseCompressor(int argc, char ** argv)
{
boost::program_options::options_description desc("Allowed options");
desc.add_options()
("help,h", "produce help message")
("decompress,d", "decompress")
("block-size,b", boost::program_options::value<unsigned>()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size")
("hc", "use LZ4HC instead of LZ4")
("zstd", "use ZSTD instead of LZ4")
("none", "use no compression instead of LZ4")
("stat", "print block statistics of compressed data")
;
boost::program_options::variables_map options;
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
if (options.count("help"))
{
std::cout << "Usage: " << argv[0] << " [options] < in > out" << std::endl;
std::cout << desc << std::endl;
return 1;
}
try
{
bool decompress = options.count("decompress");
bool use_lz4hc = options.count("hc");
bool use_zstd = options.count("zstd");
bool stat_mode = options.count("stat");
bool use_none = options.count("none");
unsigned block_size = options["block-size"].as<unsigned>();
DB::CompressionMethod method = DB::CompressionMethod::LZ4;
if (use_lz4hc)
method = DB::CompressionMethod::LZ4HC;
else if (use_zstd)
method = DB::CompressionMethod::ZSTD;
else if (use_none)
method = DB::CompressionMethod::NONE;
DB::ReadBufferFromFileDescriptor rb(STDIN_FILENO);
DB::WriteBufferFromFileDescriptor wb(STDOUT_FILENO);
if (stat_mode)
{
/// Output statistic for compressed file.
checkAndWriteHeader(rb, wb);
}
else if (decompress)
{
/// Decompression
DB::CompressedReadBuffer from(rb);
DB::copyData(from, wb);
}
else
{
/// Compression
DB::CompressedWriteBuffer to(wb, method, block_size);
DB::copyData(rb, to);
}
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true);
return DB::getCurrentExceptionCode();
}
return 0;
}

View File

@ -74,7 +74,7 @@ int mainEntryClickHouseExtractFromConfig(int argc, char ** argv)
{
std::cerr << "Preprocess config file and extract value of the given key." << std::endl
<< std::endl;
std::cerr << "Usage: clickhouse --extract-from-config [options]" << std::endl
std::cerr << "Usage: clickhouse extract-from-config [options]" << std::endl
<< std::endl;
std::cerr << options_desc << std::endl;
return 0;

View File

@ -35,6 +35,8 @@
#include <Interpreters/Quota.h>
#include <Common/typeid_cast.h>
#include <Poco/Net/HTTPStream.h>
#include "HTTPHandler.h"
namespace DB
@ -213,11 +215,16 @@ void HTTPHandler::processQuery(
if (!query_param.empty())
query_param += '\n';
/// User name and password can be passed using query parameters or using HTTP Basic auth (both methods are insecure).
/// The user and password can be passed by headers (similar to X-Auth-*), which is used by load balancers to pass authentication information
std::string user = request.get("X-ClickHouse-User", params.get("user", "default"));
std::string password = request.get("X-ClickHouse-Key", params.get("password", ""));
/// The user and password can be passed by headers (similar to X-Auth-*),
/// which is used by load balancers to pass authentication information.
std::string user = request.get("X-ClickHouse-User", "");
std::string password = request.get("X-ClickHouse-Key", "");
std::string quota_key = request.get("X-ClickHouse-Quota", "");
if (user.empty() && password.empty() && quota_key.empty())
{
/// User name and password can be passed using query parameters
/// or using HTTP Basic auth (both methods are insecure).
if (request.hasCredentials())
{
Poco::Net::HTTPBasicCredentials credentials(request);
@ -225,8 +232,26 @@ void HTTPHandler::processQuery(
user = credentials.getUsername();
password = credentials.getPassword();
}
else
{
user = params.get("user", "default");
password = params.get("password", "");
}
quota_key = params.get("quota_key", "");
}
else
{
/// It is prohibited to mix different authorization schemes.
if (request.hasCredentials()
|| params.has("user")
|| params.has("password")
|| params.has("quota_key"))
{
throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD);
}
}
std::string quota_key = request.get("X-ClickHouse-Quota", params.get("quota_key", ""));
std::string query_id = params.get("query_id", "");
const auto & config = server.config();
@ -354,7 +379,14 @@ void HTTPHandler::processQuery(
std::unique_ptr<ReadBuffer> in_param = std::make_unique<ReadBufferFromString>(query_param);
std::unique_ptr<ReadBuffer> in_post_raw = std::make_unique<ReadBufferFromIStream>(istr);
std::unique_ptr<ReadBuffer> in_post_raw;
/// A grubby workaround for CLICKHOUSE-3333 problem. This condition should detect POST query with empty body.
/// In that case Poco doesn't work properly and returns HTTPInputStream which just listen TCP connection.
/// NOTE: if Poco are updated, this heuristic might not work properly.
if (typeid_cast<Poco::Net::HTTPInputStream *>(&istr) == nullptr)
in_post_raw = std::make_unique<ReadBufferFromIStream>(istr);
else
in_post_raw = std::make_unique<ReadBufferFromString>(String()); // will read empty body.
/// Request body can be compressed using algorithm specified in the Content-Encoding header.
std::unique_ptr<ReadBuffer> in_post;

View File

@ -13,26 +13,56 @@ int mainEntryClickHouseLocal(int argc, char ** argv);
int mainEntryClickHouseBenchmark(int argc, char ** argv);
int mainEntryClickHousePerformanceTest(int argc, char ** argv);
int mainEntryClickHouseExtractFromConfig(int argc, char ** argv);
int mainEntryClickHouseCompressor(int argc, char ** argv);
static bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
namespace
{
std::string arg_mode_app = "--" + app_suffix;
/// Use app if --app arg is passed (the arg should be quietly removed)
auto arg_it = std::find_if(argv.begin(), argv.end(), [&](const char * arg) { return !arg_mode_app.compare(arg); } );
if (arg_it != argv.end())
using MainFunc = int (*)(int, char**);
/// Add an item here to register new application
std::pair<const char *, MainFunc> clickhouse_applications[] =
{
{"local", mainEntryClickHouseLocal},
{"client", mainEntryClickHouseClient},
{"benchmark", mainEntryClickHouseBenchmark},
{"server", mainEntryClickHouseServer},
{"performance-test", mainEntryClickHousePerformanceTest},
{"extract-from-config", mainEntryClickHouseExtractFromConfig},
{"compressor", mainEntryClickHouseCompressor}
};
int printHelp(int argc, char ** argv)
{
std::cerr << "Use one of the following commands:" << std::endl;
for (auto & application : clickhouse_applications)
std::cerr << "clickhouse " << application.first << " [args] " << std::endl;
return -1;
};
bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
{
/// Use app if the first arg 'app' is passed (the arg should be quietly removed)
if (argv.size() >= 2)
{
argv.erase(arg_it);
auto first_arg = argv.begin() + 1;
/// 'clickhouse --client ...' and 'clickhouse client ...' are Ok
if (*first_arg == "--" + app_suffix || *first_arg == app_suffix)
{
argv.erase(first_arg);
return true;
}
std::string app_name = "clickhouse-" + app_suffix;
}
/// Use app if clickhouse binary is run through symbolic link with name clickhouse-app
if (!argv.empty() && (!app_name.compare(argv[0]) || endsWith(argv[0], "/" + app_name)))
return true;
std::string app_name = "clickhouse-" + app_suffix;
return !argv.empty() && (app_name == argv[0] || endsWith(argv[0], "/" + app_name));
}
return false;
}
@ -44,20 +74,17 @@ int main(int argc_, char ** argv_)
std::vector<char *> argv(argv_, argv_ + argc_);
auto main_func = mainEntryClickHouseServer;
/// Print a basic help if nothing was matched
MainFunc main_func = printHelp;
if (isClickhouseApp("local", argv))
main_func = mainEntryClickHouseLocal;
else if (isClickhouseApp("client", argv))
main_func = mainEntryClickHouseClient;
else if (isClickhouseApp("benchmark", argv))
main_func = mainEntryClickHouseBenchmark;
else if (isClickhouseApp("server", argv)) /// --server arg should be cut
main_func = mainEntryClickHouseServer;
else if (isClickhouseApp("performance-test", argv))
main_func = mainEntryClickHousePerformanceTest;
else if (isClickhouseApp("extract-from-config", argv))
main_func = mainEntryClickHouseExtractFromConfig;
for (auto & application : clickhouse_applications)
{
if (isClickhouseApp(application.first, argv))
{
main_func = application.second;
break;
}
}
return main_func(static_cast<int>(argv.size()), argv.data());
}

View File

@ -8,6 +8,7 @@
#include <mutex>
#include <shared_mutex>
#include <atomic>
#include <functional>
#include <Poco/Event.h>
#include <Poco/Timestamp.h>
#include <Core/Types.h>

View File

@ -391,6 +391,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
if (!post_filter_pos)
{
if (task->size_predictor)
task->size_predictor->updateFilteredRowsRation(pre_filter.size(), pre_filter.size());
res.clear();
continue;
@ -412,6 +413,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
col.column->filter(task->column_name_set.count(col.name) ? post_filter : pre_filter, -1);
rows = col.column->size();
}
if (task->size_predictor)
task->size_predictor->updateFilteredRowsRation(pre_filter.size(), pre_filter.size() - rows);
/// Replace column with condition value from PREWHERE to a constant.

View File

@ -88,22 +88,33 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPart(const String & par
/// If the part is not in ZooKeeper, we'll check if it's at least somewhere.
auto part_info = MergeTreePartInfo::fromPartName(part_name);
/** The logic is this:
/** The logic is as follows:
* - if some live or inactive replica has such a part, or a part covering it
* - it is Ok, nothing is needed, it is then downloaded when processing the queue, when the replica comes to life;
* - or, if the replica never comes to life, then the administrator will delete or create a new replica with the same address and see everything from the beginning;
* - if no one has such part or a part covering it, then
* - if someone has all the constituent parts, then we will do nothing - it simply means that other replicas have not yet completed merge
* - if no one has all the constituent parts, then agree the part forever lost,
* and remove the entry from the replication queue.
* - if there are two smaller parts, one with the same min block and the other with the same
* max block, we hope that all parts in between are present too and the needed part
* will appear on other replicas as a result of a merge.
* - otherwise, consider the part lost and delete the entry from the queue.
*
* Note that this logic is not perfect - some part in the interior may be missing and the
* needed part will never appear. But precisely determining whether the part will appear as
* a result of a merge is complicated - we can't just check if all block numbers covered
* by the missing part are present somewhere (because gaps between blocks are possible)
* and to determine the constituent parts of the merge we need to query the replication log
* (both the common log and the queues of the individual replicas) and then, if the
* constituent parts are in turn not found, solve the problem recursively for them.
*
* Considering the part lost when it is not in fact lost is very dangerous because it leads
* to divergent replicas and intersecting parts. So we err on the side of caution
* and don't delete the queue entry when in doubt.
*/
LOG_WARNING(log, "Checking if anyone has part covering " << part_name << ".");
LOG_WARNING(log, "Checking if anyone has a part covering " << part_name << ".");
bool found = false;
size_t part_length_in_blocks = part_info.max_block + 1 - part_info.min_block;
std::vector<char> found_blocks(part_length_in_blocks);
bool found_part_with_the_same_min_block = false;
bool found_part_with_the_same_max_block = false;
Strings replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas");
for (const String & replica : replicas)
@ -113,47 +124,40 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPart(const String & par
{
auto part_on_replica_info = MergeTreePartInfo::fromPartName(part_on_replica);
if (part_on_replica == part_name || part_on_replica_info.contains(part_info))
if (part_on_replica_info.contains(part_info))
{
found = true;
LOG_WARNING(log, "Found part " << part_on_replica << " on " << replica);
break;
LOG_WARNING(log, "Found part " << part_on_replica << " on " << replica << " that covers the missing part " << part_name);
return;
}
if (part_info.contains(part_on_replica_info))
{
if (part_on_replica_info.min_block == part_info.min_block)
found_part_with_the_same_min_block = true;
if (part_on_replica_info.max_block == part_info.max_block)
found_part_with_the_same_max_block = true;
for (auto block_num = part_on_replica_info.min_block; block_num <= part_on_replica_info.max_block; ++block_num)
found_blocks.at(block_num - part_info.min_block) = 1;
}
}
if (found)
break;
}
if (found)
if (found_part_with_the_same_min_block && found_part_with_the_same_max_block)
{
/// On some live or dead replica there is a necessary part or part covering it.
LOG_WARNING(log,
"Found parts with the same min block and with the same max block as the missing part "
<< part_name << ". Hoping that it will eventually appear as a result of a merge.");
return;
}
size_t num_found_blocks = 0;
for (auto found_block : found_blocks)
num_found_blocks += (found_block == 1);
if (num_found_blocks == part_length_in_blocks)
{
/// On a set of live or dead lines, there are all parts from which you can compound the desired part. We will do nothing.
LOG_WARNING(log, "Found all blocks for missing part " << part_name << ". Will wait for them to be merged.");
return;
}
}
}
/// No one has such a part.
LOG_ERROR(log, "No replica has part covering " << part_name);
if (num_found_blocks != 0)
LOG_WARNING(log, "When looking for smaller parts, that is covered by " << part_name
<< ", we found just " << num_found_blocks << " of " << part_length_in_blocks << " blocks.");
/// No one has such a part and the merge is impossible.
String not_found_msg;
if (found_part_with_the_same_min_block)
not_found_msg = "a smaller part with the same max block.";
else if (found_part_with_the_same_min_block)
not_found_msg = "a smaller part with the same min block.";
else
not_found_msg = "smaller parts with either the same min block or the same max block.";
LOG_ERROR(log, "No replica has part covering " << part_name
<< " and a merge is impossible: we didn't find " << not_found_msg);
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
@ -185,6 +189,7 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPart(const String & par
* NOTE It is possible to also add a condition if the entry in the queue is very old.
*/
size_t part_length_in_blocks = part_info.max_block + 1 - part_info.min_block;
if (part_length_in_blocks > 1000)
{
LOG_ERROR(log, "Won't add nonincrement_block_numbers because part spans too much blocks (" << part_length_in_blocks << ")");

View File

@ -82,7 +82,7 @@ void ReplicatedMergeTreeRestartingThread::run()
partialShutdown();
}
while (true)
while (!need_stop)
{
try
{
@ -106,6 +106,9 @@ void ReplicatedMergeTreeRestartingThread::run()
break;
}
if (need_stop)
break;
if (storage.is_readonly)
CurrentMetrics::sub(CurrentMetrics::ReadonlyReplica);
storage.is_readonly = false;

View File

@ -5,7 +5,7 @@ set -e
# Test 1. Complex test checking columns.txt
chl="clickhouse-client -q"
ch_dir=`clickhouse --extract-from-config -c /etc/clickhouse-server/config.xml -k path`
ch_dir=`clickhouse extract-from-config -c /etc/clickhouse-server/config.xml -k path`
$chl "DROP TABLE IF EXISTS test.partition_428"
$chl "CREATE TABLE test.partition_428 (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) ENGINE = MergeTree(p, k, 1)"

View File

@ -7,9 +7,9 @@
├────────────┼────────────┤
│ 100 │ 100 │
├────────────┼────────────┤
│ 1001 │ 1001
│ 1000 │ 1000
├────────────┼────────────┤
│ 10001 │ 10001
│ 10000 │ 10000
├────────────┼────────────┤
│ 100000 │ 100000 │
├────────────┼────────────┤
@ -25,8 +25,8 @@
│ 1 │ 1 │
│ 10 │ 10 │
│ 100 │ 100 │
│ 1001 │ 1001
│ 10001 │ 10001
│ 1000 │ 1000
│ 10000 │ 10000
│ 100000 │ 100000 │
│ 1000000 │ 1000000 │
│ 10000000 │ 10000000 │
@ -38,8 +38,8 @@
1 1
10 10
100 100
1001 1001
10001 10001
1000 1000
10000 10000
100000 100000
1000000 1000000
10000000 10000000
@ -49,8 +49,8 @@
│ 1 │ 1 │
│ 10 │ 10 │
│ 100 │ 100 │
│ 1001 │ 1001
│ 10001 │ 10001
│ 1000 │ 1000
│ 10000 │ 10000
│ 100000 │ 100000 │
│ 1000000 │ 1000000 │
│ 10000000 │ 10000000 │

View File

@ -1,7 +1,7 @@
SELECT toUInt64(ceil(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT Pretty;
SELECT toUInt64(ceil(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompact;
SELECT toUInt64(ceil(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT Pretty;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompact;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace;
SET max_block_size = 5;
SELECT toUInt64(ceil(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompactMonoBlock;
SELECT toUInt64(round(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompactMonoBlock;
SELECT '\\''\'' FORMAT Pretty;
SELECT '\\''\'', 1 FORMAT Vertical;

View File

@ -1,6 +1,7 @@
drop table if exists test.tab;
create table test.tab (date Date, x UInt64, s FixedString(128)) engine = MergeTree(date, (date, x), 8192);
insert into test.tab select today(), number, toFixedString('', 128) from system.numbers limit 8192;
set preferred_block_size_bytes = 2000000;
set preferred_max_column_in_block_size_bytes = 0;
select max(blockSize()), min(blockSize()), any(ignore(*)) from test.tab;
@ -24,3 +25,10 @@ create table test.tab (date Date, x UInt64, s FixedString(128)) engine = MergeTr
insert into test.tab select today(), number, toFixedString('', 128) from system.numbers limit 10;
set preferred_max_column_in_block_size_bytes = 128;
select s from test.tab where s == '' format Null;
drop table if exists test.tab;
create table test.tab (date Date, x UInt64, s String) engine = MergeTree(date, (date, x), 8192);
insert into test.tab select today(), number, 'abc' from system.numbers limit 81920;
set preferred_block_size_bytes = 0;
select count(*) from test.tab prewhere s != 'abc' format Null;
select count(*) from test.tab prewhere s = 'abc' format Null;

View File

@ -7,3 +7,59 @@
1
0
0
inner
1
outer
0
single line
0
1
shifted grid
0
1
pair of lines, single polygon
1
0
0
1
0
0
pair of lines, different polygons
0
0
1
1
0
0
1
0
complex polygon
0
1
0
1
0
1
polygon with holes
0
1
0
1
0
1
0
1
0
1
0
1
polygons with reversed direction
0
1
0
1
0
1
0
eps for complex polygon in grid
1

View File

@ -1,9 +1,75 @@
SELECT pointInPolygonFranklin(tuple(2.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonFranklin(tuple(1.0,2.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonFranklin(tuple(4.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygon(tuple(2.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygon(tuple(1.0,2.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygon(tuple(4.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonWinding(tuple(2.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonWinding(tuple(1.0,2.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonWinding(tuple(4.0,1.0), tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0));
SELECT pointInPolygonFranklin(tuple(2.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonFranklin(tuple(1.0,2.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonFranklin(tuple(4.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygon(tuple(2.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygon(tuple(1.0,2.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygon(tuple(4.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonWinding(tuple(2.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonWinding(tuple(1.0,2.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT pointInPolygonWinding(tuple(4.0,1.0), [tuple(0.0,0.0), tuple(3.0,3.0), tuple(3.0,0.0), tuple(0.0,0.0)]);
SELECT 'inner';
SELECT pointInPolygonWithGrid((3., 3.), [(6, 0), (8, 4), (5, 8), (0, 2), (6, 0)]);
SELECT 'outer';
SELECT pointInPolygonWithGrid((0.1, 0.1), [(6., 0.), (8., 4.), (5., 8.), (0., 2.), (6., 0.)]);
SELECT 'single line';
SELECT pointInPolygonWithGrid((4.1, 0.1), [(6., 0.), (8., 4.), (5., 8.), (0., 2.), (6., 0.)]);
SELECT pointInPolygonWithGrid((4.9, 0.9), [(6., 0.), (8., 4.), (5., 8.), (0., 2.), (6., 0.)]);
SELECT 'shifted grid';
SELECT pointInPolygonWithGrid((0., 0.), [(6., 1.), (8., 4.), (5., 8.), (1., 2.), (6., 1.)]);
SELECT pointInPolygonWithGrid((6., 5.), [(6., 1.), (8., 4.), (5., 8.), (1., 2.), (6., 1.)]);
SELECT 'pair of lines, single polygon';
SELECT pointInPolygonWithGrid((0.1, 0.1), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((0.9, 0.1), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((0.1, 0.9), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((2.2, 2.2), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((2.1, 2.9), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT pointInPolygonWithGrid((2.9, 2.1), [(0., 0.), (8., 7.), (7., 8.), (0., 0.)]);
SELECT 'pair of lines, different polygons';
SELECT pointInPolygonWithGrid((0.1, 0.1), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((1., 1.), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((0.7, 0.1), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((0.1, 0.7), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((1.1, 0.1), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((0.1, 1.1), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((5.0, 5.0), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT pointInPolygonWithGrid((7.9, 7.9), [(0.5, 0.), (1.0, 0.), (8.0, 7.5), (7.5, 8.0), (0., 1.), (0., 0.5), (4.5, 5.5), (5.5, 4.5), (0.5, 0.0)]);
SELECT 'complex polygon';
SELECT pointInPolygonWithGrid((0.05, 0.05), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.15, 0.15), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.3, 0.4), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.4, 0.7), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.7, 0.6), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT pointInPolygonWithGrid((0.9, 0.1), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1.0, 0.0), (8.0, 7.0), (7.0, 8.0), (0., 1.)]);
SELECT 'polygon with holes';
SELECT pointInPolygonWithGrid((1., 1.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((2.5, 2.5), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((4., 4.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((4., 2.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((9., 9.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (3., 5.), (5., 5.), (5., 3.)]);
SELECT pointInPolygonWithGrid((0.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((1.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((2.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((3.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((4.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((5.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT pointInPolygonWithGrid((6.5, 1.5), [(0., 0.), (7., 0.), (7., 3.), (0., 3.)], [(1., 1.), (2., 1.), (2., 2.), (1., 2.)], [(3., 1.), (4., 1.), (4., 2.), (3., 2.)], [(5., 1.), (6., 1.), (6., 2.), (5., 2.)]);
SELECT 'polygons with reversed direction';
SELECT pointInPolygonWithGrid((4.1, .1), [(6., 0.), (0., 2.), (5., 8.), (8., 4.)]);
SELECT pointInPolygonWithGrid((4.1, .9), [(6., 0.), (0., 2.), (5., 8.), (8., 4.)]);
SELECT pointInPolygonWithGrid((1., 1.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT pointInPolygonWithGrid((2.5, 2.5), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT pointInPolygonWithGrid((4., 4.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT pointInPolygonWithGrid((4., 2.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)], [(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT pointInPolygonWithGrid((9., 9.), [(4., 0.), (8., 4.), (4., 8.), (0., 4.)],[(3., 3.), (5., 3.), (5., 5.), (3., 5.)]);
SELECT 'eps for complex polygon in grid';
SELECT pointInPolygonWithGrid((0., 0.), [(0., 1.), (0.2, 0.5), (0.6, 0.5), (0.8, 0.8), (0.8, 0.3), (0.1, 0.3), (0.1, 0.1), (0.8, 0.1), (1., 0.), (-6., -7.), (-7., -6.), (0., 1.)])

View File

@ -10,3 +10,5 @@ Content-Type: text/tab-separated-values; charset=UTF-8
Transfer-Encoding: chunked
Keep-Alive: timeout=3
1
1

View File

@ -2,3 +2,6 @@
( curl -s --head "${CLICKHOUSE_URL:=http://localhost:8123/}?query=SELECT%201";
curl -s --head "${CLICKHOUSE_URL:=http://localhost:8123/}?query=select+*+from+system.numbers+limit+1000000" ) | grep -v "Date:"
curl -sS -X POST "http://127.0.0.1:8123?query=SELECT+1"
curl -sS -X POST "http://127.0.0.1:8123?query=SELECT+1" --data ''

View File

@ -0,0 +1,10 @@
2000-01-01 2000-01-01 00:00:00 [1,2,3] [10,10,10]
2000-01-01 2000-01-01 00:00:00 [3,4,5] [10,10,10]
2000-01-01 2000-01-01 00:01:00 [4,5,6] [10,10,10]
2000-01-01 2000-01-01 00:01:00 [6,7,8] [10,10,10]
([1,2,3,4,5,6,7,8],[10,10,20,20,20,20,10,10])
([1,2,3,4,5,6,7,8],[10,10,20,20,20,20,10,10])
2000-01-01 00:00:00 ([1,2,3,4,5],[10,10,20,10,10])
2000-01-01 00:01:00 ([4,5,6,7,8],[10,10,20,10,10])
2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10]
2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10]

View File

@ -0,0 +1,12 @@
DROP TABLE IF EXISTS test.sum_map;
CREATE TABLE test.sum_map(date Date, timeslot DateTime, statusMap Nested(status UInt16, requests UInt64)) ENGINE = Log;
INSERT INTO test.sum_map VALUES ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10]), ('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10]), ('2000-01-01', '2000-01-01 00:01:00', [4, 5, 6], [10, 10, 10]), ('2000-01-01', '2000-01-01 00:01:00', [6, 7, 8], [10, 10, 10]);
SELECT * FROM test.sum_map ORDER BY timeslot;
SELECT sumMap(statusMap.status, statusMap.requests) FROM test.sum_map;
SELECT sumMapMerge(s) FROM (SELECT sumMapState(statusMap.status, statusMap.requests) AS s FROM test.sum_map);
SELECT timeslot, sumMap(statusMap.status, statusMap.requests) FROM test.sum_map GROUP BY timeslot ORDER BY timeslot;
SELECT timeslot, sumMap(statusMap.status, statusMap.requests).1, sumMap(statusMap.status, statusMap.requests).2 FROM test.sum_map GROUP BY timeslot ORDER BY timeslot;
DROP TABLE test.sum_map;

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (1.1.54288) unstable; urgency=low
clickhouse (1.1.54292) unstable; urgency=low
* Modified source code
-- <robot-metrika-test@yandex-team.ru> Fri, 08 Sep 2017 07:26:49 +0300
-- <robot-metrika-test@yandex-team.ru> Wed, 20 Sep 2017 21:05:46 +0300

View File

@ -88,7 +88,7 @@ die()
check_config()
{
if [ -x "$BINDIR/$GENERIC_PROGRAM" ]; then
su -s $SHELL ${CLICKHOUSE_USER} -c "$BINDIR/$GENERIC_PROGRAM --extract-from-config --config-file=\"$CLICKHOUSE_CONFIG\" --key=path" >/dev/null || die "Configuration file ${CLICKHOUSE_CONFIG} doesn't parse successfully. Won't restart server. You may use forcerestart if you are sure.";
su -s $SHELL ${CLICKHOUSE_USER} -c "$BINDIR/$GENERIC_PROGRAM extract-from-config --config-file=\"$CLICKHOUSE_CONFIG\" --key=path" >/dev/null || die "Configuration file ${CLICKHOUSE_CONFIG} doesn't parse successfully. Won't restart server. You may use forcerestart if you are sure.";
fi
}
@ -96,7 +96,7 @@ check_config()
initdb()
{
if [ -x "$BINDIR/$GENERIC_PROGRAM" ]; then
CLICKHOUSE_DATADIR_FROM_CONFIG=$(su -s $SHELL ${CLICKHOUSE_USER} -c "$BINDIR/$GENERIC_PROGRAM --extract-from-config --config-file=\"$CLICKHOUSE_CONFIG\" --key=path")
CLICKHOUSE_DATADIR_FROM_CONFIG=$(su -s $SHELL ${CLICKHOUSE_USER} -c "$BINDIR/$GENERIC_PROGRAM extract-from-config --config-file=\"$CLICKHOUSE_CONFIG\" --key=path")
if [ "(" "$?" -ne "0" ")" -o "(" -z "${CLICKHOUSE_DATADIR_FROM_CONFIG}" ")" ]; then
die "Cannot obtain value of path from config file: ${CLICKHOUSE_CONFIG}";
fi

View File

@ -44,6 +44,42 @@ sum(x)
Calculates the sum.
Only works for numbers.
sumMap(key, value)
------
Performs summation of array 'value' by corresponding keys of array 'key'.
Number of elements in 'key' and 'value' arrays should be the same for each row, on which summation is being performed.
Returns a tuple of two arrays - sorted keys and values, summed up by corresponding keys.
Example:
.. code-block:: sql
CREATE TABLE sum_map(
date Date,
timeslot DateTime,
statusMap Nested(
status UInt16,
requests UInt64
)
) ENGINE = Log;
INSERT INTO sum_map VALUES
('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10]),
('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10]),
('2000-01-01', '2000-01-01 00:01:00', [4, 5, 6], [10, 10, 10]),
('2000-01-01', '2000-01-01 00:01:00', [6, 7, 8], [10, 10, 10]);
SELECT
timeslot,
sumMap(statusMap.status, statusMap.requests)
FROM sum_map
GROUP BY timeslot
.. code-block:: text
┌────────────timeslot─┬─sumMap(statusMap.status, statusMap.requests)─┐
│ 2000-01-01 00:00:00 │ ([1,2,3,4,5],[10,10,20,10,10]) │
│ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │
└─────────────────────┴──────────────────────────────────────────────┘
avg(x)
------
Calculates the average.

View File

@ -20,6 +20,8 @@ Install Git and CMake
sudo apt-get install git cmake3
Or just cmake on newer systems.
Detect number of threads
------------------------

View File

@ -285,7 +285,7 @@ BTW, you could run OPTIMIZE for MergeTree table. But this is not necessary, ever
Results on single server
========================
------------------------
Q1:
@ -377,7 +377,7 @@ In that case, query execution speed is dominated by latency.
We do queries from client located in Yandex datacenter in Mäntsälä (Finland) to cluster somewhere in Russia, that adds at least 20 ms of latency.
Summary
=======
-------
.. code-block:: text

View File

@ -11,6 +11,7 @@ There exist third-party client libraries for ClickHouse:
- `clickhouse-php-client <https://github.com/8bitov/clickhouse-php-client>`_
- `PhpClickHouseClient <https://github.com/SevaCode/PhpClickHouseClient>`_
- `phpClickHouse <https://github.com/smi2/phpClickHouse>`_
- `clickhouse-client <https://github.com/bozerkins/clickhouse-client>`_
* Go
- `clickhouse <https://github.com/kshvakov/clickhouse/>`_
- `go-clickhouse <https://github.com/roistat/go-clickhouse>`_

View File

@ -37,6 +37,8 @@ Examples:
[(1, 100)] + [(1, 150), (2, 150)] -> [(1, 250), (2, 150)]
[(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)]
For aggregating Map use function sumMap(key, value).
For nested data structures, you don't need to specify the columns as a list of columns for totaling.
This table engine is not particularly useful. Remember that when saving just pre-aggregated data, you lose some of the system's advantages.

View File

@ -44,6 +44,42 @@ sum(x)
Вычисляет сумму.
Работает только для чисел.
sumMap(key, value)
------
Производит суммирование массива 'value' по соотвествующим ключам заданным в массиве 'key'.
Количество элементов в 'key' и 'value' должно быть одинаковым для каждой строки, для которой происходит суммирование.
Возвращает кортеж из двух массивов - ключи в отсортированном порядке и значения, просуммированные по соотвествующим ключам.
Пример:
.. code-block:: sql
CREATE TABLE sum_map(
date Date,
timeslot DateTime,
statusMap Nested(
status UInt16,
requests UInt64
)
) ENGINE = Log;
INSERT INTO sum_map VALUES
('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10]),
('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10]),
('2000-01-01', '2000-01-01 00:01:00', [4, 5, 6], [10, 10, 10]),
('2000-01-01', '2000-01-01 00:01:00', [6, 7, 8], [10, 10, 10]);
SELECT
timeslot,
sumMap(statusMap.status, statusMap.requests)
FROM sum_map
GROUP BY timeslot
.. code-block:: text
┌────────────timeslot─┬─sumMap(statusMap.status, statusMap.requests)─┐
│ 2000-01-01 00:00:00 │ ([1,2,3,4,5],[10,10,20,10,10]) │
│ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │
└─────────────────────┴──────────────────────────────────────────────┘
avg(x)
------
Вычисляет среднее.

View File

@ -20,6 +20,8 @@ Install Git and CMake
sudo apt-get install git cmake3
Or just cmake on newer systems.
Detect number of threads
------------------------

View File

@ -282,7 +282,7 @@
Между прочим, на MergeTree можно запустить запрос OPTIMIZE. Но это не обязательно, всё будет в порядке и без этого.
Results on single server
========================
------------------------
Q1:
@ -373,7 +373,7 @@ Q4: 0.072 sec.
Мы выполняли запросы с помощью клиента, расположенного в датацентре Яндекса в Мянтсяля (Финляндия), на кластер в России, что добавляет порядка 20 мс задержки.
Резюме
======
------
.. code-block:: text

View File

@ -11,6 +11,7 @@
- `clickhouse-php-client <https://github.com/8bitov/clickhouse-php-client>`_
- `PhpClickHouseClient <https://github.com/SevaCode/PhpClickHouseClient>`_
- `phpClickHouse <https://github.com/smi2/phpClickHouse>`_
- `clickhouse-client <https://github.com/bozerkins/clickhouse-client>`_
* Go
- `clickhouse <https://github.com/kshvakov/clickhouse/>`_
- `go-clickhouse <https://github.com/roistat/go-clickhouse>`_

View File

@ -36,6 +36,8 @@ SummingMergeTree
[(1, 100)] + [(1, 150), (2, 150)] -> [(1, 250), (2, 150)]
[(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)]
Для агрегации Map используйте функцию sumMap(key, value).
Для вложенных структур данных не нужно указывать её столбцы в качестве списка столбцов для суммирования.
Этот движок таблиц разработан по просьбе БК, и является мало полезным. Помните, что при хранении лишь предагрегированных данных, вы теряете часть преимуществ системы.

View File

@ -1,6 +1,10 @@
include (CMakePushCheckState)
cmake_push_check_state ()
option (ENABLE_UNWIND "Enable libunwind (better stacktraces)" ON)
if (ENABLE_UNWIND)
if (CMAKE_SYSTEM MATCHES "Linux" AND NOT ARCH_ARM AND NOT ARCH_32)
option (USE_INTERNAL_UNWIND_LIBRARY "Set to FALSE to use system unwind library instead of bundled" ${NOT_UNBUNDLED})
else ()
@ -41,6 +45,8 @@ elseif (CMAKE_SYSTEM MATCHES "Linux" AND NOT ARCH_ARM AND NOT ARCH_32)
set (USE_UNWIND 1)
endif ()
endif ()
message (STATUS "Using unwind=${USE_UNWIND}: ${UNWIND_INCLUDE_DIR} : ${UNWIND_LIBRARY}")
cmake_pop_check_state ()

View File

@ -29,9 +29,8 @@ if (ENABLE_MYSQL)
if (MYSQL_INCLUDE_DIR AND (STATIC_MYSQLCLIENT_LIB OR MYSQLCLIENT_LIBRARIES))
set (USE_MYSQL 1)
endif ()
set (MYSQLXX_LIBRARY mysqlxx)
endif ()
endif ()
if (USE_MYSQL)

View File

@ -26,6 +26,7 @@ inc="-I. \
-I./contrib/libpoco/Foundation/include \
-I./contrib/libboost/boost_1_62_0 \
-I./contrib/libbtrie/include \
-I./contrib/libpcg-random/include \
-I./libs/libmysqlxx/include \
-I./libs/libcommon/include \
-I./build/libs/libcommon/include \
@ -41,5 +42,5 @@ if [ -z $1 ]; then
else
echo -n "$1 "
echo -n `grep "#include" $1| wc -l` " "
echo -e "#include <$1> \n int main() {return 0;}" | time --format "%e %M" g++-6 -c -std=gnu++1z $inc -x c++ -
echo -e "#include <$1> \n int main() {return 0;}" | time --format "%e %M" g++-7 -c -std=gnu++1z $inc -x c++ -
fi

View File

@ -1,6 +1,6 @@
add_executable (clickhouse-compressor main.cpp)
target_link_libraries (clickhouse-compressor dbms ${Boost_PROGRAM_OPTIONS_LIBRARY})
target_link_libraries (clickhouse-compressor clickhouse-compressor-lib)
install (TARGETS clickhouse-compressor RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse-compressor)

View File

@ -1,119 +1,6 @@
#include <iostream>
#include <boost/program_options.hpp>
#include <Common/Exception.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/CompressedWriteBuffer.h>
#include <IO/CompressedReadBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_SIZE_COMPRESSED;
}
}
/// Outputs sizes of uncompressed and compressed blocks for compressed file.
void stat(DB::ReadBuffer & in, DB::WriteBuffer & out)
{
while (!in.eof())
{
in.ignore(16); /// checksum
char header[COMPRESSED_BLOCK_HEADER_SIZE];
in.readStrict(header, COMPRESSED_BLOCK_HEADER_SIZE);
UInt32 size_compressed = unalignedLoad<UInt32>(&header[1]);
if (size_compressed > DBMS_MAX_COMPRESSED_SIZE)
throw DB::Exception("Too large size_compressed. Most likely corrupted data.", DB::ErrorCodes::TOO_LARGE_SIZE_COMPRESSED);
UInt32 size_decompressed = unalignedLoad<UInt32>(&header[5]);
DB::writeText(size_decompressed, out);
DB::writeChar('\t', out);
DB::writeText(size_compressed, out);
DB::writeChar('\n', out);
in.ignore(size_compressed - COMPRESSED_BLOCK_HEADER_SIZE);
}
}
int mainEntryClickHouseCompressor(int argc, char ** argv);
int main(int argc, char ** argv)
{
boost::program_options::options_description desc("Allowed options");
desc.add_options()
("help,h", "produce help message")
("decompress,d", "decompress")
("block-size,b", boost::program_options::value<unsigned>()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size")
("hc", "use LZ4HC instead of LZ4")
("zstd", "use ZSTD instead of LZ4")
("none", "use no compression instead of LZ4")
("stat", "print block statistics of compressed data")
;
boost::program_options::variables_map options;
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
if (options.count("help"))
{
std::cout << "Usage: " << argv[0] << " [options] < in > out" << std::endl;
std::cout << desc << std::endl;
return 1;
}
try
{
bool decompress = options.count("decompress");
bool use_lz4hc = options.count("hc");
bool use_zstd = options.count("zstd");
bool stat_mode = options.count("stat");
bool use_none = options.count("none");
unsigned block_size = options["block-size"].as<unsigned>();
DB::CompressionMethod method = DB::CompressionMethod::LZ4;
if (use_lz4hc)
method = DB::CompressionMethod::LZ4HC;
else if (use_zstd)
method = DB::CompressionMethod::ZSTD;
else if (use_none)
method = DB::CompressionMethod::NONE;
DB::ReadBufferFromFileDescriptor rb(STDIN_FILENO);
DB::WriteBufferFromFileDescriptor wb(STDOUT_FILENO);
if (stat_mode)
{
/// Output statistic for compressed file.
stat(rb, wb);
}
else if (decompress)
{
/// Decompression
DB::CompressedReadBuffer from(rb);
DB::copyData(from, wb);
}
else
{
/// Compression
DB::CompressedWriteBuffer to(wb, method, block_size);
DB::copyData(rb, to);
}
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true);
return DB::getCurrentExceptionCode();
}
return 0;
return mainEntryClickHouseCompressor(argc, argv);
}