Added missing ErrorCodes, part 2

This commit is contained in:
Alexey Milovidov 2020-02-25 21:10:48 +03:00
parent 766ae8261a
commit cb6cd2ffbb
137 changed files with 581 additions and 0 deletions

View File

@ -5,6 +5,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_PACKET_FROM_SERVER;
}
void Suggest::load(const ConnectionParameters & connection_parameters, size_t suggestion_limit)
{

View File

@ -7,6 +7,13 @@
namespace DB
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
extern const int UNFINISHED;
extern const int BAD_ARGUMENTS;
}
{
void ClusterCopier::init()

View File

@ -2,6 +2,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data)
{

View File

@ -4,6 +4,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
struct TaskCluster
{

View File

@ -6,6 +6,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int LOGICAL_ERROR;
}
struct TaskShard;

View File

@ -11,6 +11,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
GenericRoleSet::GenericRoleSet() = default;
GenericRoleSet::GenericRoleSet(const GenericRoleSet & src) = default;
GenericRoleSet & GenericRoleSet::operator =(const GenericRoleSet & src) = default;

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{

View File

@ -10,6 +10,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{

View File

@ -16,6 +16,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
using FuncLinearRegression = AggregateFunctionMLMethod<LinearModelData, NameLinearRegression>;

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{

View File

@ -9,6 +9,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
/** Not an aggregate function, but an adapter of aggregate functions,

View File

@ -15,6 +15,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
/** Aggregate functions that store one of passed values.
* For example: min, max, any, anyLast.

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{

View File

@ -8,6 +8,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{

View File

@ -9,6 +9,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
template <bool rate>

View File

@ -11,6 +11,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{

View File

@ -14,6 +14,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
class Arena;
class ReadBuffer;

View File

@ -11,6 +11,10 @@ using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** A column containing a lambda expression.

View File

@ -8,6 +8,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int LOGICAL_ERROR;
}
namespace
{

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
}
template <typename Derived>
std::vector<IColumn::MutablePtr> IColumn::scatterImpl(ColumnIndex num_columns,

View File

@ -4,6 +4,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
class IColumnUnique : public IColumn
{

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{

View File

@ -18,6 +18,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace ColumnsHashing
{

View File

@ -8,6 +8,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace ColumnsHashing
{

View File

@ -8,6 +8,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace details
{

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER;
}
template <typename T>
static inline String formatQuoted(T x)

View File

@ -15,6 +15,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/** Cursor allows to compare rows in different blocks (and parts).
* Cursor moves inside single block.

View File

@ -8,6 +8,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_TYPE_OF_FIELD;
}
/// Collapses the same rows with the opposite sign roughly like CollapsingSortedBlockInputStream.
/// Outputs the rows in random order (the input streams must still be ordered).

View File

@ -18,6 +18,10 @@ namespace ProfileEvents
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_ENOUGH_SPACE;
}
MergeSortingBlockInputStream::MergeSortingBlockInputStream(
const BlockInputStreamPtr & input, SortDescription & description_,

View File

@ -13,6 +13,10 @@ namespace CurrentMetrics
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/** Scheme of operation:

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
}
SquashingTransform::SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_, bool reserve_memory_)
: min_block_size_rows(min_block_size_rows_)

View File

@ -22,6 +22,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
template <typename... Ts>
static inline bool typeIsEither(const IDataType & type)

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
class Context;
/// Allows loading dictionaries from a file with given format, does not support "random access"

View File

@ -17,6 +17,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
static const UInt64 max_block_size = 8192;

View File

@ -14,6 +14,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
struct IDictionaryBase;
using DictionaryPtr = std::unique_ptr<IDictionaryBase>;

View File

@ -20,6 +20,10 @@ namespace Util
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
class CStringsHolder;
/// Allows loading dictionaries from dynamic libraries (.so)

View File

@ -23,6 +23,10 @@ namespace MongoDB
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/// Allows loading dictionaries from a MongoDB collection
class MongoDBDictionarySource final : public IDictionarySource
{
@ -90,6 +94,10 @@ private:
/*namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
class DictionarySourceFactory;
void registerDictionarySourceMongoDB(DictionarySourceFactory & factory);
}*/

View File

@ -26,6 +26,10 @@ namespace Poco
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
enum class RedisStorageType
{
SIMPLE,

View File

@ -12,6 +12,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_ENOUGH_SPACE;
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_POLICY;
extern const int UNKNOWN_DISK;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
}
DiskSelector::DiskSelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Context & context)
{
Poco::Util::AbstractConfiguration::Keys keys;

View File

@ -10,6 +10,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
class ExecutableFunctionExpression : public IExecutableFunctionImpl
{

View File

@ -14,6 +14,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness.
*/
struct PositionCaseSensitiveASCII

View File

@ -8,6 +8,10 @@
namespace DB::GatherUtils
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// Base classes which selects template function implementation with concrete ArraySource or ArraySink
/// Derived classes should implement selectImpl for ArraySourceSelector and ArraySinkSelector,
/// selectSourceSink for ArraySinkSourceSelector and selectSourcePair for ArraySourcePairSelector

View File

@ -4,6 +4,10 @@
namespace DB::GatherUtils
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
struct ArrayConcat : public ArraySinkSourceSelector<ArrayConcat>
{

View File

@ -18,6 +18,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
/// Cache for functions result if it was executed on low cardinality column.
class ExecutableFunctionLowCardinalityResultCache;

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class URLPathHierarchyImpl
{

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class URLHierarchyImpl
{

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/// We assume that size of the dst buf isn't less than src_size.
static size_t decodeURL(const char * src, size_t src_size, char * dst)

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class ExtractURLParameterNamesImpl
{

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class ExtractURLParametersImpl
{

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** arrayAll(x1,...,xn -> expression, array1,...,arrayn) - is the expression true for all elements of the array.
* An overload of the form f(array) is available, which works in the same way as f(x -> x, array).

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** arrayCount(x1,...,xn -> expression, array1,...,arrayn) - for how many elements of the array the expression is true.
* An overload of the form f(array) is available, which works in the same way as f(x -> x, array).

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** arrayExists(x1,...,xn -> expression, array1,...,arrayn) - is the expression true for at least one array element.
* An overload of the form f(array) is available, which works in the same way as f(x -> x, array).

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
template <bool reverse>
struct ArrayFillImpl

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** arrayFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true.
*/

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
struct ArrayFirstImpl
{

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
struct ArrayFirstIndexImpl
{

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
template <bool reverse>
struct ArraySplitImpl

View File

@ -4,6 +4,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A, typename B>
struct BitAndImpl

View File

@ -5,6 +5,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A>
struct BitNotImpl

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A, typename B>
struct BitOrImpl

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A, typename B>
struct BitRotateLeftImpl

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A, typename B>
struct BitRotateRightImpl

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A, typename B>
struct BitShiftLeftImpl

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A, typename B>
struct BitShiftRightImpl

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A, typename B>
struct BitXorImpl

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A, typename B>
struct DivideFloatingImpl

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
/** Calculates the distance between two geographical locations.
* There are three variants:

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionH3EdgeAngle : public IFunction
{
public:

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionH3GetBaseCell : public IFunction
{
public:

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionH3GetResolution : public IFunction
{
public:

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionH3IndexesAreNeighbors : public IFunction
{
public:

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionH3IsValid : public IFunction
{
public:

View File

@ -14,6 +14,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionH3ToChildren : public IFunction
{
public:

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionH3ToParent : public IFunction
{
public:

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionH3ToString : public IFunction
{
public:

View File

@ -16,6 +16,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionH3KRing : public IFunction
{
public:

View File

@ -11,6 +11,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_DIVISION;
}
/// Optimizations for integer division by a constant.

View File

@ -5,6 +5,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <typename A>
struct IntExp2Impl

View File

@ -12,6 +12,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
/// inspired by https://github.com/cyb70289/utf8/
struct ValidUTF8Impl
{

View File

@ -4,6 +4,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
template <typename ToType, typename Name>
class ExecutableFunctionRandomConstant : public IExecutableFunctionImpl

View File

@ -14,6 +14,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
using namespace GatherUtils;

View File

@ -2,6 +2,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int MEMORY_LIMIT_EXCEEDED;
extern const int LOGICAL_ERROR;
}
PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ /*= DBMS_DEFAULT_BUFFER_SIZE*/,
size_t unread_limit_ /* = default_limit*/)

View File

@ -8,6 +8,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ATTEMPT_TO_READ_AFTER_EOF;
}
/** Write UInt64 in variable length format (base128) NOTE Only up to 2^63 - 1 are supported. */

View File

@ -3,6 +3,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ZLIB_INFLATE_FAILED;
}
ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
std::unique_ptr<ReadBuffer> in_,

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ATTEMPT_TO_READ_AFTER_EOF;
}
namespace
{

View File

@ -22,6 +22,10 @@ struct DefaultHash<StringRef> : public StringRefHash {};
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
using Sizes = std::vector<size_t>;

View File

@ -10,6 +10,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
static constexpr UInt64 SEED_GEN_A = 845897321;
static constexpr UInt64 SEED_GEN_B = 217728422;

View File

@ -8,6 +8,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// Cluster contains connection pools to each node
/// With the local nodes, the connection is not established, but the request is executed directly.

View File

@ -13,6 +13,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database)
{

View File

@ -19,6 +19,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
class GlobalSubqueriesMatcher

View File

@ -6,6 +6,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** Interpreters interface for different queries.
*/

View File

@ -13,6 +13,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/** Methods for different implementations of sets (used in right hand side of IN or for DISTINCT).
* To use as template parameter.

View File

@ -6,6 +6,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
extern const int LOGICAL_ERROR;
}
void TableStatus::write(WriteBuffer & out) const
{

View File

@ -15,6 +15,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns)

View File

@ -7,6 +7,10 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
}
ASTPtr ASTColumnsMatcher::clone() const
{

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