FunctionsMiscellaneous.h 39.3 KB
Newer Older
A
Alexey Milovidov 已提交
1 2
#pragma once

3
#include <Poco/Net/DNS.h>
4
#include <Yandex/Revision.h>
5

6
#include <DB/Core/Defines.h>
7
#include <DB/IO/WriteBufferFromString.h>
A
Alexey Milovidov 已提交
8 9 10 11 12
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
13
#include <DB/DataTypes/DataTypeTuple.h>
14
#include <DB/DataTypes/DataTypeArray.h>
15
#include <DB/DataTypes/DataTypeAggregateFunction.h>
A
Alexey Milovidov 已提交
16 17 18 19
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnVector.h>
20
#include <DB/Columns/ColumnSet.h>
21
#include <DB/Columns/ColumnTuple.h>
22
#include <DB/Columns/ColumnArray.h>
23
#include <DB/Columns/ColumnAggregateFunction.h>
24
#include <DB/Common/UnicodeBar.h>
A
Alexey Milovidov 已提交
25
#include <DB/Functions/IFunction.h>
26
#include <DB/Interpreters/ExpressionActions.h>
27
#include <statdaemons/ext/range.hpp>
A
Alexey Milovidov 已提交
28

29
#include <cmath>
A
Alexey Milovidov 已提交
30 31 32 33 34 35

namespace DB
{

/** Вспомогательные функции:
  *
36
  * visibleWidth(x)	- вычисляет приблизительную ширину при выводе значения в текстовом (tab-separated) виде на консоль.
37
  *
38 39 40
  * toTypeName(x)	- получить имя типа
  * blockSize()		- получить размер блока
  * materialize(x)	- материализовать константу
41
  * ignore(...)		- функция, принимающая любые аргументы, и всегда возвращающая 0.
42
  * sleep(seconds)	- спит указанное количество секунд каждый блок.
43
  *
44 45
  * in(x, set)		- функция для вычисления оператора IN
  * notIn(x, set)	-  и NOT IN.
46 47
  *
  * tuple(x, y, ...) - функция, позволяющая сгруппировать несколько столбцов
48
  * tupleElement(tuple, n) - функция, позволяющая достать столбец из tuple.
49
  *
50
  * arrayJoin(arr)	- особая функция - выполнить её напрямую нельзя;
51
  *                   используется только чтобы получить тип результата соответствующего выражения.
52
  *
53 54
  * replicate(x, arr) - создаёт массив такого же размера как arr, все элементы которого равны x;
  * 					например: replicate(1, ['a', 'b', 'c']) = [1, 1, 1].
55
  *
56
  * sleep(n)		- спит n секунд каждый блок.
57 58
  *
  * bar(x, min, max, width) - рисует полосу из количества символов, пропорционального (x - min) и равного width при x == max.
59 60
  *
  * version()       - возвращает текущую версию сервера в строке.
61 62 63 64 65
  *
  * finalizeAggregation(agg_state) - по состоянию агрегации получить результат.
  *
  * runningAccumulate(agg_state) - принимает состояния агрегатной функции и возвращает столбец со значениями,
  *  являющимися результатом накопления этих состояний для множества строк блока, от первой до текущей строки.
A
Alexey Milovidov 已提交
66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81
  */


static inline UInt64 stringWidth(const UInt8 * pos, const UInt8 * end)
{
	UInt64 res = 0;
 	for (; pos < end; ++pos)
	{
		if (*pos == '\b' || *pos == '\f' || *pos == '\n' || *pos == '\r' || *pos == '\t' || *pos == '\0' || *pos == '\'' || *pos == '\\')
			++res;
		if (*pos <= 0x7F || *pos >= 0xC0)
			++res;
	}
	return res;
}

82
static inline void stringWidthConstant(const String & data, UInt64 & res)
A
Alexey Milovidov 已提交
83 84 85 86
{
	res = stringWidth(reinterpret_cast<const UInt8 *>(data.data()), reinterpret_cast<const UInt8 *>(data.data()) + data.size());
}

87 88
class FunctionCurrentDatabase : public IFunction
{
89
	const String db_name;
90 91

public:
92 93 94 95
	static constexpr auto name = "currentDatabase";
	static IFunction * create(const Context & context) { return new FunctionCurrentDatabase{context.getCurrentDatabase()}; }

	explicit FunctionCurrentDatabase(const String & db_name) : db_name{db_name} {}
96 97

	String getName() const {
98
		return name;
99 100 101 102 103 104 105 106 107 108 109 110 111 112 113
	}

	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 0)
			throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
				+ toString(arguments.size()) + ", should be 0.",
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		return new DataTypeString;
	}

	void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
	{
		block.getByPosition(result).column = new ColumnConstString{
114
			block.rowsInFirstColumn(), db_name
115 116 117 118
		};
	}
};

119

120 121 122 123
/// Получить имя хоста. (Оно - константа, вычисляется один раз за весь запрос.)
class FunctionHostName : public IFunction
{
public:
124 125 126
	static constexpr auto name = "hostName";
	static IFunction * create(const Context & context) { return new FunctionHostName; }

127 128 129
	/// Получить имя функции.
	String getName() const
	{
130
		return name;
131 132 133 134 135 136 137 138 139 140 141 142 143
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 0)
			throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
				+ toString(arguments.size()) + ", should be 0.",
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		return new DataTypeString;
	}

144 145
	/** Выполнить функцию над блоком. convertToFullColumn вызывается для того, чтобы в случае
	 *	распределенного выполнения запроса каждый сервер возвращал свое имя хоста. */
146 147
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
A
Alexey Milovidov 已提交
148
		block.getByPosition(result).column = ColumnConstString(
149
			block.rowsInFirstColumn(),
A
Alexey Milovidov 已提交
150
			Poco::Net::DNS::hostName()).convertToFullColumn();
151 152
	}
};
A
Alexey Milovidov 已提交
153

154

A
Alexey Milovidov 已提交
155 156 157
class FunctionVisibleWidth : public IFunction
{
public:
158 159 160
	static constexpr auto name = "visibleWidth";
	static IFunction * create(const Context & context) { return new FunctionVisibleWidth; }

A
Alexey Milovidov 已提交
161 162 163
	/// Получить имя функции.
	String getName() const
	{
164
		return name;
A
Alexey Milovidov 已提交
165 166 167 168 169 170 171
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 1)
			throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
172
				+ toString(arguments.size()) + ", should be 1.",
A
Alexey Milovidov 已提交
173 174 175 176 177 178
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		return new DataTypeUInt64;
	}

	/// Выполнить функцию над блоком.
179
	void execute(Block & block, const ColumnNumbers & arguments, size_t result);
A
Alexey Milovidov 已提交
180 181
};

182 183 184 185

class FunctionToTypeName : public IFunction
{
public:
186 187 188
	static constexpr auto name = "toTypeName";
	static IFunction * create(const Context & context) { return new FunctionToTypeName; }

189 190 191
	/// Получить имя функции.
	String getName() const
	{
192
		return name;
193 194 195 196 197
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
198
		if (arguments.size() != 1)
199
			throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
200
				+ toString(arguments.size()) + ", should be 1.",
201 202 203 204 205 206 207 208
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		return new DataTypeString;
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
209
		block.getByPosition(result).column = new ColumnConstString(block.rowsInFirstColumn(), block.getByPosition(arguments[0]).type->getName());
210 211 212
	}
};

213 214 215 216

class FunctionBlockSize : public IFunction
{
public:
217 218 219
	static constexpr auto name = "blockSize";
	static IFunction * create(const Context & context) { return new FunctionBlockSize; }

220 221 222
	/// Получить имя функции.
	String getName() const
	{
223
		return name;
224 225 226 227 228 229 230
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (!arguments.empty())
			throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
231
				+ toString(arguments.size()) + ", should be 0.",
232 233 234 235 236 237 238 239
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		return new DataTypeUInt64;
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
240
		size_t size = block.rowsInFirstColumn();
241
		block.getByPosition(result).column = ColumnConstUInt64(size, size).convertToFullColumn();
242 243 244 245
	}
};


246 247 248
class FunctionSleep : public IFunction
{
public:
249 250 251
	static constexpr auto name = "sleep";
	static IFunction * create(const Context & context) { return new FunctionSleep; }

252 253 254
	/// Получить имя функции.
	String getName() const
	{
255
		return name;
256
	}
257

258 259 260 261 262
	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 1)
			throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
263
			+ toString(arguments.size()) + ", should be 1.",
264
							ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
265 266 267 268 269 270 271

		if (!typeid_cast<const DataTypeFloat64 *>(&*arguments[0]) &&
			!typeid_cast<const DataTypeFloat32 *>(&*arguments[0]) &&
			!typeid_cast<const DataTypeUInt64 *>(&*arguments[0]) &&
			!typeid_cast<const DataTypeUInt32 *>(&*arguments[0]) &&
			!typeid_cast<const DataTypeUInt16 *>(&*arguments[0]) &&
			!typeid_cast<const DataTypeUInt8 *>(&*arguments[0]))
272 273
			throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected Float64",
			ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
274

275 276
		return new DataTypeUInt8;
	}
277

278 279 280
	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
281 282
		IColumn * col = &*block.getByPosition(arguments[0]).column;
		double seconds;
283
		size_t size = col->size();
284 285

		if (ColumnConst<Float64> * column = typeid_cast<ColumnConst<Float64> *>(col))
286
			seconds = column->getData();
287 288

		else if (ColumnConst<Float32> * column = typeid_cast<ColumnConst<Float32> *>(col))
289
			seconds = static_cast<double>(column->getData());
290 291

		else if (ColumnConst<UInt64> * column = typeid_cast<ColumnConst<UInt64> *>(col))
292
			seconds = static_cast<double>(column->getData());
293 294

		else if (ColumnConst<UInt32> * column = typeid_cast<ColumnConst<UInt32> *>(col))
295
			seconds = static_cast<double>(column->getData());
296 297

		else if (ColumnConst<UInt16> * column = typeid_cast<ColumnConst<UInt16> *>(col))
298
			seconds = static_cast<double>(column->getData());
299 300

		else if (ColumnConst<UInt8> * column = typeid_cast<ColumnConst<UInt8> *>(col))
301
			seconds = static_cast<double>(column->getData());
302

303
		else
304
			throw Exception("The argument of function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
305

306 307 308
		/// Не спим, если блок пустой.
		if (size > 0)
			usleep(static_cast<unsigned>(seconds * 1e6));
309

310
		block.getByPosition(result).column = ColumnConst<UInt8>(size, 0).convertToFullColumn();
311 312 313 314
	}
};


315 316 317
class FunctionMaterialize : public IFunction
{
public:
318 319 320
	static constexpr auto name = "materialize";
	static IFunction * create(const Context & context) { return new FunctionMaterialize; }

321 322 323
	/// Получить имя функции.
	String getName() const
	{
324
		return name;
325 326 327 328 329 330 331
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 1)
			throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
332
				+ toString(arguments.size()) + ", should be 1.",
333 334 335 336 337 338 339 340 341 342
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		return arguments[0];
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
		const IColumn & argument = *block.getByPosition(arguments[0]).column;
		if (!argument.isConst())
343
			throw Exception("Argument for function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
344

345 346 347 348
		block.getByPosition(result).column = dynamic_cast<const IColumnConst &>(argument).convertToFullColumn();
	}
};

349 350 351 352 353
template <bool negative, bool global> struct FunctionInName;
template <> struct FunctionInName<false, false>	{ static constexpr auto name = "in"; };
template <> struct FunctionInName<false, true>	{ static constexpr auto name = "globalIn"; };
template <> struct FunctionInName<true, false>	{ static constexpr auto name = "notIn"; };
template <> struct FunctionInName<true, true>	{ static constexpr auto name = "globalNotIn"; };
354

355
template <bool negative, bool global>
356 357 358
class FunctionIn : public IFunction
{
public:
359 360
	static constexpr auto name = FunctionInName<negative, global>::name;
	static IFunction * create(const Context & context) { return new FunctionIn; }
361

362 363 364
	/// Получить имя функции.
	String getName() const
	{
365
		return name;
366 367 368 369 370 371
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 2)
372
			throw Exception("Number of arguments for function '" + getName() + "' doesn't match: passed "
373
				+ toString(arguments.size()) + ", should be 2.",
374 375 376 377 378 379 380 381 382
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		return new DataTypeUInt8;
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
		/// Второй аргумент - обязательно ColumnSet.
383
		ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column;
384
		const ColumnSet * column_set = typeid_cast<const ColumnSet *>(&*column_set_ptr);
385
		if (!column_set)
386 387
			throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(),
							ErrorCodes::ILLEGAL_COLUMN);
388

389 390
		/// Столбцы, которые проверяются на принадлежность множеству.
		ColumnNumbers left_arguments;
391

392
		/// Первый аргумент может быть tuple или одиночным столбцом.
393
		const ColumnTuple * tuple = typeid_cast<const ColumnTuple *>(&*block.getByPosition(arguments[0]).column);
394 395 396 397 398 399 400 401 402 403 404
		if (tuple)
		{
			/// Находим в блоке столбцы из tuple.
			const Block & tuple_elems = tuple->getData();
			size_t tuple_size = tuple_elems.columns();
			for (size_t i = 0; i < tuple_size; ++i)
				left_arguments.push_back(block.getPositionByName(tuple_elems.getByPosition(i).name));
		}
		else
			left_arguments.push_back(arguments[0]);

405
		column_set->getData()->execute(block, left_arguments, result, negative);
406 407 408 409 410 411 412
	}
};


class FunctionTuple : public IFunction
{
public:
413 414 415
	static constexpr auto name = "tuple";
	static IFunction * create(const Context & context) { return new FunctionTuple; }

416 417 418
	/// Получить имя функции.
	String getName() const
	{
419
		return name;
420 421 422 423 424
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
425 426 427
		if (arguments.size() < 1)
			throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

428 429 430 431 432 433 434 435 436 437
		return new DataTypeTuple(arguments);
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
		Block tuple_block;

		for (ColumnNumbers::const_iterator it = arguments.begin(); it != arguments.end(); ++it)
			tuple_block.insert(block.getByPosition(*it));
438

439
		block.getByPosition(result).column = new ColumnTuple(tuple_block);
440 441 442
	}
};

443 444 445 446

class FunctionTupleElement : public IFunction
{
public:
447 448 449
	static constexpr auto name = "tupleElement";
	static IFunction * create(const Context & context) { return new FunctionTupleElement; }

450 451 452
	/// Получить имя функции.
	String getName() const
	{
453
		return name;
454 455
	}

456
	void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
457 458
										DataTypePtr & out_return_type,
										ExpressionActions::Actions & out_prerequisites)
459 460
	{
		if (arguments.size() != 2)
461
			throw Exception("Function " + getName() + " requires exactly two arguments: tuple and element index.",
462
							ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
463 464

		const ColumnConstUInt8 * index_col = typeid_cast<const ColumnConstUInt8 *>(&*arguments[1].column);
465
		if (!index_col)
466
			throw Exception("Second argument to " + getName() + " must be a constant UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
467

468
		size_t index = index_col->getData();
469 470

		const DataTypeTuple * tuple = typeid_cast<const DataTypeTuple *>(&*arguments[0].type);
471
		if (!tuple)
472
			throw Exception("First argument for function " + getName() + " must be tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
473

474 475
		if (index == 0)
			throw Exception("Indices in tuples are 1-based.", ErrorCodes::ILLEGAL_INDEX);
476

477
		const DataTypes & elems = tuple->getElements();
478

479 480
		if (index > elems.size())
			throw Exception("Index for tuple element is out of range.", ErrorCodes::ILLEGAL_INDEX);
481

482 483
		out_return_type = elems[index - 1]->clone();
	}
484

485 486 487
	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
488 489
		const ColumnTuple * tuple_col = typeid_cast<const ColumnTuple *>(&*block.getByPosition(arguments[0]).column);
		const ColumnConstUInt8 * index_col = typeid_cast<const ColumnConstUInt8 *>(&*block.getByPosition(arguments[1]).column);
490 491

		if (!tuple_col)
492
			throw Exception("First argument for function " + getName() + " must be tuple.", ErrorCodes::ILLEGAL_COLUMN);
493 494

		if (!index_col)
495
			throw Exception("Second argument for function " + getName() + " must be UInt8 constant literal.", ErrorCodes::ILLEGAL_COLUMN);
496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513

		size_t index = index_col->getData();
		if (index == 0)
			throw Exception("Indices in tuples is 1-based.", ErrorCodes::ILLEGAL_INDEX);

		const Block & tuple_block = tuple_col->getData();

		if (index > tuple_block.columns())
			throw Exception("Index for tuple element is out of range.", ErrorCodes::ILLEGAL_INDEX);

		block.getByPosition(result).column = tuple_block.getByPosition(index - 1).column;
	}
};


class FunctionIgnore : public IFunction
{
public:
514 515 516
	static constexpr auto name = "ignore";
	static IFunction * create(const Context & context) { return new FunctionIgnore; }

517 518 519
	/// Получить имя функции.
	String getName() const
	{
520
		return name;
521 522 523 524 525 526 527 528 529 530 531
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		return new DataTypeUInt8;
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
532
		block.getByPosition(result).column = new ColumnConstUInt8(block.rowsInFirstColumn(), 0);
533 534 535
	}
};

536

537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566
class FunctionIdentity : public IFunction
{
public:
	static constexpr auto name = "identity";
	static IFunction * create(const Context & context) { return new FunctionIdentity; }

	/// Получить имя функции.
	String getName() const
	{
		return name;
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 1)
			throw Exception("Function " + getName() + " requires exactly one argument.",
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		return arguments.front()->clone();
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
		block.getByPosition(result).column = block.getByPosition(arguments.front()).column;
	}
};


567 568 569
class FunctionArrayJoin : public IFunction
{
public:
570 571 572 573
	static constexpr auto name = "arrayJoin";
	static IFunction * create(const Context & context) { return new FunctionArrayJoin; }


574 575 576
	/// Получить имя функции.
	String getName() const
	{
577
		return name;
578 579 580 581 582 583
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 1)
584
			throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
585

586
		const DataTypeArray * arr = typeid_cast<const DataTypeArray *>(&*arguments[0]);
587
		if (!arr)
588
			throw Exception("Argument for function " + getName() + " must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
589

590 591 592 593 594 595
		return arr->getNestedType()->clone();
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
596
		throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL);
597 598 599
	}
};

600

601
/** Создаёт массив, размножая столбец (первый аргумент) по количеству элементов в массиве (втором аргументе).
602 603
  * Используется только в качестве prerequisites для функций высшего порядка.
  */
604 605
class FunctionReplicate : public IFunction
{
606
public:
607 608 609
	static constexpr auto name = "replicate";
	static IFunction * create(const Context & context) { return new FunctionReplicate; }

610 611 612
	/// Получить имя функции.
	String getName() const
	{
613
		return name;
614 615 616 617 618 619 620
	}

	/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 2)
			throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
621
							+ toString(arguments.size()) + ", should be 2.",
622 623
							ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

624
		const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(&*arguments[1]);
625 626 627
		if (!array_type)
			throw Exception("Second argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

628
		return new DataTypeArray(arguments[0]->clone());
629 630 631 632 633 634
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
		ColumnPtr first_column = block.getByPosition(arguments[0]).column;
635

636
		ColumnArray * array_column = typeid_cast<ColumnArray *>(&*block.getByPosition(arguments[1]).column);
637
		ColumnPtr temp_column;
638

639 640
		if (!array_column)
		{
641
			ColumnConstArray * const_array_column = typeid_cast<ColumnConstArray *>(&*block.getByPosition(arguments[1]).column);
642 643
			if (!const_array_column)
				throw Exception("Unexpected column for replicate", ErrorCodes::ILLEGAL_COLUMN);
644
			temp_column = const_array_column->convertToFullColumn();
645
			array_column = typeid_cast<ColumnArray *>(&*temp_column);
646
		}
647

648 649 650
		block.getByPosition(result).column = new ColumnArray(
			first_column->replicate(array_column->getOffsets()),
			array_column->getOffsetsColumn());
651 652 653
	}
};

654 655 656 657

class FunctionBar : public IFunction
{
public:
658 659 660
	static constexpr auto name = "bar";
	static IFunction * create(const Context & context) { return new FunctionBar; }

661 662 663
	/// Получить имя функции.
	String getName() const
	{
664
		return name;
665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766
	}

	/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
	DataTypePtr getReturnType(const DataTypes & arguments) const
	{
		if (arguments.size() != 3 && arguments.size() != 4)
			throw Exception("Function " + getName() + " requires from 3 or 4 parameters: value, min_value, max_value, [max_width_of_bar = 80]. Passed "
				+ toString(arguments.size()) + ".",
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		if (!arguments[0]->isNumeric() || !arguments[1]->isNumeric() || !arguments[2]->isNumeric()
			|| (arguments.size() == 4 && !arguments[3]->isNumeric()))
			throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

		return new DataTypeString;
	}

	/// Выполнить функцию над блоком.
	void execute(Block & block, const ColumnNumbers & arguments, size_t result)
	{
		Int64 min = extractConstant<Int64>(block, arguments, 1, "Second");	/// Уровень значения, при котором полоска имеет нулевую длину.
		Int64 max = extractConstant<Int64>(block, arguments, 2, "Third");	/// Уровень значения, при котором полоска имеет максимальную длину.

		/// Максимальная ширина полоски в символах, по-умолчанию.
		Float64 max_width = arguments.size() == 4
			? extractConstant<Float64>(block, arguments, 3, "Fourth")
			: 80;

		if (max_width < 1)
			throw Exception("Max_width argument must be >= 1.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);

		if (max_width > 1000)
			throw Exception("Too large max_width.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);

		const auto & src = *block.getByPosition(arguments[0]).column;

		if (src.isConst())
		{
			auto res_column = new ColumnConstString(block.rowsInFirstColumn(), "");
			block.getByPosition(result).column = res_column;

			if (   executeConstNumber<UInt8>	(src, *res_column, min, max, max_width)
				|| executeConstNumber<UInt16>	(src, *res_column, min, max, max_width)
				|| executeConstNumber<UInt32>	(src, *res_column, min, max, max_width)
				|| executeConstNumber<UInt64>	(src, *res_column, min, max, max_width)
				|| executeConstNumber<Int8>		(src, *res_column, min, max, max_width)
				|| executeConstNumber<Int16>	(src, *res_column, min, max, max_width)
				|| executeConstNumber<Int32>	(src, *res_column, min, max, max_width)
				|| executeConstNumber<Int64>	(src, *res_column, min, max, max_width)
				|| executeConstNumber<Float32>	(src, *res_column, min, max, max_width)
				|| executeConstNumber<Float64>	(src, *res_column, min, max, max_width))
			{
			}
			else
				throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
					+ " of argument of function " + getName(),
					ErrorCodes::ILLEGAL_COLUMN);
		}
		else
		{
			auto res_column = new ColumnString;
			block.getByPosition(result).column = res_column;

			if (   executeNumber<UInt8>		(src, *res_column, min, max, max_width)
				|| executeNumber<UInt16>	(src, *res_column, min, max, max_width)
				|| executeNumber<UInt32>	(src, *res_column, min, max, max_width)
				|| executeNumber<UInt64>	(src, *res_column, min, max, max_width)
				|| executeNumber<Int8>		(src, *res_column, min, max, max_width)
				|| executeNumber<Int16>		(src, *res_column, min, max, max_width)
				|| executeNumber<Int32>		(src, *res_column, min, max, max_width)
				|| executeNumber<Int64>		(src, *res_column, min, max, max_width)
				|| executeNumber<Float32>	(src, *res_column, min, max, max_width)
				|| executeNumber<Float64>	(src, *res_column, min, max, max_width))
			{
			}
			else
				throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
					+ " of argument of function " + getName(),
					ErrorCodes::ILLEGAL_COLUMN);
		}
	}

private:
	template <typename T>
	T extractConstant(Block & block, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const
	{
		const auto & column = *block.getByPosition(arguments[argument_pos]).column;

		if (!column.isConst())
			throw Exception(which_argument + String(" argument for function ") + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);

		return apply_visitor(FieldVisitorConvertToNumber<T>(), column[0]);
	}

	template <typename T>
	static void fill(const PODArray<T> & src, ColumnString::Chars_t & dst_chars, ColumnString::Offsets_t & dst_offsets,
		Int64 min, Int64 max, Float64 max_width)
	{
		size_t size = src.size();
		size_t current_offset = 0;

		dst_offsets.resize(size);
767
		dst_chars.reserve(size * (UnicodeBar::getWidthInBytes(max_width) + 1));	/// строки 0-terminated.
768 769 770

		for (size_t i = 0; i < size; ++i)
		{
771 772
			Float64 width = UnicodeBar::getWidth(src[i], min, max, max_width);
			size_t next_size = current_offset + UnicodeBar::getWidthInBytes(width) + 1;
773
			dst_chars.resize(next_size);
774
			UnicodeBar::render(width, reinterpret_cast<char *>(&dst_chars[current_offset]));
775 776 777 778 779 780 781 782 783
			current_offset = next_size;
			dst_offsets[i] = current_offset;
		}
	}

	template <typename T>
	static void fill(T src, String & dst_chars,
		Int64 min, Int64 max, Float64 max_width)
	{
784 785 786
		Float64 width = UnicodeBar::getWidth(src, min, max, max_width);
		dst_chars.resize(UnicodeBar::getWidthInBytes(width));
		UnicodeBar::render(width, &dst_chars[0]);
787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813
	}

	template <typename T>
	static bool executeNumber(const IColumn & src, ColumnString & dst, Int64 min, Int64 max, Float64 max_width)
	{
		if (const ColumnVector<T> * col = typeid_cast<const ColumnVector<T> *>(&src))
		{
			fill(col->getData(), dst.getChars(), dst.getOffsets(), min, max, max_width);
			return true;
		}
		else
			return false;
	}

	template <typename T>
	static bool executeConstNumber(const IColumn & src, ColumnConstString & dst, Int64 min, Int64 max, Float64 max_width)
	{
		if (const ColumnConst<T> * col = typeid_cast<const ColumnConst<T> *>(&src))
		{
			fill(col->getData(), dst.getData(), min, max, max_width);
			return true;
		}
		else
			return false;
	}
};

814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916

template <typename Impl>
class FunctionNumericPredicate : public IFunction
{
public:
	static constexpr auto name = Impl::name;
	static IFunction * create(const Context &) { return new FunctionNumericPredicate; }

	String getName() const override { return name; }

	DataTypePtr getReturnType(const DataTypes & arguments) const override
	{
		const auto args_size = arguments.size();
		if (args_size != 1)
			throw Exception{
				"Number of arguments for function " + getName() + " doesn't match: passed " +
					toString(args_size) + ", should be 1",
				ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
			};

		const auto arg = arguments.front().get();
		if (!typeid_cast<const DataTypeUInt8 *>(arg) &&
			!typeid_cast<const DataTypeUInt16 *>(arg) &&
			!typeid_cast<const DataTypeUInt32 *>(arg) &&
			!typeid_cast<const DataTypeUInt64 *>(arg) &&
			!typeid_cast<const DataTypeInt8 *>(arg) &&
			!typeid_cast<const DataTypeInt16 *>(arg) &&
			!typeid_cast<const DataTypeInt32 *>(arg) &&
			!typeid_cast<const DataTypeInt64 *>(arg) &&
			!typeid_cast<const DataTypeFloat32 *>(arg) &&
			!typeid_cast<const DataTypeFloat64 *>(arg))
			throw Exception{
				"Argument for function " + getName() + " must be numeric",
				ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
			};

		return new DataTypeUInt8;
	}

	void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
	{
		const auto in = block.getByPosition(arguments.front()).column.get();

		if (!execute<UInt8>(block, in, result) &&
			!execute<UInt16>(block, in, result) &&
			!execute<UInt32>(block, in, result) &&
			!execute<UInt64>(block, in, result) &&
			!execute<Int8>(block, in, result) &&
			!execute<Int16>(block, in, result) &&
			!execute<Int32>(block, in, result) &&
			!execute<Int64>(block, in, result) &&
			!execute<Float32>(block, in, result)  &&
			!execute<Float64>(block, in, result))
			throw Exception{
				"Illegal column " + in->getName() + " of first argument of function " + getName(),
				ErrorCodes::ILLEGAL_COLUMN
			};
	}

	template <typename T>
	bool execute(Block & block, const IColumn * in_untyped, const size_t result) override
	{
		if (const auto in = typeid_cast<const ColumnVector<T> *>(in_untyped))
		{
			const auto size = in->size();

			const auto out = new ColumnVector<UInt8>{size};
			block.getByPosition(result).column = out;

			const auto & in_data = in->getData();
			auto & out_data = out->getData();

			for (const auto i : ext::range(0, size))
				out_data[i] = Impl::execute(in_data[i]);

			return true;
		}
		else if (const auto in = typeid_cast<const ColumnConst<T> *>(in_untyped))
		{
			block.getByPosition(result).column = new ColumnConstUInt8{
				in->size(),
				Impl::execute(in->getData())
			};

			return true;
		}

		return false;
	}
};

struct IsFiniteImpl
{
	static constexpr auto name = "isFinite";
	template <typename T> static bool execute(const T t) { return std::isfinite(t); }
};

struct IsInfiniteImpl
{
	static constexpr auto name = "isInfinite";
	template <typename T> static bool execute(const T t) { return std::isinf(t); }
};

917
struct IsNaNImpl
918
{
919
	static constexpr auto name = "isNaN";
920 921 922 923 924
	template <typename T> static bool execute(const T t) { return std::isnan(t); }
};

using FunctionIsFinite = FunctionNumericPredicate<IsFiniteImpl>;
using FunctionIsInfinite = FunctionNumericPredicate<IsInfiniteImpl>;
925
using FunctionIsNaN = FunctionNumericPredicate<IsNaNImpl>;
926

927

928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956
class FunctionVersion : public IFunction
{
public:
	static constexpr auto name = "version";
	static IFunction * create(const Context & context) { return new FunctionVersion; }

	String getName() const override { return name; }

	DataTypePtr getReturnType(const DataTypes & arguments) const override
	{
		if (!arguments.empty())
			throw Exception("Function " + getName() + " must be called without arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
		return new DataTypeString;
	}

	void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
	{
		static const std::string version = getVersion();
		block.getByPosition(result).column = new ColumnConstString(version.length(), version);
	}

private:
	std::string getVersion() const
	{
		std::ostringstream os;
		os << DBMS_VERSION_MAJOR << "." << DBMS_VERSION_MINOR << "." << Revision::get();
		return os.str();
	}
};
957

958 959 960 961 962 963 964 965 966

/** Весьма необычная функция.
  * Принимает состояние агрегатной функции (например runningAccumulate(uniqState(UserID))),
  *  и для каждой строки блока, возвращает результат агрегатной функции по объединению состояний от всех предыдущих строк блока и текущей строки.
  *
  * То есть, функция зависит от разбиения данных на блоки и от порядка строк в блоке.
  */
class FunctionRunningAccumulate : public IFunction
{
967 968 969 970 971 972 973
public:
	static constexpr auto name = "runningAccumulate";
	static IFunction * create(const Context & context) { return new FunctionRunningAccumulate; }

	String getName() const override { return name; }

	DataTypePtr getReturnType(const DataTypes & arguments) const override
974
	{
975 976 977 978 979 980 981 982 983
		if (arguments.size() != 1)
			throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		const DataTypeAggregateFunction * type = typeid_cast<const DataTypeAggregateFunction *>(&*arguments[0]);
		if (!type)
			throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.",
				ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

		return type->getReturnType()->clone();
984 985
	}

986
	void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013
	{
		const ColumnAggregateFunction * column_with_states = typeid_cast<const ColumnAggregateFunction *>(&*block.getByPosition(arguments.at(0)).column);
		if (!column_with_states)
			throw Exception(
				"Illegal column " + block.getByPosition(arguments.at(0)).column->getName() + " of first argument of function " + getName(),
				ErrorCodes::ILLEGAL_COLUMN);

		AggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction();
		const IAggregateFunction & agg_func = *aggregate_function_ptr;

		auto deleter = [&agg_func] (char * ptr) { agg_func.destroy(ptr); free(ptr); };
		std::unique_ptr<char, decltype(deleter)> place { reinterpret_cast<char *>(malloc(agg_func.sizeOfData())), deleter };

		agg_func.create(place.get());	/// Немного не exception-safe. Если здесь выкинется исключение, то зря вызовется destroy.

		ColumnPtr result_column_ptr = agg_func.getReturnType()->createColumn();
		block.getByPosition(result).column = result_column_ptr;
		IColumn & result_column = *result_column_ptr;
		result_column.reserve(column_with_states->size());

		const auto & states = column_with_states->getData();
		for (const auto & state_to_add : states)
		{
			agg_func.merge(place.get(), state_to_add);
			agg_func.insertResultInto(place.get(), result_column);
		}
	}
1014
};
1015

1016

1017 1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051
/** Принимает состояние агрегатной функции. Возвращает результат агрегации.
  */
class FunctionFinalizeAggregation : public IFunction
{
public:
	static constexpr auto name = "finalizeAggregation";
	static IFunction * create(const Context & context) { return new FunctionFinalizeAggregation; }

	String getName() const override { return name; }

	DataTypePtr getReturnType(const DataTypes & arguments) const override
	{
		if (arguments.size() != 1)
			throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

		const DataTypeAggregateFunction * type = typeid_cast<const DataTypeAggregateFunction *>(&*arguments[0]);
		if (!type)
			throw Exception("Argument for function " + getName() + " must have type AggregateFunction - state of aggregate function.",
				ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

		return type->getReturnType()->clone();
	}

	void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
	{
		ColumnAggregateFunction * column_with_states = typeid_cast<ColumnAggregateFunction *>(&*block.getByPosition(arguments.at(0)).column);
		if (!column_with_states)
			throw Exception(
				"Illegal column " + block.getByPosition(arguments.at(0)).column->getName() + " of first argument of function " + getName(),
				ErrorCodes::ILLEGAL_COLUMN);

		block.getByPosition(result).column = column_with_states->convertToValues();
	}
};

A
Alexey Milovidov 已提交
1052
}