Moved files [#METR-17973].

This commit is contained in:
Alexey Milovidov 2015-10-05 03:33:43 +03:00
parent 0ced5b4013
commit fefd3791f1
35 changed files with 263 additions and 32 deletions

View File

@ -15,7 +15,7 @@
#include <DB/Columns/ColumnArray.h>
#include <stats/IntHash.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
namespace DB

View File

@ -4,7 +4,7 @@
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Parsers/CommonParsers.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <boost/range/iterator_range_core.hpp>
#include <DB/Parsers/ExpressionElementParsers.h>
#include <DB/Parsers/ASTLiteral.h>

View File

@ -3,7 +3,7 @@
#include <map>
#include <tuple>
#include <mutex>
#include <statdaemons/ext/function_traits.hpp>
#include <ext/function_traits.hpp>
/** Простейший кэш для свободной функции.

View File

@ -2,7 +2,7 @@
#include <DB/Columns/ColumnConst.h>
#include <DB/DataStreams/IBlockOutputStream.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
namespace DB

View File

@ -6,8 +6,8 @@
#include <DB/Core/ColumnNumbers.h>
#include <DB/DataStreams/MergingSortedBlockInputStream.h>
#include <DB/Storages/MergeTree/PKCondition.h>
#include <statdaemons/ext/range.hpp>
#include <statdaemons/ext/map.hpp>
#include <ext/range.hpp>
#include <ext/map.hpp>
namespace DB

View File

@ -6,7 +6,7 @@
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Common/HashTable/HashMap.h>
#include <statdaemons/ext/scope_guard.hpp>
#include <ext/scope_guard.hpp>
#include <Poco/RWLock.h>
#include <cmath>
#include <atomic>

View File

@ -5,7 +5,7 @@
#include <DB/DataStreams/RemoteBlockInputStream.h>
#include <DB/Interpreters/executeQuery.h>
#include <DB/Common/isLocalAddress.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <Poco/Util/AbstractConfiguration.h>
#include "writeParenthesisedString.h"

View File

@ -5,7 +5,7 @@
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Common/Arena.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <atomic>
#include <vector>
#include <tuple>

View File

@ -5,7 +5,7 @@
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <atomic>
#include <memory>
#include <tuple>

View File

@ -7,7 +7,7 @@
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <mysqlxx/Query.h>
#include <mysqlxx/PoolWithFailover.h>
#include <vector>

View File

@ -2,7 +2,7 @@
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/MySQLBlockInputStream.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <mysqlxx/Pool.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <strconvert/escape.h>

View File

@ -5,7 +5,7 @@
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <atomic>
#include <memory>
#include <tuple>

View File

@ -17,7 +17,7 @@
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/FunctionsConditional.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <unordered_map>

View File

@ -14,7 +14,7 @@
#include <DB/Functions/IFunction.h>
#include <arpa/inet.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <array>

View File

@ -11,7 +11,7 @@
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
namespace DB

View File

@ -20,7 +20,7 @@
#include <DB/Dictionaries/CacheDictionary.h>
#include <DB/Dictionaries/RangeHashedDictionary.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
namespace DB

View File

@ -22,7 +22,7 @@
#include <DB/Common/HashTable/Hash.h>
#include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <stats/IntHash.h>

View File

@ -24,7 +24,7 @@
#include <DB/Common/UnicodeBar.h>
#include <DB/Functions/IFunction.h>
#include <DB/Interpreters/ExpressionActions.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <cmath>

View File

@ -12,7 +12,7 @@
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <emmintrin.h>
#include <nmmintrin.h>

View File

@ -19,7 +19,7 @@
#include <mutex>
#include <stack>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <Poco/Unicode.h>

View File

@ -2,7 +2,7 @@
#include <DB/Core/NamesAndTypes.h>
#include <DB/Storages/MergeTree/RangesInDataPart.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <mutex>

View File

@ -10,8 +10,8 @@
#include <DB/Parsers/ASTSet.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Common/escapeForFileName.h>
#include <statdaemons/ext/scope_guard.hpp>
#include <statdaemons/ext/map.hpp>
#include <ext/scope_guard.hpp>
#include <ext/map.hpp>
#include <memory>
#include <unordered_map>
#include <map>

View File

@ -2,7 +2,7 @@
#include <Poco/DirectoryIterator.h>
#include <common/Revision.h>
#include <statdaemons/ext/unlock_guard.hpp>
#include <ext/unlock_guard.hpp>
#include <DB/Common/SipHash.h>

View File

@ -3,7 +3,6 @@
#include <DB/Interpreters/Join.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Functions/IFunction.h>
#include <DB/Functions/FunctionsArray.h>

View File

@ -44,7 +44,7 @@
#include <DB/Functions/FunctionFactory.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
namespace DB

View File

@ -2,7 +2,7 @@
#include <DB/Dictionaries/DictionaryFactory.h>
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Dictionaries/IDictionarySource.h>
#include <statdaemons/ext/scope_guard.hpp>
#include <ext/scope_guard.hpp>
#include <Poco/Util/Application.h>
#include <Poco/Glob.h>

View File

@ -10,7 +10,7 @@
#include <common/Revision.h>
#include <statdaemons/ConfigProcessor.h>
#include <statdaemons/ext/scope_guard.hpp>
#include <ext/scope_guard.hpp>
#include <memory>
#include <thread>

View File

@ -7,7 +7,7 @@
#include <DB/Common/setThreadName.h>
#include <Poco/Ext/ThreadNumber.h>
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
namespace DB

View File

@ -1,4 +1,4 @@
#include <statdaemons/ext/range.hpp>
#include <ext/range.hpp>
#include <DB/Storages/ColumnsDescription.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>

View File

@ -11,7 +11,7 @@
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Interpreters/ExternalDictionaries.h>
#include <statdaemons/ext/map.hpp>
#include <ext/map.hpp>
#include <mutex>
namespace DB

View File

@ -0,0 +1,16 @@
#pragma once
#include <tuple>
#include <type_traits>
template <typename T>
struct function_traits;
template <typename ReturnType, typename... Args>
struct function_traits<ReturnType(Args...)>
{
using result = ReturnType;
using arguments = std::tuple<Args...>;
using arguments_decay = std::tuple<typename std::decay<Args>::type...>;
};

View File

@ -0,0 +1,86 @@
#pragma once
#include <iterator>
#include <type_traits>
namespace ext
{
/// \brief Strip type off top level reference and cv-qualifiers thus allowing storage in containers
template <typename T>
using unqualified_t = std::remove_cv_t<std::remove_reference_t<T>>;
template <typename It, typename Mapper>
using apply_t = typename std::result_of<Mapper(typename It::reference)>::type;
template <typename It, typename Mapper>
struct map_iterator : std::iterator<
typename It::iterator_category,
std::remove_reference_t<apply_t<It, Mapper>>,
std::ptrdiff_t,
std::add_pointer_t<std::remove_reference_t<apply_t<It, Mapper>>>,
apply_t<It, Mapper>>
{
using base_iterator = std::iterator<
typename It::iterator_category,
std::remove_reference_t<apply_t<It, Mapper>>,
std::ptrdiff_t,
std::add_pointer<std::remove_reference_t<apply_t<It, Mapper>>>,
apply_t<It, Mapper>>;
It current;
Mapper mapper;
map_iterator(const It it, const Mapper mapper) : current{it}, mapper{mapper} {}
typename base_iterator::reference operator*() { return mapper(*current); }
map_iterator & operator++() { return ++current, *this; }
map_iterator & operator--() { return --current, *this; }
bool operator==(const map_iterator & other) { return current == other.current; }
bool operator!=(const map_iterator & other) { return current != other.current; }
typename base_iterator::difference_type operator-(const map_iterator & other) { return current - other.current; }
};
template <typename It, typename Mapper>
auto make_map_iterator(const It it, const Mapper mapper) -> ext::map_iterator<It, Mapper>
{
return { it, mapper };
}
/** \brief Returns collection of the same container-type as the input collection,
* with each element transformed by the application of `mapper`. */
template <template <typename...> class Collection, typename... Params, typename Mapper>
auto map(const Collection<Params...> & collection, const Mapper mapper)
-> Collection<unqualified_t<decltype(mapper(*std::begin(collection)))>>
{
using value_type = unqualified_t<decltype(mapper(*std::begin(collection)))>;
return Collection<value_type>(ext::make_map_iterator(std::begin(collection), mapper),
ext::make_map_iterator(std::end(collection), mapper));
};
/** \brief Returns collection of specified container-type,
* with each element transformed by the application of `mapper`.
* Allows conversion between different container-types, e.g. std::vector to std::list */
template <template <typename...> class ResultCollection, typename Collection, typename Mapper>
auto map(const Collection & collection, const Mapper mapper)
-> ResultCollection<unqualified_t<decltype(mapper(*std::begin(collection)))>>
{
using value_type = unqualified_t<decltype(mapper(*std::begin(collection)))>;
return ResultCollection<value_type>(ext::make_map_iterator(std::begin(collection), mapper),
ext::make_map_iterator(std::end(collection), mapper));
};
/** \brief Returns collection of specified type,
* with each element transformed by the application of `mapper`.
* Allows leveraging implicit conversion between the result of applying `mapper` and R::value_type. */
template <typename ResultCollection, typename Collection, typename Mapper>
ResultCollection map(const Collection & collection, const Mapper mapper)
{
return ResultCollection(ext::make_map_iterator(std::begin(collection), mapper),
ext::make_map_iterator(std::end(collection), mapper));
}
}

View File

@ -0,0 +1,77 @@
#pragma once
#include <type_traits>
#include <utility>
#include <iterator>
/** \brief Numeric range iterator, used to represent a half-closed interval [begin, end).
* In conjunction with std::reverse_iterator allows for forward and backward iteration
* over corresponding interval. */
namespace ext
{
/// @todo check whether difference_type should be signed (make_signed_t<T>)
template<typename T> struct range_iterator : std::iterator<
std::bidirectional_iterator_tag, T, T, void, T>
{
T current{};
range_iterator() = default;
range_iterator(const T t) : current(t) {}
T operator*() const { return current; }
range_iterator & operator++() { return ++current, *this; }
range_iterator & operator--() { return --current, *this; }
bool operator==(const range_iterator & other) const { return current == other.current; }
bool operator!=(const range_iterator & other) const { return current != other.current; }
};
template<typename T> using reverse_range_iterator = std::reverse_iterator<range_iterator<T>>;
/** \brief Range-based for loop adapter for (reverse_)range_iterator.
* By and large should be in conjunction with ext::range and ext::reverse_range. */
template<typename T, bool forward> struct range_wrapper
{
using value_type = typename std::remove_reference<T>::type;
using range_iterator_t = ext::range_iterator<value_type>;
using iterator = typename std::conditional<forward,
range_iterator_t,
ext::reverse_range_iterator<value_type>>::type;
value_type begin_, end_;
iterator begin() const { return iterator(range_iterator_t{begin_}); }
iterator end() const { return iterator(range_iterator_t{end_}); }
};
/** \brief Constructs range_wrapper for forward-iteration over [begin, end) in range-based for loop.
* Usage example:
* for (const auto i : ext::range(0, 4)) print(i);
* Output:
* 0 1 2 3 */
template<typename T1, typename T2>
inline ext::range_wrapper<typename std::common_type<T1, T2>::type, true> range(T1 begin, T2 end)
{
using common_type = typename std::common_type<T1, T2>::type;
return { static_cast<common_type>(begin), static_cast<common_type>(end) };
}
/** \brief Constructs range_wrapper for backward-iteration over [begin, end) in range-based for loop.
* Usage example:
* for (const auto i : ext::reverse_range(0, 4)) print(i);
* Output:
* 3 2 1 0 */
template<typename T1, typename T2>
inline ext::range_wrapper<typename std::common_type<T1, T2>::type, false> reverse_range(T1 begin, T2 end)
{
using common_type = typename std::common_type<T1, T2>::type;
return { static_cast<common_type>(end), static_cast<common_type>(begin) };
}
template<typename T>
inline ext::range_iterator<T> make_range_iterator(const T value)
{
return { value };
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <utility>
namespace ext
{
template <class F> class scope_guard {
const F function;
public:
constexpr scope_guard(const F & function) : function{function} {}
constexpr scope_guard(F && function) : function{std::move(function)} {}
~scope_guard() { function(); }
};
template <class F>
inline scope_guard<F> make_scope_guard(F && function) { return std::forward<F>(function); }
}
#define SCOPE_EXIT_CONCAT(n, ...) \
const auto scope_exit##n = ext::make_scope_guard([&] { __VA_ARGS__; })
#define SCOPE_EXIT_FWD(n, ...) SCOPE_EXIT_CONCAT(n, __VA_ARGS__)
#define SCOPE_EXIT(...) SCOPE_EXIT_FWD(__LINE__, __VA_ARGS__)

View File

@ -0,0 +1,27 @@
#pragma once
namespace ext
{
template <typename T>
class unlock_guard
{
public:
unlock_guard(T & mutex_) : mutex(mutex_)
{
mutex.unlock();
}
~unlock_guard()
{
mutex.lock();
}
unlock_guard(const unlock_guard &) = delete;
unlock_guard & operator=(const unlock_guard &) = delete;
private:
T & mutex;
};
}