Using std::shared_ptr for functions and AST [#METR-21503].

This commit is contained in:
Alexey Milovidov 2016-05-28 18:42:22 +03:00
parent c1eae3e234
commit 8c4abcdd43
113 changed files with 549 additions and 606 deletions

View File

@ -2,8 +2,6 @@
#include <memory>
#include <Poco/SharedPtr.h>
#include <DB/Core/Row.h>
#include <DB/DataTypes/IDataType.h>
#include <DB/Common/typeid_cast.h>
@ -149,8 +147,6 @@ public:
};
using Poco::SharedPtr;
using AggregateFunctionPtr = SharedPtr<IAggregateFunction>;
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
}

View File

@ -12,6 +12,8 @@
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
#include <Poco/SharedPtr.h>
namespace DB
{
@ -57,7 +59,7 @@ private:
struct Holder
{
using Ptr = SharedPtr<Holder>;
using Ptr = Poco::SharedPtr<Holder>;
AggregateFunctionPtr func; /// Используется для уничтожения состояний и для финализации значений.
const Ptr src; /// Источник. Используется, если данный столбец создан из другого и использует все или часть его значений.
@ -68,11 +70,9 @@ private:
~Holder()
{
IAggregateFunction * function = func;
if (!function->hasTrivialDestructor() && src.isNull())
if (!func->hasTrivialDestructor() && src.isNull())
for (auto val : data)
function->destroy(val);
func->destroy(val);
}
void popBack(size_t n)
@ -188,7 +188,7 @@ public:
void insert(const Field & x) override
{
IAggregateFunction * function = holder.get()->func;
IAggregateFunction * function = holder.get()->func.get();
Arena & arena = createOrGetArena();
@ -200,7 +200,7 @@ public:
void insertDefault() override
{
IAggregateFunction * function = holder.get()->func;
IAggregateFunction * function = holder.get()->func.get();
Arena & arena = createOrGetArena();

View File

@ -19,7 +19,7 @@ class FunctionFactory : public Singleton<FunctionFactory>
friend class StorageSystemFunctions;
private:
typedef IFunction* (*Creator)(const Context & context); /// Не std::function, так как меньше indirection и размер объекта.
typedef FunctionPtr (*Creator)(const Context & context); /// Не std::function, так как меньше indirection и размер объекта.
std::unordered_map<String, Creator> functions;
public:

View File

@ -469,7 +469,7 @@ class FunctionBinaryArithmetic : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionBinaryArithmetic; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBinaryArithmetic>(); }
private:
/// Overload for InvalidType
@ -729,7 +729,7 @@ class FunctionUnaryArithmetic : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionUnaryArithmetic; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionUnaryArithmetic>(); }
private:
template <typename T0>

View File

@ -71,7 +71,7 @@ class FunctionArray : public IFunction
{
public:
static constexpr auto name = "array";
static IFunction * create(const Context & context) { return new FunctionArray; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArray>(); }
private:
/// Получить имя функции.
@ -479,7 +479,7 @@ class FunctionArrayElement : public IFunction
{
public:
static constexpr auto name = "arrayElement";
static IFunction * create(const Context & context) { return new FunctionArrayElement; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayElement>(); }
private:
template <typename T>
@ -958,7 +958,7 @@ class FunctionArrayIndex : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionArrayIndex; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayIndex>(); }
private:
using ResultColumnType = ColumnVector<typename IndexConv::ResultType>;
@ -1162,7 +1162,7 @@ class FunctionArrayEnumerate : public IFunction
{
public:
static constexpr auto name = "arrayEnumerate";
static IFunction * create (const Context & context) { return new FunctionArrayEnumerate; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayEnumerate>(); }
/// Получить имя функции.
String getName() const override
@ -1238,7 +1238,7 @@ class FunctionArrayUniq : public IFunction
{
public:
static constexpr auto name = "arrayUniq";
static IFunction * create(const Context & context) { return new FunctionArrayUniq; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayUniq>(); }
/// Получить имя функции.
String getName() const override
@ -1468,7 +1468,7 @@ class FunctionArrayEnumerateUniq : public IFunction
{
public:
static constexpr auto name = "arrayEnumerateUniq";
static IFunction * create(const Context & context) { return new FunctionArrayEnumerateUniq; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayEnumerateUniq>(); }
/// Получить имя функции.
String getName() const override
@ -1713,7 +1713,7 @@ struct FunctionEmptyArray : public IFunction
{
static constexpr auto base_name = "emptyArray";
static const String name;
static IFunction * create(const Context & context) { return new FunctionEmptyArray; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionEmptyArray>(); }
private:
String getName() const override
@ -1749,7 +1749,7 @@ class FunctionRange : public IFunction
public:
static constexpr auto max_elements = 100000000;
static constexpr auto name = "range";
static IFunction * create(const Context &) { return new FunctionRange; }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRange>(); }
private:
String getName() const override
@ -1888,7 +1888,7 @@ class FunctionEmptyArrayToSingle : public IFunction
{
public:
static constexpr auto name = "emptyArrayToSingle";
static IFunction * create(const Context & context) { return new FunctionEmptyArrayToSingle; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionEmptyArrayToSingle>(); }
/// Получить имя функции.
String getName() const override
@ -2138,7 +2138,7 @@ class FunctionArrayReverse : public IFunction
{
public:
static constexpr auto name = "reverse";
static IFunction * create(const Context & context) { return new FunctionArrayReverse; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayReverse>(); }
/// Получить имя функции.
String getName() const override
@ -2360,7 +2360,7 @@ class FunctionArrayReduce : public IFunction
{
public:
static constexpr auto name = "arrayReduce";
static IFunction * create(const Context & context) { return new FunctionArrayReduce; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayReduce>(); }
/// Получить имя функции.
String getName() const override

View File

@ -171,7 +171,7 @@ class FunctionIPv6NumToString : public IFunction
{
public:
static constexpr auto name = "IPv6NumToString";
static IFunction * create(const Context & context) { return new FunctionIPv6NumToString; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIPv6NumToString>(); }
String getName() const override { return name; }
@ -257,7 +257,7 @@ class FunctionCutIPv6 : public IFunction
{
public:
static constexpr auto name = "cutIPv6";
static IFunction * create(const Context & context) { return new FunctionCutIPv6; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionCutIPv6>(); }
String getName() const override { return name; }
@ -423,7 +423,7 @@ class FunctionIPv6StringToNum : public IFunction
{
public:
static constexpr auto name = "IPv6StringToNum";
static IFunction * create(const Context & context) { return new FunctionIPv6StringToNum; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIPv6StringToNum>(); }
String getName() const override { return name; }
@ -638,7 +638,7 @@ class FunctionIPv4NumToString : public IFunction
{
public:
static constexpr auto name = "IPv4NumToString";
static IFunction * create(const Context & context) { return new FunctionIPv4NumToString; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIPv4NumToString>(); }
/// Получить имя функции.
String getName() const override
@ -743,7 +743,7 @@ class FunctionIPv4StringToNum : public IFunction
{
public:
static constexpr auto name = "IPv4StringToNum";
static IFunction * create(const Context & context) { return new FunctionIPv4StringToNum; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIPv4StringToNum>(); }
/// Получить имя функции.
String getName() const override
@ -834,7 +834,7 @@ class FunctionIPv4NumToStringClassC : public IFunction
{
public:
static constexpr auto name = "IPv4NumToStringClassC";
static IFunction * create(const Context & context) { return new FunctionIPv4NumToStringClassC; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIPv4NumToStringClassC>(); }
/// Получить имя функции.
String getName() const override
@ -942,7 +942,7 @@ class FunctionIPv4ToIPv6 : public IFunction
{
public:
static constexpr auto name = "IPv4ToIPv6";
static IFunction * create(const Context & context) { return new FunctionIPv4ToIPv6; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIPv4ToIPv6>(); }
String getName() const override { return name; }
@ -1007,7 +1007,7 @@ class FunctionHex : public IFunction
{
public:
static constexpr auto name = "hex";
static IFunction * create(const Context & context) { return new FunctionHex; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionHex>(); }
/// Получить имя функции.
String getName() const override
@ -1255,7 +1255,7 @@ class FunctionUnhex : public IFunction
{
public:
static constexpr auto name = "unhex";
static IFunction * create(const Context & context) { return new FunctionUnhex; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionUnhex>(); }
/// Получить имя функции.
String getName() const override
@ -1363,7 +1363,7 @@ class FunctionBitmaskToArray : public IFunction
{
public:
static constexpr auto name = "bitmaskToArray";
static IFunction * create(const Context & context) { return new FunctionBitmaskToArray; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBitmaskToArray>(); }
/// Получить имя функции.
String getName() const override
@ -1476,7 +1476,7 @@ class FunctionToStringCutToZero : public IFunction
{
public:
static constexpr auto name = "toStringCutToZero";
static IFunction * create(const Context & context) { return new FunctionToStringCutToZero; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionToStringCutToZero>(); }
/// Получить имя функции.
String getName() const override
@ -1624,7 +1624,7 @@ class FunctionBitTest : public IFunction
{
public:
static constexpr auto name = "bitTest";
static IFunction * create(const Context &) { return new FunctionBitTest; }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionBitTest>(); }
String getName() const override { return name; }
@ -1805,7 +1805,7 @@ struct FunctionBitTestMany : public IFunction
{
public:
static constexpr auto name = Impl::name;
static IFunction * create(const Context &) { return new FunctionBitTestMany; }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionBitTestMany>(); }
String getName() const override { return name; }

View File

@ -404,7 +404,7 @@ class FunctionComparison : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionComparison; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionComparison>(); };
private:
template <typename T0, typename T1>

View File

@ -821,7 +821,7 @@ class FunctionIf : public IFunction
{
public:
static constexpr auto name = "if";
static IFunction * create(const Context & context) { return new FunctionIf; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIf>(); }
private:
template <typename T0, typename T1>
@ -1399,7 +1399,7 @@ class FunctionMultiIf final : public IFunction
{
public:
static constexpr auto name = "multiIf";
static IFunction * create(const Context & context) { return new FunctionMultiIf; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionMultiIf>(); }
public:
String getName() const override

View File

@ -1066,7 +1066,7 @@ public:
using Monotonic = MonotonicityImpl;
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionConvert; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionConvert>(); }
/// Получить имя функции.
String getName() const override
@ -1213,7 +1213,7 @@ class FunctionToFixedString : public IFunction
{
public:
static constexpr auto name = "toFixedString";
static IFunction * create(const Context & context) { return new FunctionToFixedString; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionToFixedString>(); };
/// Получить имя функции.
String getName() const override
@ -1870,7 +1870,7 @@ class FunctionCast final : public IFunction
public:
static constexpr auto name = "CAST";
static IFunction * create(const Context & context) { return new FunctionCast{context}; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionCast>(context); }
String getName() const override { return name; }

View File

@ -571,7 +571,7 @@ class FunctionDateOrDateTimeToSomething : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionDateOrDateTimeToSomething; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionDateOrDateTimeToSomething>(); };
/// Получить имя функции.
String getName() const override
@ -669,7 +669,7 @@ class FunctionNow : public IFunction
{
public:
static constexpr auto name = "now";
static IFunction * create(const Context & context) { return new FunctionNow; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionNow>(); };
/// Получить имя функции.
String getName() const override
@ -702,7 +702,7 @@ class FunctionToday : public IFunction
{
public:
static constexpr auto name = "today";
static IFunction * create(const Context & context) { return new FunctionToday; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionToday>(); };
/// Получить имя функции.
String getName() const override
@ -735,7 +735,7 @@ class FunctionYesterday : public IFunction
{
public:
static constexpr auto name = "yesterday";
static IFunction * create(const Context & context) { return new FunctionYesterday; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionYesterday>(); };
/// Получить имя функции.
String getName() const override
@ -768,7 +768,7 @@ class FunctionTimeSlot : public IFunction
{
public:
static constexpr auto name = "timeSlot";
static IFunction * create(const Context & context) { return new FunctionTimeSlot; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionTimeSlot>(); };
/// Получить имя функции.
String getName() const override
@ -904,7 +904,7 @@ class FunctionTimeSlots : public IFunction
{
public:
static constexpr auto name = "timeSlots";
static IFunction * create(const Context & context) { return new FunctionTimeSlots; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionTimeSlots>(); };
/// Получить имя функции.
String getName() const override

View File

@ -540,7 +540,7 @@ struct NameSEHierarchy { static constexpr auto name = "SEHierarchy"; };
struct FunctionRegionToCity :
public FunctionTransformWithDictionary<UInt32, RegionToCityImpl, RegionsHierarchyGetter, NameRegionToCity>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getRegionsHierarchies()};
}
@ -549,7 +549,7 @@ struct FunctionRegionToCity :
struct FunctionRegionToArea :
public FunctionTransformWithDictionary<UInt32, RegionToAreaImpl, RegionsHierarchyGetter, NameRegionToArea>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getRegionsHierarchies()};
}
@ -558,7 +558,7 @@ struct FunctionRegionToArea :
struct FunctionRegionToDistrict :
public FunctionTransformWithDictionary<UInt32, RegionToDistrictImpl, RegionsHierarchyGetter, NameRegionToDistrict>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getRegionsHierarchies()};
}
@ -567,7 +567,7 @@ struct FunctionRegionToDistrict :
struct FunctionRegionToCountry :
public FunctionTransformWithDictionary<UInt32, RegionToCountryImpl, RegionsHierarchyGetter, NameRegionToCountry>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getRegionsHierarchies()};
}
@ -576,7 +576,7 @@ struct FunctionRegionToCountry :
struct FunctionRegionToContinent :
public FunctionTransformWithDictionary<UInt32, RegionToContinentImpl, RegionsHierarchyGetter, NameRegionToContinent>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getRegionsHierarchies()};
}
@ -585,7 +585,7 @@ struct FunctionRegionToContinent :
struct FunctionRegionToTopContinent :
public FunctionTransformWithDictionary<UInt32, RegionToTopContinentImpl, RegionsHierarchyGetter, NameRegionToTopContinent>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getRegionsHierarchies()};
}
@ -594,7 +594,7 @@ struct FunctionRegionToTopContinent :
struct FunctionRegionToPopulation :
public FunctionTransformWithDictionary<UInt32, RegionToPopulationImpl, RegionsHierarchyGetter, NameRegionToPopulation>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getRegionsHierarchies()};
}
@ -603,7 +603,7 @@ struct FunctionRegionToPopulation :
struct FunctionOSToRoot :
public FunctionTransformWithDictionary<UInt8, OSToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSToRoot>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getTechDataHierarchy()};
}
@ -612,7 +612,7 @@ struct FunctionOSToRoot :
struct FunctionSEToRoot :
public FunctionTransformWithDictionary<UInt8, SEToRootImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEToRoot>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getTechDataHierarchy()};
}
@ -621,7 +621,7 @@ struct FunctionSEToRoot :
struct FunctionRegionIn :
public FunctionIsInWithDictionary<UInt32, RegionInImpl, RegionsHierarchyGetter, NameRegionIn>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getRegionsHierarchies()};
}
@ -630,7 +630,7 @@ struct FunctionRegionIn :
struct FunctionOSIn :
public FunctionIsInWithDictionary<UInt8, OSInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSIn>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getTechDataHierarchy()};
}
@ -639,7 +639,7 @@ struct FunctionOSIn :
struct FunctionSEIn :
public FunctionIsInWithDictionary<UInt8, SEInImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEIn>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getTechDataHierarchy()};
}
@ -648,7 +648,7 @@ struct FunctionSEIn :
struct FunctionRegionHierarchy :
public FunctionHierarchyWithDictionary<UInt32, RegionHierarchyImpl, RegionsHierarchyGetter, NameRegionHierarchy>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getRegionsHierarchies()};
}
@ -657,7 +657,7 @@ struct FunctionRegionHierarchy :
struct FunctionOSHierarchy :
public FunctionHierarchyWithDictionary<UInt8, OSHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameOSHierarchy>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getTechDataHierarchy()};
}
@ -666,7 +666,7 @@ struct FunctionOSHierarchy :
struct FunctionSEHierarchy :
public FunctionHierarchyWithDictionary<UInt8, SEHierarchyImpl, IdentityDictionaryGetter<TechDataHierarchy>, NameSEHierarchy>
{
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new base_type{context.getDictionaries().getTechDataHierarchy()};
}
@ -678,9 +678,9 @@ class FunctionRegionToName : public IFunction
{
public:
static constexpr auto name = "regionToName";
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new FunctionRegionToName(context.getDictionaries().getRegionsNames());
return std::make_shared<FunctionRegionToName>(context.getDictionaries().getRegionsNames());
}
private:
@ -772,9 +772,9 @@ class FunctionDictHas final : public IFunction
public:
static constexpr auto name = "dictHas";
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new FunctionDictHas{context.getExternalDictionaries()};
return std::make_shared<FunctionDictHas>(context.getExternalDictionaries());
}
FunctionDictHas(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
@ -915,9 +915,9 @@ class FunctionDictGetString final : public IFunction
public:
static constexpr auto name = "dictGetString";
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new FunctionDictGetString{context.getExternalDictionaries()};
return std::make_shared<FunctionDictGetString>(context.getExternalDictionaries());
}
FunctionDictGetString(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
@ -1190,9 +1190,9 @@ class FunctionDictGetStringOrDefault final : public IFunction
public:
static constexpr auto name = "dictGetStringOrDefault";
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new FunctionDictGetStringOrDefault{context.getExternalDictionaries()};
return std::make_shared<FunctionDictGetStringOrDefault>(context.getExternalDictionaries());
}
FunctionDictGetStringOrDefault(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
@ -1483,9 +1483,9 @@ class FunctionDictGet final : public IFunction
public:
static const std::string name;
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new FunctionDictGet{context.getExternalDictionaries()};
return std::make_shared<FunctionDictGet>(context.getExternalDictionaries());
}
FunctionDictGet(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
@ -1795,9 +1795,9 @@ class FunctionDictGetOrDefault final : public IFunction
public:
static const std::string name;
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new FunctionDictGetOrDefault{context.getExternalDictionaries()};
return std::make_shared<FunctionDictGetOrDefault>(context.getExternalDictionaries());
}
FunctionDictGetOrDefault(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
@ -2066,9 +2066,9 @@ class FunctionDictGetHierarchy final : public IFunction
public:
static constexpr auto name = "dictGetHierarchy";
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new FunctionDictGetHierarchy{context.getExternalDictionaries()};
return std::make_shared<FunctionDictGetHierarchy>(context.getExternalDictionaries());
}
FunctionDictGetHierarchy(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
@ -2229,9 +2229,9 @@ class FunctionDictIsIn final : public IFunction
public:
static constexpr auto name = "dictIsIn";
static IFunction * create(const Context & context)
static FunctionPtr create(const Context & context)
{
return new FunctionDictIsIn{context.getExternalDictionaries()};
return std::make_shared<FunctionDictIsIn>(context.getExternalDictionaries());
}
FunctionDictIsIn(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}

View File

@ -22,7 +22,7 @@ class FunctionBitmaskToList : public IFunction
{
public:
static constexpr auto name = "bitmaskToList";
static IFunction * create(const Context & context) { return new FunctionBitmaskToList; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBitmaskToList>(); }
/// Получить основное имя функции.
virtual String getName() const override
@ -135,7 +135,7 @@ class FunctionFormatReadableSize : public IFunction
{
public:
static constexpr auto name = "formatReadableSize";
static IFunction * create(const Context & context) { return new FunctionFormatReadableSize; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionFormatReadableSize>(); }
/// Получить основное имя функции.
virtual String getName() const override

View File

@ -171,7 +171,7 @@ class FunctionStringHash64 : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionStringHash64; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionStringHash64>(); };
/// Получить имя функции.
String getName() const override
@ -232,7 +232,7 @@ class FunctionStringHashFixedString : public IFunction
{
public:
static constexpr auto name = Impl::name;
static IFunction * create(const Context & context) { return new FunctionStringHashFixedString; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionStringHashFixedString>(); };
/// Получить имя функции.
String getName() const override
@ -300,7 +300,7 @@ class FunctionIntHash : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionIntHash; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIntHash>(); };
private:
using ToType = typename Impl::ReturnType;
@ -396,7 +396,7 @@ class FunctionNeighbourhoodHash64 : public IFunction
{
public:
static constexpr auto name = Impl::name;
static IFunction * create(const Context & context) { return new FunctionNeighbourhoodHash64; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionNeighbourhoodHash64>(); };
private:
template <typename FromType, bool first>
@ -699,7 +699,7 @@ class FunctionURLHash : public IFunction
{
public:
static constexpr auto name = "URLHash";
static IFunction * create(const Context &) { return new FunctionURLHash; }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionURLHash>(); }
String getName() const override { return name; }

View File

@ -545,7 +545,7 @@ class FunctionArrayMapped : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionArrayMapped; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayMapped>(); };
/// Получить имя функции.
String getName() const override
@ -640,7 +640,7 @@ public:
Names replicate_arguments;
replicate_arguments.push_back(required_columns[i]);
replicate_arguments.push_back(arguments[1].name);
out_prerequisites.push_back(ExpressionAction::applyFunction(new FunctionReplicate, replicate_arguments));
out_prerequisites.push_back(ExpressionAction::applyFunction(std::make_shared<FunctionReplicate>(), replicate_arguments));
}
DataTypePtr return_type = column_expression->getReturnType();

View File

@ -154,7 +154,7 @@ class FunctionAnyArityLogical : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionAnyArityLogical; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionAnyArityLogical>(); };
private:
bool extractConstColumns(ColumnPlainPtrs & in, UInt8 & res)
@ -363,7 +363,7 @@ class FunctionUnaryLogical : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionUnaryLogical; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionUnaryLogical>(); };
private:
template <typename T>

View File

@ -17,7 +17,7 @@ class FunctionMathNullaryConstFloat64 : public IFunction
{
public:
static constexpr auto name = Impl::name;
static IFunction * create(const Context &) { return new FunctionMathNullaryConstFloat64; }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathNullaryConstFloat64>(); }
private:
String getName() const override { return name; }
@ -47,7 +47,7 @@ template <typename Impl> class FunctionMathUnaryFloat64 : public IFunction
{
public:
static constexpr auto name = Impl::name;
static IFunction * create(const Context &) { return new FunctionMathUnaryFloat64; }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathUnaryFloat64>(); }
static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration");
private:
@ -198,7 +198,7 @@ template <typename Impl> class FunctionMathBinaryFloat64 : public IFunction
{
public:
static constexpr auto name = Impl::name;
static IFunction * create(const Context &) { return new FunctionMathBinaryFloat64; }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionMathBinaryFloat64>(); }
static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration");
private:

View File

@ -98,7 +98,7 @@ class FunctionCurrentDatabase : public IFunction
public:
static constexpr auto name = "currentDatabase";
static IFunction * create(const Context & context) { return new FunctionCurrentDatabase{context.getCurrentDatabase()}; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionCurrentDatabase>(context.getCurrentDatabase()); }
explicit FunctionCurrentDatabase(const String & db_name) : db_name{db_name} {}
@ -129,7 +129,7 @@ class FunctionHostName : public IFunction
{
public:
static constexpr auto name = "hostName";
static IFunction * create(const Context & context) { return new FunctionHostName; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionHostName>(); }
/// Получить имя функции.
String getName() const override
@ -163,7 +163,7 @@ class FunctionVisibleWidth : public IFunction
{
public:
static constexpr auto name = "visibleWidth";
static IFunction * create(const Context & context) { return new FunctionVisibleWidth; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionVisibleWidth>(); }
/// Получить имя функции.
String getName() const override
@ -191,7 +191,7 @@ class FunctionToTypeName : public IFunction
{
public:
static constexpr auto name = "toTypeName";
static IFunction * create(const Context & context) { return new FunctionToTypeName; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionToTypeName>(); }
/// Получить имя функции.
String getName() const override
@ -222,7 +222,7 @@ class FunctionBlockSize : public IFunction
{
public:
static constexpr auto name = "blockSize";
static IFunction * create(const Context & context) { return new FunctionBlockSize; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBlockSize>(); }
/// Получить имя функции.
String getName() const override
@ -254,7 +254,7 @@ class FunctionRowNumberInBlock : public IFunction
{
public:
static constexpr auto name = "rowNumberInBlock";
static IFunction * create(const Context & context) { return new FunctionRowNumberInBlock; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionRowNumberInBlock>(); }
/// Получить имя функции.
String getName() const override
@ -295,7 +295,7 @@ private:
public:
static constexpr auto name = "blockNumber";
static IFunction * create(const Context & context) { return new FunctionBlockNumber; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBlockNumber>(); }
/// Получить имя функции.
String getName() const override
@ -327,7 +327,7 @@ class FunctionSleep : public IFunction
{
public:
static constexpr auto name = "sleep";
static IFunction * create(const Context & context) { return new FunctionSleep; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionSleep>(); }
/// Получить имя функции.
String getName() const override
@ -396,7 +396,7 @@ class FunctionMaterialize : public IFunction
{
public:
static constexpr auto name = "materialize";
static IFunction * create(const Context & context) { return new FunctionMaterialize; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionMaterialize>(); }
/// Получить имя функции.
String getName() const override
@ -437,7 +437,7 @@ class FunctionIn : public IFunction
{
public:
static constexpr auto name = FunctionInName<negative, global>::name;
static IFunction * create(const Context & context) { return new FunctionIn; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIn>(); }
/// Получить имя функции.
String getName() const override
@ -484,7 +484,7 @@ class FunctionTuple : public IFunction
{
public:
static constexpr auto name = "tuple";
static IFunction * create(const Context & context) { return new FunctionTuple; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionTuple>(); }
/// Получить имя функции.
String getName() const override
@ -518,7 +518,7 @@ class FunctionTupleElement : public IFunction
{
public:
static constexpr auto name = "tupleElement";
static IFunction * create(const Context & context) { return new FunctionTupleElement; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionTupleElement>(); }
/// Получить имя функции.
String getName() const override
@ -585,7 +585,7 @@ class FunctionIgnore : public IFunction
{
public:
static constexpr auto name = "ignore";
static IFunction * create(const Context & context) { return new FunctionIgnore; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIgnore>(); }
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override { return std::make_shared<DataTypeUInt8>(); }
@ -614,7 +614,7 @@ class FunctionIndexHint : public IFunction
{
public:
static constexpr auto name = "indexHint";
static IFunction * create(const Context & context) { return new FunctionIndexHint; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIndexHint>(); }
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override { return std::make_shared<DataTypeUInt8>(); }
@ -631,7 +631,7 @@ class FunctionIdentity : public IFunction
{
public:
static constexpr auto name = "identity";
static IFunction * create(const Context & context) { return new FunctionIdentity; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIdentity>(); }
/// Получить имя функции.
String getName() const override
@ -661,7 +661,7 @@ class FunctionArrayJoin : public IFunction
{
public:
static constexpr auto name = "arrayJoin";
static IFunction * create(const Context & context) { return new FunctionArrayJoin; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayJoin>(); }
/// Получить имя функции.
@ -698,7 +698,7 @@ class FunctionReplicate : public IFunction
{
public:
static constexpr auto name = "replicate";
static IFunction * create(const Context & context) { return new FunctionReplicate; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionReplicate>(); }
/// Получить имя функции.
String getName() const override
@ -749,7 +749,7 @@ class FunctionBar : public IFunction
{
public:
static constexpr auto name = "bar";
static IFunction * create(const Context & context) { return new FunctionBar; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionBar>(); }
/// Получить имя функции.
String getName() const override
@ -910,7 +910,7 @@ class FunctionNumericPredicate : public IFunction
{
public:
static constexpr auto name = Impl::name;
static IFunction * create(const Context &) { return new FunctionNumericPredicate; }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionNumericPredicate>(); }
String getName() const override { return name; }
@ -1021,7 +1021,7 @@ class FunctionVersion : public IFunction
{
public:
static constexpr auto name = "version";
static IFunction * create(const Context & context) { return new FunctionVersion; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionVersion>(); }
String getName() const override { return name; }
@ -1052,7 +1052,7 @@ class FunctionUptime : public IFunction
{
public:
static constexpr auto name = "uptime";
static IFunction * create(const Context & context) { return new FunctionUptime(context.getUptimeSeconds()); }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionUptime>(context.getUptimeSeconds()); }
FunctionUptime(time_t uptime_) : uptime(uptime_) {}
@ -1085,7 +1085,7 @@ class FunctionRunningAccumulate : public IFunction
{
public:
static constexpr auto name = "runningAccumulate";
static IFunction * create(const Context & context) { return new FunctionRunningAccumulate; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionRunningAccumulate>(); }
String getName() const override { return name; }
@ -1139,7 +1139,7 @@ class FunctionFinalizeAggregation : public IFunction
{
public:
static constexpr auto name = "finalizeAggregation";
static IFunction * create(const Context & context) { return new FunctionFinalizeAggregation; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionFinalizeAggregation>(); }
String getName() const override { return name; }

View File

@ -154,7 +154,7 @@ private:
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionRandom; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionRandom>(); }
/// Получить имя функции.
String getName() const override
@ -200,7 +200,7 @@ private:
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionRandomConstant; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionRandomConstant>(); }
/// Получить имя функции.
String getName() const override

View File

@ -24,7 +24,7 @@ class FunctionReinterpretAsStringImpl : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionReinterpretAsStringImpl; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionReinterpretAsStringImpl>(); };
/// Получить имя функции.
String getName() const override
@ -120,7 +120,7 @@ class FunctionReinterpretStringAs : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionReinterpretStringAs; };
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionReinterpretStringAs>(); };
using ToFieldType = typename ToDataType::FieldType;

View File

@ -1034,7 +1034,7 @@ namespace
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionRounding; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionRounding>(); }
private:
template<typename T>

View File

@ -775,7 +775,7 @@ class FunctionStringOrArrayToT : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionStringOrArrayToT; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionStringOrArrayToT>(); }
/// Получить имя функции.
String getName() const override
@ -871,7 +871,7 @@ class FunctionStringToString : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionStringToString; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionStringToString>(); }
/// Получить имя функции.
String getName() const override
@ -932,7 +932,7 @@ class FunctionReverse : public IFunction
{
public:
static constexpr auto name = "reverse";
static IFunction * create(const Context & context) { return new FunctionReverse; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionReverse>(); }
/// Получить имя функции.
String getName() const override
@ -998,7 +998,7 @@ class ConcatImpl : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new ConcatImpl; }
static FunctionPtr create(const Context & context) { return new ConcatImpl; }
/// Получить имя функции.
String getName() const override
@ -1469,7 +1469,7 @@ class FunctionStringNumNumToString : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionStringNumNumToString; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionStringNumNumToString>(); }
/// Получить имя функции.
String getName() const override
@ -1550,7 +1550,7 @@ class FunctionAppendTrailingCharIfAbsent : public IFunction
{
public:
static constexpr auto name = "appendTrailingCharIfAbsent";
static IFunction * create(const Context & context) { return new FunctionAppendTrailingCharIfAbsent; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionAppendTrailingCharIfAbsent>(); }
String getName() const override
{

View File

@ -314,7 +314,7 @@ class FunctionTokens : public IFunction
{
public:
static constexpr auto name = Generator::name;
static IFunction * create(const Context & context) { return new FunctionTokens; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionTokens>(); }
/// Получить имя функции.
String getName() const override
@ -479,7 +479,7 @@ private:
public:
static constexpr auto name = "arrayStringConcat";
static IFunction * create(const Context & context) { return new FunctionArrayStringConcat; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayStringConcat>(); }
/// Получить имя функции.
String getName() const override

View File

@ -1161,7 +1161,7 @@ class FunctionStringReplace : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionStringReplace; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionStringReplace>(); }
/// Получить имя функции.
String getName() const override
@ -1248,7 +1248,7 @@ class FunctionsStringSearch : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionsStringSearch; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionsStringSearch>(); }
/// Получить имя функции.
String getName() const override
@ -1333,7 +1333,7 @@ class FunctionsStringSearchToString : public IFunction
{
public:
static constexpr auto name = Name::name;
static IFunction * create(const Context & context) { return new FunctionsStringSearchToString; }
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionsStringSearchToString>(); }
/// Получить имя функции.
String getName() const override

View File

@ -48,7 +48,7 @@ class FunctionTransform : public IFunction
{
public:
static constexpr auto name = "transform";
static IFunction * create(const Context &) { return new FunctionTransform; }
static FunctionPtr create(const Context &) { return std::make_shared<FunctionTransform>(); }
String getName() const override
{

View File

@ -1,6 +1,6 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <memory>
#include <DB/Core/Names.h>
#include <DB/Core/Block.h>
@ -122,9 +122,7 @@ public:
};
using Poco::SharedPtr;
using FunctionPtr = SharedPtr<IFunction>;
using FunctionPtr = std::shared_ptr<IFunction>;
}

View File

@ -271,8 +271,8 @@ private:
if (sub_select_query.database.isNull())
{
sub_select_query.database = new ASTIdentifier{{}, distributed_storage.getRemoteDatabaseName(),
ASTIdentifier::Database};
sub_select_query.database = std::make_shared<ASTIdentifier>({}, distributed_storage.getRemoteDatabaseName(),
ASTIdentifier::Database);
/// Поскольку был создан новый узел для БД, необходимо его вставить в список
/// потомков этого подзапроса. См. ParserSelectQuery для структуры потомков.

View File

@ -1,6 +1,6 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <memory>
#include <DB/Core/Field.h>
@ -14,6 +14,6 @@ class Context;
* Используется в редких случаях - для элемента множества в IN, для данных для INSERT.
* Весьма неоптимально.
*/
Field evaluateConstantExpression(Poco::SharedPtr<IAST> & node, const Context & context);
Field evaluateConstantExpression(std::shared_ptr<IAST> & node, const Context & context);
}

View File

@ -1,6 +1,6 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <memory>
namespace DB
@ -13,6 +13,6 @@ class Context;
/** \brief if `expr` is not already ASTIdentifier evaluates it
* and replaces by a new ASTIdentifier with the result of evaluation as its name.
* `expr` must evaluate to a String type */
ASTIdentifier & reinterpretAsIdentifier(Poco::SharedPtr<IAST> & expr, const Context & context);
ASTIdentifier & reinterpretAsIdentifier(std::shared_ptr<IAST> & expr, const Context & context);
}

View File

@ -14,7 +14,7 @@ public:
ASTAsterisk() = default;
ASTAsterisk(StringRange range_) : IAST(range_) {}
String getID() const override { return "Asterisk"; }
ASTPtr clone() const override { return new ASTAsterisk(*this); }
ASTPtr clone() const override { return std::make_shared<ASTAsterisk>(*this); }
String getColumnName() const override { return "*"; }
protected:

View File

@ -14,7 +14,7 @@ struct ASTCheckQuery : public ASTQueryWithOutput
ASTPtr clone() const override
{
return new ASTCheckQuery(*this);
return std::make_shared<ASTCheckQuery>(*this);
}
std::string database;

View File

@ -23,7 +23,7 @@ public:
ASTPtr clone() const override
{
const auto res = new ASTColumnDeclaration{*this};
const auto res = std::make_shared<ASTColumnDeclaration>(*this);
ASTPtr ptr{res};
res->children.clear();

View File

@ -36,9 +36,7 @@ public:
ASTPtr clone() const override
{
ASTCreateQuery * res = new ASTCreateQuery(*this);
ASTPtr ptr{res};
auto res = std::make_shared<ASTCreateQuery>(*this);
res->children.clear();
if (columns) { res->columns = columns->clone(); res->children.push_back(res->columns); }
@ -46,7 +44,7 @@ public:
if (select) { res->select = select->clone(); res->children.push_back(res->select); }
if (inner_storage) { res->inner_storage = inner_storage->clone(); res->children.push_back(res->inner_storage); }
return ptr;
return res;
}
protected:

View File

@ -23,7 +23,7 @@ public:
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return (detach ? "DetachQuery_" : "DropQuery_") + database + "_" + table; };
ASTPtr clone() const override { return new ASTDropQuery(*this); }
ASTPtr clone() const override { return std::make_shared<ASTDropQuery>(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override

View File

@ -21,7 +21,7 @@ public:
ASTPtr clone() const override
{
return new ASTEnumElement{{}, name, value};
return std::make_shared<ASTEnumElement>({}, name, value);
}
protected:

View File

@ -19,7 +19,7 @@ public:
ASTPtr clone() const override
{
const auto res = new ASTExpressionList(*this);
const auto res = std::make_shared<ASTExpressionList>(*this);
ASTPtr ptr{res};
res->children.clear();

View File

@ -66,11 +66,11 @@ private:
template <typename... Args>
ASTPtr makeASTFunction(const String & name, Args &&... args)
{
const auto function = new ASTFunction{};
const auto function = std::make_shared<ASTFunction>();
ASTPtr result{function};
function->name = name;
function->arguments = new ASTExpressionList{};
function->arguments = std::make_shared<ASTExpressionList>();
function->children.push_back(function->arguments);
function->arguments->children = { std::forward<Args>(args)... };
@ -83,11 +83,11 @@ template <typename... Args>
ASTPtr makeASTFunction(const String & name, const StringRange & function_range,
const StringRange & arguments_range, Args &&... args)
{
const auto function = new ASTFunction{function_range};
const auto function = std::make_shared<ASTFunction>(function_range);
ASTPtr result{function};
function->name = name;
function->arguments = new ASTExpressionList{arguments_range};
function->arguments = std::make_shared<ASTExpressionList>(arguments_range);
function->children.push_back(function->arguments);
function->arguments->children = { std::forward<Args>(args)... };

View File

@ -36,7 +36,7 @@ public:
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "Identifier_" + name; }
ASTPtr clone() const override { return new ASTIdentifier(*this); }
ASTPtr clone() const override { return std::make_shared<ASTIdentifier>(*this); }
void collectIdentifierNames(IdentifierNameSet & set) const override
{

View File

@ -26,21 +26,19 @@ public:
ASTInsertQuery() = default;
ASTInsertQuery(const StringRange range_) : IAST(range_) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "InsertQuery_" + database + "_" + table; };
ASTPtr clone() const override
{
ASTInsertQuery * res = new ASTInsertQuery(*this);
ASTPtr ptr{res};
auto res = std::make_shared<ASTInsertQuery>(*this);
res->children.clear();
if (columns) { res->columns = columns->clone(); res->children.push_back(res->columns); }
if (select) { res->select = select->clone(); res->children.push_back(res->select); }
return ptr;
return res;
}
protected:

View File

@ -74,15 +74,13 @@ public:
ASTPtr clone() const override
{
ASTJoin * res = new ASTJoin(*this);
ASTPtr ptr{res};
auto res = std::make_shared<ASTJoin>(*this);
res->children.clear();
if (table) { res->table = table->clone(); res->children.push_back(res->table); }
if (using_expr_list) { res->using_expr_list = using_expr_list->clone(); res->children.push_back(res->using_expr_list); }
return ptr;
return res;
}
protected:

View File

@ -24,7 +24,7 @@ public:
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "Literal_" + apply_visitor(FieldVisitorDump(), value); }
ASTPtr clone() const override { return new ASTLiteral(*this); }
ASTPtr clone() const override { return std::make_shared<ASTLiteral>(*this); }
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override

View File

@ -25,14 +25,12 @@ public:
ASTPtr clone() const override
{
ASTNameTypePair * res = new ASTNameTypePair(*this);
ASTPtr ptr{res};
auto res = std::make_shared<ASTNameTypePair>(*this);
res->children.clear();
if (type) { res->type = type->clone(); res->children.push_back(res->type); }
return ptr;
return res;
}
protected:

View File

@ -26,7 +26,7 @@ public:
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "OptimizeQuery_" + database + "_" + table + "_" + partition + "_" + toString(final); };
ASTPtr clone() const override { return new ASTOptimizeQuery(*this); }
ASTPtr clone() const override { return std::make_shared<ASTOptimizeQuery>(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override

View File

@ -25,7 +25,7 @@ public:
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "OrderByElement"; }
ASTPtr clone() const override { return new ASTOrderByElement(*this); }
ASTPtr clone() const override { return std::make_shared<ASTOrderByElement>(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override

View File

@ -33,7 +33,7 @@ public:
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "Rename"; };
ASTPtr clone() const override { return new ASTRenameQuery(*this); }
ASTPtr clone() const override { return std::make_shared<ASTRenameQuery>(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override

View File

@ -28,7 +28,7 @@ public:
String getID() const override { return "SampleRatio_" + toString(ratio); }
ASTPtr clone() const override { return new ASTSampleRatio(*this); }
ASTPtr clone() const override { return std::make_shared<ASTSampleRatio>(*this); }
static String toString(BigNum num);
static String toString(Rational ratio);

View File

@ -20,7 +20,7 @@ public:
ASTSet(const String & column_name_) : column_name(column_name_) {}
ASTSet(const StringRange range_, const String & column_name_) : IAST(range_), column_name(column_name_) {}
String getID() const override { return "Set_" + getColumnName(); }
ASTPtr clone() const override { return new ASTSet(*this); }
ASTPtr clone() const override { return std::make_shared<ASTSet>(*this); }
String getColumnName() const override { return column_name; }
protected:

View File

@ -31,7 +31,7 @@ public:
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "Set"; };
ASTPtr clone() const override { return new ASTSetQuery(*this); }
ASTPtr clone() const override { return std::make_shared<ASTSetQuery>(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override

View File

@ -27,9 +27,7 @@ public:
ASTPtr clone() const override
{
ASTShowTablesQuery * res = new ASTShowTablesQuery(*this);
ASTPtr ptr{res};
auto res = std::make_shared<ASTShowTablesQuery>(*this);
res->children.clear();
if (format)
@ -38,7 +36,7 @@ public:
res->children.push_back(res->format);
}
return ptr;
return res;
}
protected:

View File

@ -22,7 +22,7 @@ public:
ASTPtr clone() const override
{
const auto res = new ASTSubquery{*this};
const auto res = std::make_shared<ASTSubquery>(*this);
ASTPtr ptr{res};
res->children.clear();

View File

@ -20,7 +20,7 @@ public:
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "UseQuery_" + database; };
ASTPtr clone() const override { return new ASTUseQuery(*this); }
ASTPtr clone() const override { return std::make_shared<ASTUseQuery>(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override

View File

@ -13,7 +13,7 @@ public:
ASTWeightedZooKeeperPath() = default;
ASTWeightedZooKeeperPath(StringRange range_) : IAST(range_) {}
String getID() const override { return "Weighted_ZooKeeper_Path"; }
ASTPtr clone() const override { return new ASTWeightedZooKeeperPath(*this); }
ASTPtr clone() const override { return std::make_shared<ASTWeightedZooKeeperPath>(*this); }
public:
String path;

View File

@ -4,10 +4,9 @@
#include <set>
#include <sstream>
#include <iostream>
#include <memory>
#include <set>
#include <Poco/SharedPtr.h>
#include <common/Common.h>
#include <DB/Core/Types.h>
@ -28,16 +27,18 @@ namespace ErrorCodes
extern const int UNKNOWN_ELEMENT_IN_AST;
}
using Poco::SharedPtr;
using IdentifierNameSet = std::set<String>;
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
using ASTs = std::vector<ASTPtr>;
/** Элемент синтаксического дерева (в дальнейшем - направленного ациклического графа с элементами семантики)
*/
class IAST
{
public:
using ASTs = std::vector<SharedPtr<IAST>>;
ASTs children;
StringRange range;
@ -98,7 +99,7 @@ public:
virtual String getID() const = 0;
/** Получить глубокую копию дерева. */
virtual SharedPtr<IAST> clone() const = 0;
virtual ASTPtr clone() const = 0;
/// Рекурсивно установить атрибуты в поддереве, корнем которого является текущий узел.
void setAttributes(Attributes attributes_)
@ -135,8 +136,8 @@ public:
{
String indent_str(indent, '-');
ostr << indent_str << getID() << ", " << this << std::endl;
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
(*it)->dumpTree(ostr, indent + 1);
for (const auto & child : children)
child->dumpTree(ostr, indent + 1);
}
/** Проверить глубину дерева.
@ -153,8 +154,8 @@ public:
size_t checkSize(size_t max_size) const
{
size_t res = 1;
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
res += (*it)->checkSize(max_size);
for (const auto & child : children)
res += child->checkSize(max_size);
if (res > max_size)
throw Exception("AST is too big. Maximum: " + toString(max_size), ErrorCodes::TOO_BIG_AST);
@ -166,8 +167,8 @@ public:
*/
virtual void collectIdentifierNames(IdentifierNameSet & set) const
{
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
(*it)->collectIdentifierNames(set);
for (const auto & child : children)
child->collectIdentifierNames(set);
}
@ -236,11 +237,11 @@ private:
size_t checkDepthImpl(size_t max_depth, size_t level) const
{
size_t res = level + 1;
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
for (const auto & child : children)
{
if (level >= max_depth)
throw Exception("AST is too deep. Maximum: " + toString(max_depth), ErrorCodes::TOO_DEEP_AST);
res = std::max(res, (*it)->checkDepthImpl(max_depth, level + 1));
res = std::max(res, child->checkDepthImpl(max_depth, level + 1));
}
return res;
@ -248,10 +249,6 @@ private:
};
using ASTPtr = SharedPtr<IAST>;
using ASTs = std::vector<ASTPtr>;
/// Квотировать идентификатор обратными кавычками, если это требуется.
String backQuoteIfNeed(const String & x);

View File

@ -82,11 +82,11 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & nod
&& ws_parser.ignore(pos, end, max_parsed_pos, expected)
&& type_parser.parse(pos, end, type, max_parsed_pos, expected))
{
ASTNameTypePair * name_type_pair = new ASTNameTypePair(StringRange(begin, pos));
node = name_type_pair;
auto name_type_pair = std::make_shared<ASTNameTypePair>(StringRange(begin, pos));
name_type_pair->name = typeid_cast<ASTIdentifier &>(*name).name;
name_type_pair->type = type;
name_type_pair->children.push_back(type);
node = name_type_pair;
return true;
}
@ -167,7 +167,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr
else if (!type)
return false; /// reject sole column name without type
const auto column_declaration = new ASTColumnDeclaration{StringRange{begin, pos}};
const auto column_declaration = std::make_shared<ASTColumnDeclaration>(StringRange{begin, pos});
node = column_declaration;
column_declaration->name = typeid_cast<ASTIdentifier &>(*name).name;
if (type)

View File

@ -36,11 +36,10 @@ protected:
if (!value_parser.parse(pos, end, value, max_parsed_pos, expected))
return false;
node = new ASTEnumElement{
node = std::make_shared<ASTEnumElement>(
{ begin, pos },
static_cast<const ASTLiteral &>(*name).value.get<String>(),
static_cast<const ASTLiteral &>(*value).value
};
static_cast<const ASTLiteral &>(*value).value);
return true;
}

View File

@ -24,8 +24,7 @@ protected:
ParserString s_show("SHOW", true, true);
ParserString s_processlist("PROCESSLIST", true, true);
ASTShowProcesslistQuery * query = new ASTShowProcesslistQuery;
ASTPtr query_ptr = query;
auto query = std::make_shared<ASTShowProcesslistQuery>();
ws.ignore(pos, end);
@ -44,7 +43,7 @@ protected:
return false;
query->range = StringRange(begin, pos);
node = query_ptr;
node = query;
return true;
}

View File

@ -38,10 +38,9 @@ protected:
ws.ignore(pos, end);
ASTUseQuery * query = new ASTUseQuery(StringRange(begin, pos));
node = query;
auto query = std::make_shared<ASTUseQuery>(StringRange(begin, pos));
query->database = typeid_cast<ASTIdentifier &>(*database).name;
node = query;
return true;
}

View File

@ -5,7 +5,7 @@ namespace DB
AggregateFunctionPtr createAggregateFunctionArray(AggregateFunctionPtr & nested)
{
return new AggregateFunctionArray(nested);
return std::make_shared<AggregateFunctionArray>(nested);
}
}

View File

@ -13,7 +13,7 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionAvg>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionAvg>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -9,7 +9,7 @@ namespace
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types)
{
return new AggregateFunctionCount;
return std::make_shared<AggregateFunctionCount>();
}
}

View File

@ -78,7 +78,7 @@ public:
AggregateFunctionPtr createAggregateFunctionDebug(const std::string & name, const DataTypes & argument_types)
{
return new AggregateFunctionDebug;
return std::make_shared<AggregateFunctionDebug>();
}

View File

@ -14,10 +14,10 @@ AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name,
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionGroupArrayNumeric>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionGroupArrayNumeric>(*argument_types[0]));
if (!res)
res = new AggregateFunctionGroupArrayGeneric;
res = std::make_shared<AggregateFunctionGroupArrayGeneric>();
return res;
}

View File

@ -14,7 +14,7 @@ AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & n
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionGroupUniqArray>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionGroupUniqArray>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() +

View File

@ -5,7 +5,7 @@ namespace DB
AggregateFunctionPtr createAggregateFunctionIf(AggregateFunctionPtr & nested)
{
return new AggregateFunctionIf(nested);
return std::make_shared<AggregateFunctionIf>(nested);
}
}

View File

@ -5,7 +5,7 @@ namespace DB
AggregateFunctionPtr createAggregateFunctionMerge(AggregateFunctionPtr & nested)
{
return new AggregateFunctionMerge(nested);
return std::make_shared<AggregateFunctionMerge>(nested);
}
}

View File

@ -15,18 +15,20 @@ AggregateFunctionPtr createAggregateFunctionQuantileExact(const std::string & na
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantileExact<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantileExact<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantileExact<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantileExact<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantileExact<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantileExact<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantileExact<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantileExact<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantileExact<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantileExact<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantileExact<DataTypeDate::FieldType>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantileExact<DataTypeDateTime::FieldType>;
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<UInt8>>();
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<UInt16>>();
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<UInt32>>();
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<UInt64>>();
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<Int8>>();
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<Int16>>();
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<Int32>>();
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<Int64>>();
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<Float32>>();
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileExact<Float64>>();
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return std::make_shared<AggregateFunctionQuantileExact<DataTypeDate::FieldType>>();
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return std::make_shared<AggregateFunctionQuantileExact<DataTypeDateTime::FieldType>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -39,18 +41,20 @@ AggregateFunctionPtr createAggregateFunctionQuantilesExact(const std::string & n
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantilesExact<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantilesExact<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantilesExact<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantilesExact<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantilesExact<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantilesExact<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantilesExact<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantilesExact<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantilesExact<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantilesExact<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantilesExact<DataTypeDate::FieldType>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantilesExact<DataTypeDateTime::FieldType>;
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<UInt8>>();
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<UInt16>>();
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<UInt32>>();
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<UInt64>>();
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<Int8>>();
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<Int16>>();
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<Int32>>();
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<Int64>>();
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<Float32>>();
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesExact<Float64>>();
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return std::make_shared<AggregateFunctionQuantilesExact<DataTypeDate::FieldType>>();
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return std::make_shared<AggregateFunctionQuantilesExact<DataTypeDateTime::FieldType>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -13,7 +13,7 @@ AggregateFunctionPtr createAggregateFunctionQuantileExactWeighted(const std::str
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantileExactWeighted>(*argument_types[0], *argument_types[1]);
AggregateFunctionPtr res(createWithTwoNumericTypes<AggregateFunctionQuantileExactWeighted>(*argument_types[0], *argument_types[1]));
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
@ -27,7 +27,7 @@ AggregateFunctionPtr createAggregateFunctionQuantilesExactWeighted(const std::st
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantilesExactWeighted>(*argument_types[0], *argument_types[1]);
AggregateFunctionPtr res(createWithTwoNumericTypes<AggregateFunctionQuantilesExactWeighted>(*argument_types[0], *argument_types[1]));
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()

View File

@ -5,7 +5,7 @@ namespace DB
AggregateFunctionPtr createAggregateFunctionState(AggregateFunctionPtr & nested)
{
return new AggregateFunctionState(nested);
return std::make_shared<AggregateFunctionState>(nested);
}
}

View File

@ -13,7 +13,7 @@ AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionSum>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionSum>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -14,18 +14,18 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
{
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniqUpTo>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionUniqUpTo>(*argument_types[0]));
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniqUpTo<DataTypeDate::FieldType>;
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDate::FieldType>>();
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return new AggregateFunctionUniqUpTo<DataTypeDateTime::FieldType>;
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDateTime::FieldType>>();
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
return new AggregateFunctionUniqUpTo<String>;
return std::make_shared<AggregateFunctionUniqUpTo<String>>();
else if (typeid_cast<const DataTypeTuple *>(&argument_type))
return new AggregateFunctionUniqUpToVariadic<true>;
return std::make_shared<AggregateFunctionUniqUpToVariadic<true>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -37,7 +37,7 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
throw Exception("Tuple argument of function " + name + " must be the only argument",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return new AggregateFunctionUniqUpToVariadic<false>;
return std::make_shared<AggregateFunctionUniqUpToVariadic<false>>();
}
else
throw Exception("Incorrect number of arguments for aggregate function " + name,

View File

@ -9,32 +9,32 @@ namespace
AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types);
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types));
}
AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types);
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types));
}
AggregateFunctionPtr createAggregateFunctionMin(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types);
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types));
}
AggregateFunctionPtr createAggregateFunctionMax(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types);
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types));
}
AggregateFunctionPtr createAggregateFunctionArgMin(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types);
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types));
}
AggregateFunctionPtr createAggregateFunctionArgMax(const std::string & name, const DataTypes & argument_types)
{
return createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types);
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types));
}
}

View File

@ -15,18 +15,20 @@ AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, c
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantile<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantile<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantile<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantile<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantile<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantile<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantile<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantile<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantile<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantile<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantile<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantile<DataTypeDateTime::FieldType, false>;
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<UInt8>>();
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<UInt16>>();
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<UInt32>>();
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<UInt64>>();
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<Int8>>();
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<Int16>>();
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<Int32>>();
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<Int64>>();
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<Float32>>();
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantile<Float64>>();
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return std::make_shared<AggregateFunctionQuantile<DataTypeDate::FieldType, false>>();
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return std::make_shared<AggregateFunctionQuantile<DataTypeDateTime::FieldType, false>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -39,18 +41,20 @@ AggregateFunctionPtr createAggregateFunctionQuantiles(const std::string & name,
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantiles<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantiles<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantiles<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantiles<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantiles<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantiles<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantiles<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantiles<DataTypeDateTime::FieldType, false>;
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<UInt8>>();
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<UInt16>>();
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<UInt32>>();
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<UInt64>>();
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<Int8>>();
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<Int16>>();
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<Int32>>();
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<Int64>>();
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<Float32>>();
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantiles<Float64>v;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return std::make_shared<AggregateFunctionQuantiles<DataTypeDate::FieldType, false>>();
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return std::make_shared<AggregateFunctionQuantiles<DataTypeDateTime::FieldType, false>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -28,18 +28,18 @@ AggregateFunctionPtr createAggregateFunctionQuantileDeterministic(const std::str
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantileDeterministic<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantileDeterministic<DataTypeDateTime::FieldType, false>;
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<UInt8>>();
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<UInt16>>();
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<UInt32>>();
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<UInt64>>();
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Int8>>();
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Int16>>();
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Int32>>();
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Int64>>();
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Float32>>();
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Float64>>();
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<DataTypeDate::FieldType, false>>();
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<DataTypeDateTime::FieldType, false>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -65,18 +65,18 @@ AggregateFunctionPtr createAggregateFunctionQuantilesDeterministic(const std::st
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt8>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt16>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt32>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<UInt64>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int8>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int16>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int32>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Int64>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Float32>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<Float64>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantilesDeterministic<DataTypeDateTime::FieldType, false>;
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<UInt8>>();
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<UInt16>>();
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<UInt32>>();
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<UInt64>>();
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Int8>>();
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Int16>>();
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Int32>>();
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Int64>>();
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Float32>>();
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Float64>>();
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<DataTypeDate::FieldType, false>>();
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<DataTypeDateTime::FieldType, false>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -16,18 +16,18 @@ AggregateFunctionPtr createAggregateFunctionQuantileTDigest(const std::string &
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new FunctionTemplate<UInt8, true>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new FunctionTemplate<UInt16, true>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new FunctionTemplate<UInt32, true>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new FunctionTemplate<UInt64, true>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new FunctionTemplate<Int8, true>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new FunctionTemplate<Int16, true>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new FunctionTemplate<Int32, true>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new FunctionTemplate<Int64, true>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new FunctionTemplate<Float32, true>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new FunctionTemplate<Float64, true>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new FunctionTemplate<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new FunctionTemplate<DataTypeDateTime::FieldType, false>;
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<FunctionTemplate<UInt8, true>>();
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<FunctionTemplate<UInt16, true>>();
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<FunctionTemplate<UInt32, true>>();
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<FunctionTemplate<UInt64, true>>();
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<FunctionTemplate<Int8, true>>();
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<FunctionTemplate<Int16, true>>();
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<FunctionTemplate<Int32, true>>();
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<FunctionTemplate<Int64, true>>();
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<FunctionTemplate<Float32, true>>();
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<FunctionTemplate<Float64, true>>();
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return std::make_shared<FunctionTemplate<DataTypeDate::FieldType, false>>();
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return std::make_shared<FunctionTemplate<DataTypeDateTime::FieldType, false>>();
else
throw Exception("Illegal type " + argument_type.getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -37,16 +37,16 @@ AggregateFunctionPtr createAggregateFunctionQuantileTDigestWeightedImpl(const st
{
const IDataType & argument_type = *argument_types[1];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new FunctionTemplate<T, UInt8, returns_float>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new FunctionTemplate<T, UInt16, returns_float>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new FunctionTemplate<T, UInt32, returns_float>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new FunctionTemplate<T, UInt64, returns_float>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new FunctionTemplate<T, Int8, returns_float>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new FunctionTemplate<T, Int16, returns_float>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new FunctionTemplate<T, Int32, returns_float>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new FunctionTemplate<T, Int64, returns_float>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new FunctionTemplate<T, Float32, returns_float>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new FunctionTemplate<T, Float64, returns_float>;
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, UInt8, returns_float>>();
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, UInt16, returns_float>>();
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, UInt32, returns_float>>();
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, UInt64, returns_float>>();
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, Int8, returns_float>>();
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, Int16, returns_float>>();
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, Int32, returns_float>>();
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, Int64, returns_float>>();
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, Float32, returns_float>>();
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<FunctionTemplate<T, Float64, returns_float>>();
else
throw Exception("Illegal type " + argument_type.getName() + " of second argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -13,7 +13,7 @@ AggregateFunctionPtr createAggregateFunctionQuantileTiming(const std::string & n
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionQuantileTiming>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionQuantileTiming>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -26,7 +26,7 @@ AggregateFunctionPtr createAggregateFunctionQuantilesTiming(const std::string &
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionQuantilesTiming>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionQuantilesTiming>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -39,7 +39,7 @@ AggregateFunctionPtr createAggregateFunctionQuantileTimingWeighted(const std::st
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantileTimingWeighted>(*argument_types[0], *argument_types[1]);
AggregateFunctionPtr res(createWithTwoNumericTypes<AggregateFunctionQuantileTimingWeighted>(*argument_types[0], *argument_types[1]));
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
@ -53,7 +53,7 @@ AggregateFunctionPtr createAggregateFunctionQuantilesTimingWeighted(const std::s
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantilesTimingWeighted>(*argument_types[0], *argument_types[1]);
AggregateFunctionPtr res(createWithTwoNumericTypes<AggregateFunctionQuantilesTimingWeighted>(*argument_types[0], *argument_types[1]));
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()

View File

@ -12,7 +12,7 @@ AggregateFunctionPtr createAggregateFunctionSequenceCount(const std::string & na
if (!AggregateFunctionSequenceCount::sufficientArgs(argument_types.size()))
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new AggregateFunctionSequenceCount;
return std::make_shared<AggregateFunctionSequenceCount>();
}
AggregateFunctionPtr createAggregateFunctionSequenceMatch(const std::string & name, const DataTypes & argument_types)
@ -20,7 +20,7 @@ AggregateFunctionPtr createAggregateFunctionSequenceMatch(const std::string & na
if (!AggregateFunctionSequenceMatch::sufficientArgs(argument_types.size()))
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return new AggregateFunctionSequenceMatch;
return std::make_shared<AggregateFunctionSequenceMatch>();
}
}

View File

@ -13,7 +13,7 @@ AggregateFunctionPtr createAggregateFunctionVarPop(const std::string & name, con
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionVarPop>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionVarPop>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -26,7 +26,7 @@ AggregateFunctionPtr createAggregateFunctionVarSamp(const std::string & name, co
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionVarSamp>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionVarSamp>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -39,7 +39,7 @@ AggregateFunctionPtr createAggregateFunctionStdDevPop(const std::string & name,
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionStdDevPop>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionStdDevPop>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -52,7 +52,7 @@ AggregateFunctionPtr createAggregateFunctionStdDevSamp(const std::string & name,
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionStdDevSamp>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionStdDevSamp>(*argument_types[0]));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -65,7 +65,7 @@ AggregateFunctionPtr createAggregateFunctionCovarPop(const std::string & name, c
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCovarPop>(*argument_types[0], *argument_types[1]);
AggregateFunctionPtr res(createWithTwoNumericTypes<AggregateFunctionCovarPop>(*argument_types[0], *argument_types[1]));
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -78,7 +78,7 @@ AggregateFunctionPtr createAggregateFunctionCovarSamp(const std::string & name,
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCovarSamp>(*argument_types[0], *argument_types[1]);
AggregateFunctionPtr res(createWithTwoNumericTypes<AggregateFunctionCovarSamp>(*argument_types[0], *argument_types[1]));
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -92,7 +92,7 @@ AggregateFunctionPtr createAggregateFunctionCorr(const std::string & name, const
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionCorr>(*argument_types[0], *argument_types[1]);
AggregateFunctionPtr res(createWithTwoNumericTypes<AggregateFunctionCorr>(*argument_types[0], *argument_types[1]));
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -19,18 +19,18 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
{
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, Data>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionUniq, Data>(*argument_types[0]));
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType, Data>;
return std::make_shared<AggregateFunctionUniq<DataTypeDate::FieldType, Data>>();
else if (typeid_cast<const DataTypeDateTime *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, Data>;
return std::make_shared<AggregateFunctionUniq<DataTypeDateTime::FieldType, Data>>();
else if (typeid_cast<const DataTypeString *>(&argument_type) || typeid_cast<const DataTypeFixedString *>(&argument_type))
return new AggregateFunctionUniq<String, Data>;
return std::make_shared<AggregateFunctionUniq<String, Data>>();
else if (typeid_cast<const DataTypeTuple *>(&argument_type))
return new AggregateFunctionUniqVariadic<DataForVariadic, true>;
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, true>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -43,7 +43,7 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
throw Exception("Tuple argument of function " + name + " must be the only argument",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return new AggregateFunctionUniqVariadic<DataForVariadic, false>;
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, false>>();
}
else
throw Exception("Incorrect number of arguments for aggregate function " + name,
@ -57,18 +57,18 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
{
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, Data>(*argument_types[0]);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionUniq, Data>(*argument_types[0]));
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDate::FieldType, Data<DataTypeDate::FieldType>>;
return std::make_shared<AggregateFunctionUniq<DataTypeDate::FieldType, Data<DataTypeDate::FieldType>>>();
else if (typeid_cast<const DataTypeDateTime *>(&argument_type))
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, Data<DataTypeDateTime::FieldType>>;
return std::make_shared<AggregateFunctionUniq<DataTypeDateTime::FieldType, Data<DataTypeDateTime::FieldType>>>();
else if (typeid_cast<const DataTypeString *>(&argument_type) || typeid_cast<const DataTypeFixedString *>(&argument_type))
return new AggregateFunctionUniq<String, Data<String>>;
return std::make_shared<AggregateFunctionUniq<String, Data<String>>>();
else if (typeid_cast<const DataTypeTuple *>(&argument_type))
return new AggregateFunctionUniqVariadic<DataForVariadic, true>;
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, true>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -81,7 +81,7 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
throw Exception("Tuple argument of function " + name + " must be the only argument",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return new AggregateFunctionUniqVariadic<DataForVariadic, false>;
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, false>>();
}
else
throw Exception("Incorrect number of arguments for aggregate function " + name,

View File

@ -7,7 +7,7 @@ namespace DB
ColumnPtr ColumnAggregateFunction::convertToValues() const
{
const IAggregateFunction * function = holder->func;
const IAggregateFunction * function = holder->func.get();
ColumnPtr res = function->getReturnType()->createColumn();
/** Если агрегатная функция возвращает нефинализированное состояние,

View File

@ -70,7 +70,7 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va
ASTSelectQuery & select = typeid_cast<ASTSelectQuery &>(*ast);
ASTExpressionList & node = typeid_cast<ASTExpressionList &>(*select.select_expression_list);
ASTs & asts = node.children;
ASTLiteral * cur = new ASTLiteral(StringRange(), value);
auto cur = std::make_shared<ASTLiteral>(StringRange(), value);
cur->alias = column_name;
ASTPtr column_value = cur;
bool is_replaced = false;
@ -124,10 +124,10 @@ static ASTPtr buildWhereExpression(const ASTs & functions)
{
if (functions.size() == 0) return nullptr;
if (functions.size() == 1) return functions[0];
ASTPtr new_query = new ASTFunction();
ASTPtr new_query = std::make_shared<ASTFunction>();
ASTFunction & new_function = typeid_cast<ASTFunction & >(*new_query);
new_function.name = "and";
new_function.arguments = new ASTExpressionList();
new_function.arguments = std::make_shared<ASTExpressionList>();
new_function.arguments->children = functions;
new_function.children.push_back(new_function.arguments);
return new_query;

View File

@ -357,7 +357,7 @@ int main(int argc, char ** argv)
for (size_t arity = 2; arity <= columns; ++arity)
{
FunctionPtr function = new FunctionAnd;
FunctionPtr function = std::make_shared<FunctionAnd>();
function->getReturnType(DataTypes(arity, DataTypePtr(std::make_shared<DataTypeUInt8>())));
ColumnNumbers arguments(arity);

View File

@ -418,43 +418,43 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
* вместо выполнения подзапроса, надо будет просто из неё прочитать.
*/
subquery_or_table_name = new ASTIdentifier(StringRange(), external_table_name, ASTIdentifier::Table);
subquery_or_table_name = std::make_shared<ASTIdentifier>(StringRange(), external_table_name, ASTIdentifier::Table);
}
else if (settings.global_subqueries_method == GlobalSubqueriesMethod::PULL)
{
String host_port = getFQDNOrHostName() + ":" + toString(context.getTCPPort());
String database = "_query_" + context.getCurrentQueryId();
auto subquery = new ASTSubquery;
auto subquery = std::make_shared<ASTSubquery>();
subquery_or_table_name = subquery;
auto select = new ASTSelectQuery;
auto select = std::make_shared<ASTSelectQuery>();
subquery->children.push_back(select);
auto exp_list = new ASTExpressionList;
auto exp_list = std::make_shared<ASTExpressionList>();
select->select_expression_list = exp_list;
select->children.push_back(select->select_expression_list);
Names column_names = external_storage->getColumnNamesList();
for (const auto & name : column_names)
exp_list->children.push_back(new ASTIdentifier({}, name));
exp_list->children.push_back(std::make_shared<ASTIdentifier>({}, name));
auto table_func = new ASTFunction;
auto table_func = std::make_shared<ASTFunction>();
select->table = table_func;
select->children.push_back(select->table);
table_func->name = "remote";
auto args = new ASTExpressionList;
auto args = std::make_shared<ASTExpressionList>();
table_func->arguments = args;
table_func->children.push_back(table_func->arguments);
auto address_lit = new ASTLiteral({}, host_port);
auto address_lit = std::make_shared<ASTLiteral>({}, host_port);
args->children.push_back(address_lit);
auto database_lit = new ASTLiteral({}, database);
auto database_lit = std::make_shared<ASTLiteral>({}, database);
args->children.push_back(database_lit);
auto table_lit = new ASTLiteral({}, external_table_name);
auto table_lit = std::make_shared<ASTLiteral>({}, external_table_name);
args->children.push_back(table_lit);
}
else
@ -572,8 +572,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(
NamesAndTypesList::const_iterator it = findColumn(function_string);
if (columns.end() != it)
{
ASTIdentifier * ast_id = new ASTIdentifier(func_node->range, function_string);
ast = ast_id;
ast = std::make_shared<ASTIdentifier>(func_node->range, function_string);
current_asts.insert(ast);
replaced = true;
}
@ -626,7 +625,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(
{
ASTs all_columns;
for (const auto & column_name_type : columns)
all_columns.emplace_back(new ASTIdentifier(asterisk->range, column_name_type.name));
all_columns.emplace_back(std::make_shared<ASTIdentifier>(asterisk->range, column_name_type.name));
asts.erase(asts.begin() + i);
asts.insert(asts.begin() + i, all_columns.begin(), all_columns.end());
@ -759,17 +758,17 @@ void ExpressionAnalyzer::executeScalarSubqueries()
static ASTPtr addTypeConversion(ASTLiteral * ast_, const String & type_name)
{
auto ast = std::unique_ptr<ASTLiteral>(ast_);
ASTFunction * func = new ASTFunction(ast->range);
auto func = std::make_shared<ASTFunction>(ast->range);
ASTPtr res = func;
func->alias = ast->alias;
ast->alias.clear();
func->kind = ASTFunction::FUNCTION;
func->name = "CAST";
ASTExpressionList * exp_list = new ASTExpressionList(ast->range);
auto exp_list = std::make_shared<ASTExpressionList>(ast->range);
func->arguments = exp_list;
func->children.push_back(func->arguments);
exp_list->children.emplace_back(ast.release());
exp_list->children.emplace_back(new ASTLiteral{{}, type_name});
exp_list->children.emplace_back(std::make_shared<ASTLiteral>({}, type_name));
return res;
}
@ -826,18 +825,18 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
size_t columns = block.columns();
if (columns == 1)
{
ASTLiteral * lit = new ASTLiteral(ast->range, (*block.getByPosition(0).column)[0]);
auto lit = std::make_shared<ASTLiteral>(ast->range, (*block.getByPosition(0).column)[0]);
lit->alias = subquery->alias;
ast = addTypeConversion(lit, block.getByPosition(0).type->getName());
}
else
{
ASTFunction * tuple = new ASTFunction(ast->range);
auto tuple = std::make_shared<ASTFunction>(ast->range);
tuple->alias = subquery->alias;
ast = tuple;
tuple->kind = ASTFunction::FUNCTION;
tuple->name = "tuple";
ASTExpressionList * exp_list = new ASTExpressionList(ast->range);
auto exp_list = std::make_shared<ASTExpressionList>(ast->range);
tuple->arguments = exp_list;
tuple->children.push_back(tuple->arguments);
@ -845,7 +844,7 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
for (size_t i = 0; i < columns; ++i)
{
exp_list->children[i] = addTypeConversion(
new ASTLiteral(ast->range, (*block.getByPosition(i).column)[0]),
std::make_shared<ASTLiteral>(ast->range, (*block.getByPosition(i).column)[0]),
block.getByPosition(i).type->getName());
}
}
@ -975,8 +974,8 @@ void ExpressionAnalyzer::optimizeGroupBy()
unused_column_name = toString(unused_column);
}
select_query->group_expression_list = new ASTExpressionList;
select_query->group_expression_list->children.push_back(new ASTLiteral(StringRange(), UInt64(unused_column)));
select_query->group_expression_list = std::make_shared<ASTExpressionList>();
select_query->group_expression_list->children.emplace_back(std::make_shared<ASTLiteral>(StringRange(), UInt64(unused_column)));
}
}
@ -1076,10 +1075,10 @@ static std::shared_ptr<InterpreterSelectQuery> interpretSubquery(
if (table)
{
/// create ASTSelectQuery for "SELECT * FROM table" as if written by hand
const auto select_query = new ASTSelectQuery;
const auto select_query = std::make_shared<ASTSelectQuery>();
query = select_query;
const auto select_expression_list = new ASTExpressionList;
const auto select_expression_list = std::make_shared<ASTExpressionList>();
select_query->select_expression_list = select_expression_list;
select_query->children.emplace_back(select_query->select_expression_list);
@ -1090,9 +1089,8 @@ static std::shared_ptr<InterpreterSelectQuery> interpretSubquery(
/// manually substitute column names in place of asterisk
for (const auto & column : columns)
select_expression_list->children.emplace_back(new ASTIdentifier{
StringRange{}, column.name
});
select_expression_list->children.emplace_back(std::make_shared<ASTIdentifier>(
StringRange{}, column.name));
select_query->table = subquery_or_table_name;
select_query->children.emplace_back(select_query->table);
@ -1165,7 +1163,7 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
{
/// Получаем поток блоков для подзапроса. Создаём Set и кладём на место подзапроса.
String set_id = arg->getColumnName();
ASTSet * ast_set = new ASTSet(set_id);
auto ast_set = std::make_shared<ASTSet>(set_id);
ASTPtr ast_set_ptr = ast_set;
/// Особый случай - если справа оператора IN указано имя таблицы, при чём, таблица имеет тип Set (заранее подготовленное множество).
@ -1332,17 +1330,16 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl
if (single_value)
{
ASTPtr exp_list = new ASTExpressionList;
ASTPtr exp_list = std::make_shared<ASTExpressionList>();
exp_list->children.push_back(elements_ast);
elements_ast = exp_list;
}
ASTSet * ast_set = new ASTSet(arg->getColumnName());
ASTPtr ast_set_ptr = ast_set;
auto ast_set = std::make_shared<ASTSet>(arg->getColumnName());
ast_set->set = std::make_shared<Set>(settings.limits);
ast_set->is_explicit = true;
ast_set->set->createFromAST(set_element_types, elements_ast, context, create_ordered_set);
arg = ast_set_ptr;
arg = ast_set;
}

View File

@ -72,7 +72,7 @@ void InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
if (!create.storage)
{
database_engine_name = "Ordinary"; /// Движок баз данных по-умолчанию.
ASTFunction * func = new ASTFunction();
auto func = std::make_shared<ASTFunction>();
func->name = database_engine_name;
create.storage = func;
}
@ -161,7 +161,7 @@ static ColumnsAndDefaults parseColumns(
/** all default_expressions as a single expression list,
* mixed with conversion-columns for each explicitly specified type */
ASTPtr default_expr_list{new ASTExpressionList};
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
default_expr_list->children.reserve(column_list_ast.children.size());
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
@ -195,8 +195,8 @@ static ColumnsAndDefaults parseColumns(
const auto data_type_ptr = columns.back().type.get();
default_expr_list->children.emplace_back(setAlias(
makeASTFunction("CAST", ASTPtr{new ASTIdentifier{{}, tmp_column_name}},
ASTPtr{new ASTLiteral{{}, data_type_ptr->getName()}}), final_column_name));
makeASTFunction("CAST", std::make_shared<ASTIdentifier>({}, tmp_column_name),
std::make_shared<ASTLiteral>({}, data_type_ptr->getName()), final_column_name));
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), tmp_column_name));
}
else
@ -229,7 +229,7 @@ static ColumnsAndDefaults parseColumns(
if (explicit_type->getName() != deduced_type->getName())
{
col_decl_ptr->default_expression = makeASTFunction("CAST", col_decl_ptr->default_expression,
new ASTLiteral{{}, explicit_type->getName()});
std::make_shared<ASTLiteral>({}, explicit_type->getName()));
col_decl_ptr->children.clear();
col_decl_ptr->children.push_back(col_decl_ptr->type);
@ -277,12 +277,11 @@ static NamesAndTypesList removeAndReturnColumns(
ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
{
ASTPtr columns_list_ptr{new ASTExpressionList};
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*columns_list_ptr);
auto columns_list = std::make_shared<ASTExpressionList>;
for (const auto & column : columns)
{
const auto column_declaration = new ASTColumnDeclaration;
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
ASTPtr column_declaration_ptr{column_declaration};
column_declaration->name = column.name;
@ -294,10 +293,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
ParserIdentifierWithOptionalParameters storage_p;
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type->query_string = type_name;
columns_list.children.push_back(column_declaration_ptr);
columns_list->children.push_back(column_declaration_ptr);
}
return columns_list_ptr;
return columns_list;
}
ASTPtr InterpreterCreateQuery::formatColumns(NamesAndTypesList columns,
@ -308,12 +307,11 @@ ASTPtr InterpreterCreateQuery::formatColumns(NamesAndTypesList columns,
columns.insert(std::end(columns), std::begin(materialized_columns), std::end(materialized_columns));
columns.insert(std::end(columns), std::begin(alias_columns), std::end(alias_columns));
ASTPtr columns_list_ptr{new ASTExpressionList};
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*columns_list_ptr);
auto columns_list = std::make_shared<ASTExpressionList>();
for (const auto & column : columns)
{
const auto column_declaration = new ASTColumnDeclaration;
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
ASTPtr column_declaration_ptr{column_declaration};
column_declaration->name = column.name;
@ -333,10 +331,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(NamesAndTypesList columns,
column_declaration->default_expression = it->second.expression->clone();
}
columns_list.children.push_back(column_declaration_ptr);
columns_list->children.push_back(column_declaration_ptr);
}
return columns_list_ptr;
return columns_list;
}
@ -409,7 +407,7 @@ String InterpreterCreateQuery::setEngine(
auto set_engine = [&](const char * engine)
{
storage_name = engine;
ASTFunction * func = new ASTFunction();
auto func = std::make_shared<ASTFunction>();
func->name = engine;
create.storage = func;
};

View File

@ -680,7 +680,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
if (alias_columns_required)
{
/// Составим выражение для возврата всех запрошенных столбцов, с вычислением требуемых ALIAS столбцов.
ASTPtr required_columns_expr_list{new ASTExpressionList};
auto required_columns_expr_list = std::make_shared<ASTExpressionList>();
for (const auto & column : required_columns)
{
@ -688,7 +688,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
if (default_it != std::end(storage->column_defaults) && default_it->second.type == ColumnDefaultType::Alias)
required_columns_expr_list->children.emplace_back(setAlias(default_it->second.expression->clone(), column));
else
required_columns_expr_list->children.emplace_back(new ASTIdentifier{{}, column});
required_columns_expr_list->children.emplace_back(std::make_shared<ASTIdentifier>({}, column));
}
alias_actions = ExpressionAnalyzer{required_columns_expr_list, context, storage, table_column_names}.getActions(true);

View File

@ -214,8 +214,6 @@ bool LogicalExpressionsOptimizer::mayOptimizeDisjunctiveEqualityChain(const Disj
void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain & chain)
{
using ASTFunctionPtr = Poco::SharedPtr<ASTFunction>;
const auto & or_with_expression = chain.first;
const auto & equalities = chain.second;
const auto & equality_functions = equalities.functions;
@ -223,7 +221,7 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
/// 1. Создать новое выражение IN на основе информации из OR-цепочки.
/// Построить список литералов x1, ..., xN из цепочки expr = x1 OR ... OR expr = xN
ASTPtr value_list = new ASTExpressionList;
ASTPtr value_list = std::make_shared<ASTExpressionList>();
for (const auto function : equality_functions)
{
const auto & operands = getFunctionOperands(function);
@ -247,17 +245,17 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
equals_expr_lhs = operands[0];
}
ASTFunctionPtr tuple_function = new ASTFunction;
auto tuple_function = std::make_shared<ASTFunction>();
tuple_function->name = "tuple";
tuple_function->arguments = value_list;
tuple_function->children.push_back(tuple_function->arguments);
ASTPtr expression_list = new ASTExpressionList;
ASTPtr expression_list = std::make_shared<ASTExpressionList>();
expression_list->children.push_back(equals_expr_lhs);
expression_list->children.push_back(tuple_function);
/// Построить выражение expr IN (x1, ..., xN)
ASTFunctionPtr in_function = new ASTFunction;
auto in_function = std::make_shared<ASTFunction>();
in_function->name = "in";
in_function->arguments = expression_list;
in_function->children.push_back(in_function->arguments);

View File

@ -49,8 +49,7 @@ QueryLog::QueryLog(Context & context_, const String & database_name_, const Stri
while (context.isTableExist(database_name, table_name + "_" + toString(suffix)))
++suffix;
ASTRenameQuery * rename = new ASTRenameQuery;
ASTPtr holder = rename;
auto rename = std::make_shared<ASTRenameQuery>();
ASTRenameQuery::Table from;
from.database = database_name;
@ -69,7 +68,7 @@ QueryLog::QueryLog(Context & context_, const String & database_name_, const Stri
LOG_DEBUG(log, "Existing table " << description << " for query log has obsolete or different structure."
" Renaming it to " << backQuoteIfNeed(to.table));
InterpreterRenameQuery(holder, context).execute();
InterpreterRenameQuery(rename, context).execute();
/// Нужная таблица будет создана.
table = nullptr;
@ -83,8 +82,7 @@ QueryLog::QueryLog(Context & context_, const String & database_name_, const Stri
/// Создаём таблицу.
LOG_DEBUG(log, "Creating new table " << description << " for query log.");
ASTCreateQuery * create = new ASTCreateQuery;
ASTPtr holder = create;
auto create = std::make_shared<ASTCreateQuery>();
create->database = database_name;
create->table = table_name;
@ -97,7 +95,7 @@ QueryLog::QueryLog(Context & context_, const String & database_name_, const Stri
create->storage = parseQuery(engine_parser, engine.data(), engine.data() + engine.size(), "ENGINE to create table for query log");
InterpreterCreateQuery(holder, context).execute();
InterpreterCreateQuery(create, context).execute();
table = context.getTable(database_name, table_name);
}

View File

@ -16,7 +16,7 @@ namespace
{
/// for string literal return its value
if (const auto literal = typeid_cast<const ASTLiteral *>(expr.get()))
return new ASTIdentifier{{}, safeGet<const String &>(literal->value)};
return std::make_shared<ASTIdentifier>({}, safeGet<const String &>(literal->value));
/// otherwise evaluate the expression
Block block{};
@ -33,7 +33,7 @@ namespace
if (!typeid_cast<const DataTypeString *>(column_name_type.type.get()))
throw Exception{"Expression must evaluate to a String"};
return new ASTIdentifier{{}, column_name_type.column->getDataAt(0).toString()};
return std::make_shared<ASTIdentifier>({}, column_name_type.column->getDataAt(0).toString());
}
}

View File

@ -56,7 +56,7 @@ String ASTAlterQuery::getID() const
ASTPtr ASTAlterQuery::clone() const
{
ASTAlterQuery * res = new ASTAlterQuery(*this);
auto res = std::make_shared<ASTAlterQuery>(*this);
for (ParameterContainer::size_type i = 0; i < parameters.size(); ++i)
parameters[i].clone(res->parameters[i]);
return res;

View File

@ -50,15 +50,13 @@ String ASTFunction::getID() const
ASTPtr ASTFunction::clone() const
{
ASTFunction * res = new ASTFunction(*this);
ASTPtr ptr{res};
auto res = std::make_shared<ASTFunction>(*this);
res->children.clear();
if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); }
if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); }
return ptr;
return res;
}
void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const

View File

@ -66,7 +66,7 @@ void ASTSelectQuery::renameColumns(const ASTSelectQuery & source)
void ASTSelectQuery::rewriteSelectExpressionList(const Names & column_names)
{
ASTPtr result = new ASTExpressionList;
ASTPtr result = std::make_shared<ASTExpressionList>();
ASTs asts = select_expression_list->children;
/// Создать отображение.
@ -175,9 +175,7 @@ ASTPtr ASTSelectQuery::cloneFirstSelect() const
ASTPtr ASTSelectQuery::cloneImpl(bool traverse_union_all) const
{
ASTSelectQuery * res = new ASTSelectQuery(*this);
ASTPtr ptr{res};
auto res = std::make_shared<ASTSelectQuery>(*this);
res->children.clear();
#define CLONE(member) if (member) { res->member = member->clone(); res->children.push_back(res->member); }
@ -222,7 +220,7 @@ ASTPtr ASTSelectQuery::cloneImpl(bool traverse_union_all) const
else
res->next_union_all = nullptr;
return ptr;
return res;
}
const IAST * ASTSelectQuery::getFormat() const

View File

@ -51,7 +51,7 @@ bool ParserArray::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_
if (!close.ignore(pos, end, max_parsed_pos, expected))
return false;
ASTFunction * function_node = new ASTFunction(StringRange(begin, pos));
auto function_node = std::make_shared<ASTFunction>(StringRange(begin, pos));
function_node->name = "array";
function_node->arguments = contents_node;
function_node->children.push_back(contents_node);
@ -95,7 +95,7 @@ bool ParserParenthesisExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, P
}
else
{
ASTFunction * function_node = new ASTFunction(StringRange(begin, pos));
auto function_node = std::make_shared<ASTFunction>(StringRange(begin, pos));
function_node->name = "tuple";
function_node->arguments = contents_node;
function_node->children.push_back(contents_node);
@ -125,7 +125,7 @@ bool ParserSubquery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pars
if (!close.ignore(pos, end, max_parsed_pos, expected))
return false;
node = new ASTSubquery(StringRange(begin, pos));
node = std::make_shared<ASTSubquery>(StringRange(begin, pos));
typeid_cast<ASTSubquery &>(*node).children.push_back(select_node);
return true;
}
@ -146,7 +146,7 @@ bool ParserIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pa
return false;
pos += buf.count();
node = new ASTIdentifier(StringRange(begin, pos), s);
node = std::make_shared<ASTIdentifier>(StringRange(begin, pos), s);
return true;
}
else
@ -160,7 +160,7 @@ bool ParserIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pa
if (pos != begin)
{
node = new ASTIdentifier(StringRange(begin, pos), String(begin, pos - begin));
node = std::make_shared<ASTIdentifier>(StringRange(begin, pos), String(begin, pos - begin));
return true;
}
else
@ -187,7 +187,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos
name += static_cast<const ASTIdentifier &>(*child.get()).name;
}
node = new ASTIdentifier(StringRange(begin, pos), name);
node = std::make_shared<ASTIdentifier>(StringRange(begin, pos), name);
/// В children запомним идентификаторы-составляющие, если их больше одного.
if (list.children.size() > 1)
@ -266,7 +266,7 @@ bool ParserFunction::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pars
return false;
}
ASTFunction * function_node = new ASTFunction(StringRange(begin, pos));
auto function_node = std::make_shared<ASTFunction>(StringRange(begin, pos));
ASTPtr node_holder{function_node};
function_node->name = typeid_cast<ASTIdentifier &>(*identifier).name;
@ -335,10 +335,10 @@ bool ParserCastExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
return false;
}
expr_list_args = new ASTExpressionList{StringRange{contents_begin, end}};
expr_list_args = std::make_shared<ASTExpressionList>(StringRange{contents_begin, end});
first_argument->setAlias({});
expr_list_args->children.push_back(first_argument);
expr_list_args->children.emplace_back(new ASTLiteral{{}, type});
expr_list_args->children.emplace_back(std::make_shared<ASTLiteral>({}, type));
}
else
{
@ -366,7 +366,7 @@ bool ParserCastExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
return false;
}
expr_list_args = new ASTExpressionList{StringRange{contents_begin, end}};
expr_list_args = std::make_shared<ASTExpressionList>(StringRange{contents_begin, end});
expr_list_args->children.push_back(first_argument);
expr_list_args->children.push_back(type_as_literal);
}
@ -379,7 +379,7 @@ bool ParserCastExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
return false;
}
const auto function_node = new ASTFunction(StringRange(begin, pos));
const auto function_node = std::make_shared<ASTFunction>(StringRange(begin, pos));
ASTPtr node_holder{function_node};
function_node->name = name;
@ -397,7 +397,7 @@ bool ParserNull::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
ParserString nested_parser("NULL", true);
if (nested_parser.parse(pos, end, node, max_parsed_pos, expected))
{
node = new ASTLiteral(StringRange(StringRange(begin, pos)), Null());
node = std::make_shared<ASTLiteral>(StringRange(StringRange(begin, pos)), Null());
return true;
}
else
@ -451,7 +451,7 @@ bool ParserNumber::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed
}
pos += pos_double - buf;
node = new ASTLiteral(StringRange(begin, pos), res);
node = std::make_shared<ASTLiteral>(StringRange(begin, pos), res);
return true;
}
@ -474,7 +474,7 @@ bool ParserUnsignedInteger::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
res = x;
pos += in.count();
node = new ASTLiteral(StringRange(begin, pos), res);
node = std::make_shared<ASTLiteral>(StringRange(begin, pos), res);
return true;
}
@ -503,7 +503,7 @@ bool ParserStringLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max
}
pos += in.count();
node = new ASTLiteral(StringRange(begin, pos), s);
node = std::make_shared<ASTLiteral>(StringRange(begin, pos), s);
return true;
}
@ -533,7 +533,7 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
if (*pos == ']')
{
++pos;
node = new ASTLiteral(StringRange(begin, pos), arr);
node = std::make_shared<ASTLiteral>(StringRange(begin, pos), arr);
return true;
}
else if (*pos == ',')
@ -684,7 +684,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
if (asterisk_p.parse(pos, end, node, max_parsed_pos, expected))
{
node = new ASTAsterisk(StringRange(begin, pos));
node = std::make_shared<ASTAsterisk>(StringRange(begin, pos));
return true;
}
@ -790,7 +790,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
collator = std::make_shared<Collator>(locale);
}
node = new ASTOrderByElement(StringRange(begin, pos), direction, collator);
node = std::make_shared<ASTOrderByElement>(StringRange(begin, pos), direction, collator);
node->children.push_back(expr_elem);
return true;
}
@ -802,7 +802,7 @@ bool ParserWeightedZooKeeperPath::parseImpl(Pos & pos, Pos end, ASTPtr & node, P
ParserUnsignedInteger weight_p;
ParserWhiteSpaceOrComments ws;
auto weighted_zookeeper_path = new ASTWeightedZooKeeperPath;
auto weighted_zookeeper_path = std::make_shared<ASTWeightedZooKeeperPath>();
node = weighted_zookeeper_path;
ws.ignore(pos, end);

View File

@ -78,7 +78,7 @@ bool ParserList::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
bool first = true;
ParserWhiteSpaceOrComments ws;
ASTExpressionList * list = new ASTExpressionList;
auto list = std::make_shared<ASTExpressionList>();
node = list;
while (1)
@ -151,30 +151,26 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP
ws.ignore(pos, end);
/// функция, соответствующая оператору
ASTFunction * p_function = new ASTFunction;
ASTFunction & function = *p_function;
ASTPtr function_node = p_function;
auto function = std::make_shared<ASTFunction>();
/// аргументы функции
ASTExpressionList * p_exp_list = new ASTExpressionList;
ASTExpressionList & exp_list = *p_exp_list;
ASTPtr exp_list_node = p_exp_list;
auto exp_list = std::make_shared<ASTExpressionList>();
ASTPtr elem;
if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, end, elem, max_parsed_pos, expected))
return false;
/// первым аргументом функции будет предыдущий элемент, вторым - следующий
function.range.first = begin;
function.range.second = pos;
function.name = it[1];
function.arguments = exp_list_node;
function.children.push_back(exp_list_node);
function->range.first = begin;
function->range.second = pos;
function->name = it[1];
function->arguments = exp_list;
function->children.push_back(exp_list);
exp_list.children.push_back(node);
exp_list.children.push_back(elem);
exp_list.range.first = begin;
exp_list.range.second = pos;
exp_list->children.push_back(node);
exp_list->children.push_back(elem);
exp_list->range.first = begin;
exp_list->range.second = pos;
/** специальное исключение для оператора доступа к элементу массива x[y], который
* содержит инфиксную часть '[' и суффиксную ']' (задаётся в виде '[')
@ -188,7 +184,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP
return false;
}
node = function_node;
node = function;
}
}
@ -354,28 +350,24 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & nod
return false;
/// функция, соответствующая оператору
ASTFunction * p_function = new ASTFunction;
ASTFunction & function = *p_function;
ASTPtr function_node = p_function;
auto function = std::make_shared<ASTFunction>();
/// аргументы функции
ASTExpressionList * p_exp_list = new ASTExpressionList;
ASTExpressionList & exp_list = *p_exp_list;
ASTPtr exp_list_node = p_exp_list;
auto exp_list = std::make_shared<ASTExpressionList>();
function.range.first = begin;
function.range.second = pos;
function.name = "if";
function.arguments = exp_list_node;
function.children.push_back(exp_list_node);
function->range.first = begin;
function->range.second = pos;
function->name = "if";
function->arguments = exp_list;
function->children.push_back(exp_list);
exp_list.children.push_back(elem_cond);
exp_list.children.push_back(elem_then);
exp_list.children.push_back(elem_else);
exp_list.range.first = begin;
exp_list.range.second = pos;
exp_list->children.push_back(elem_cond);
exp_list->children.push_back(elem_then);
exp_list->children.push_back(elem_else);
exp_list->range.first = begin;
exp_list->range.second = pos;
node = function_node;
node = function;
}
return true;
@ -424,15 +416,15 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
/// lambda(tuple(inner_arguments), expression)
ASTFunction * lambda = new ASTFunction;
auto lambda = std::make_shared<ASTFunction>();
node = lambda;
lambda->name = "lambda";
ASTExpressionList * outer_arguments = new ASTExpressionList;
auto outer_arguments = std::make_shared<ASTExpressionList>();
lambda->arguments = outer_arguments;
lambda->children.push_back(lambda->arguments);
ASTFunction * tuple = new ASTFunction;
auto tuple = std::make_shared<ASTFunction>();
outer_arguments->children.push_back(tuple);
tuple->name = "tuple";
tuple->arguments = inner_arguments;
@ -505,26 +497,22 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr &
else
{
/// функция, соответствующая оператору
ASTFunction * p_function = new ASTFunction;
ASTFunction & function = *p_function;
ASTPtr function_node = p_function;
auto function = std::make_shared<ASTFunction>();
/// аргументы функции
ASTExpressionList * p_exp_list = new ASTExpressionList;
ASTExpressionList & exp_list = *p_exp_list;
ASTPtr exp_list_node = p_exp_list;
auto exp_list = std::make_shared<ASTExpressionList>();
function.range.first = begin;
function.range.second = pos;
function.name = it[1];
function.arguments = exp_list_node;
function.children.push_back(exp_list_node);
function->range.first = begin;
function->range.second = pos;
function->name = it[1];
function->arguments = exp_list;
function->children.push_back(exp_list);
exp_list.children.push_back(elem);
exp_list.range.first = begin;
exp_list.range.second = pos;
exp_list->children.push_back(elem);
exp_list->range.first = begin;
exp_list->range.second = pos;
node = function_node;
node = function;
}
return true;

View File

@ -57,8 +57,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pa
ASTPtr col_after;
ASTPtr col_drop;
ASTAlterQuery * query = new ASTAlterQuery();
ASTPtr query_ptr = query;
auto query = std::make_shared<ASTAlterQuery>();
ws.ignore(pos, end);
if (!s_alter.ignore(pos, end, max_parsed_pos, expected))
@ -373,7 +372,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pa
while (!parsing_finished);
query->range = StringRange(begin, end);
node = query_ptr;
node = query;
return true;
}

View File

@ -36,9 +36,6 @@ bool ParserCase::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
ASTs args;
using ASTFunctionPtr = Poco::SharedPtr<ASTFunction>;
using ASTExpressionListPtr = Poco::SharedPtr<ASTExpressionList>;
auto parse_branches = [&]()
{
bool has_branch = false;
@ -103,8 +100,8 @@ bool ParserCase::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
return false;
/// Hand-craft a transform() function.
ASTExpressionListPtr src_expr_list = new ASTExpressionList{StringRange{begin, pos}};
ASTExpressionListPtr dst_expr_list = new ASTExpressionList{StringRange{begin, pos}};
auto src_expr_list = std::make_shared<ASTExpressionList>(StringRange{begin, pos});
auto dst_expr_list = std::make_shared<ASTExpressionList>(StringRange{begin, pos});
for (size_t i = 0; i < (args.size() - 1); ++i)
{
@ -114,25 +111,25 @@ bool ParserCase::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
dst_expr_list->children.push_back(args[i]);
}
ASTFunctionPtr src_array_function = new ASTFunction{StringRange{begin, pos}};
auto src_array_function = std::make_shared<ASTFunction>(StringRange{begin, pos});
src_array_function->name = "array";
src_array_function->genus = ASTFunction::Genus::CASE_ARRAY;
src_array_function->arguments = src_expr_list;
src_array_function->children.push_back(src_array_function->arguments);
ASTFunctionPtr dst_array_function = new ASTFunction{StringRange{begin, pos}};
auto dst_array_function = std::make_shared<ASTFunction>(StringRange{begin, pos});
dst_array_function->name = "array";
dst_array_function->genus = ASTFunction::Genus::CASE_ARRAY;
dst_array_function->arguments = dst_expr_list;
dst_array_function->children.push_back(dst_array_function->arguments);
ASTPtr function_args = new ASTExpressionList{StringRange{begin, pos}};
auto function_args = std::make_shared<ASTExpressionList>(StringRange{begin, pos});
function_args->children.push_back(case_expr);
function_args->children.push_back(src_array_function);
function_args->children.push_back(dst_array_function);
function_args->children.emplace_back(args.back());
ASTFunctionPtr function = new ASTFunction{StringRange{begin, pos}};
auto function = std::make_shared<ASTFunction>(StringRange{begin, pos});
function->name = "transform";
function->genus = ASTFunction::Genus::CASE_WITH_EXPR;
function->arguments = function_args;
@ -146,10 +143,10 @@ bool ParserCase::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
return false;
/// Hand-craft a multiIf() function.
ASTPtr function_args = new ASTExpressionList{StringRange{begin, pos}};
auto function_args = std::make_shared<ASTExpressionList>(StringRange{begin, pos});
function_args->children = std::move(args);
ASTFunctionPtr function = new ASTFunction{StringRange{begin, pos}};
auto function = std::make_shared<ASTFunction>(StringRange{begin, pos});
function->name = "multiIf";
function->genus = ASTFunction::Genus::CASE_WITHOUT_EXPR;
function->arguments = function_args;

View File

@ -17,7 +17,7 @@ bool ParserCheckQuery::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr &
ASTPtr table;
ASTPtr database;
Poco::SharedPtr<ASTCheckQuery> query = new ASTCheckQuery(StringRange(pos, end));
auto query = std::make_shared<ASTCheckQuery>(StringRange(pos, end));
ws.ignore(pos, end);

View File

@ -42,11 +42,11 @@ bool ParserNestedTable::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
if (!close.ignore(pos, end))
return false;
ASTFunction * func = new ASTFunction(StringRange(begin, pos));
node = func;
auto func = std::make_shared<ASTFunction>(StringRange(begin, pos));
func->name = typeid_cast<ASTIdentifier &>(*name).name;
func->arguments = columns;
func->children.push_back(columns);
node = func;
return true;
}
@ -79,9 +79,9 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, Pos end, ASTPt
ASTPtr ident;
if (non_parametric.parse(pos, end, ident, max_parsed_pos, expected))
{
ASTFunction * func = new ASTFunction(StringRange(begin, pos));
node = func;
auto func = std::make_shared<ASTFunction>(StringRange(begin, pos));
func->name = typeid_cast<ASTIdentifier &>(*ident).name;
node = func;
return true;
}
@ -93,9 +93,7 @@ bool ParserTypeInCastExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
if (ParserIdentifierWithOptionalParameters::parseImpl(pos, end, node, max_parsed_pos, expected))
{
const auto & id_with_params = typeid_cast<const ASTFunction &>(*node);
node = new ASTIdentifier{id_with_params.range, { id_with_params.range.first, id_with_params.range.second }};
node = std::make_shared<ASTIdentifier>(id_with_params.range, { id_with_params.range.first, id_with_params.range.second });
return true;
}
@ -389,7 +387,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
ws.ignore(pos, end);
ASTCreateQuery * query = new ASTCreateQuery(StringRange(begin, pos));
auto query = std::make_shared<ASTCreateQuery>(StringRange(begin, pos));
node = query;
query->attach = attach;

View File

@ -83,7 +83,7 @@ bool ParserDropQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_par
ws.ignore(pos, end);
ASTDropQuery * query = new ASTDropQuery(StringRange(begin, pos));
auto query = std::make_shared<ASTDropQuery>(StringRange(begin, pos));
node = query;
query->detach = detach;

View File

@ -149,7 +149,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
return false;
}
ASTInsertQuery * query = new ASTInsertQuery(StringRange(begin, data ? data : pos));
auto query = std::make_shared<ASTInsertQuery>(StringRange(begin, data ? data : pos));
node = query;
if (database)

View File

@ -18,7 +18,7 @@ bool ParserJoin::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
{
Pos begin = pos;
ASTJoin * join = new ASTJoin(StringRange(begin, pos));
auto join = std::make_shared<ASTJoin>(StringRange(begin, pos));
node = join;
ParserWhiteSpaceOrComments ws;

View File

@ -69,7 +69,7 @@ bool ParserOptimizeQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max
if (s_final.ignore(pos, end, max_parsed_pos, expected))
final = true;
ASTOptimizeQuery * query = new ASTOptimizeQuery(StringRange(begin, pos));
auto query = std::make_shared<ASTOptimizeQuery>(StringRange(begin, pos));
node = query;
if (database)

View File

@ -82,7 +82,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
return false;
}
ASTRenameQuery * query = new ASTRenameQuery(StringRange(begin, pos));
auto query = std::make_shared<ASTRenameQuery>(StringRange(begin, pos));
node = query;
query->elements = elements;

Some files were not shown because too many files have changed in this diff Show More