1#pragma once
2
3#include <DataTypes/DataTypesNumber.h>
4#include <DataTypes/DataTypeArray.h>
5#include <DataTypes/DataTypeString.h>
6#include <Columns/ColumnsNumber.h>
7#include <Columns/ColumnConst.h>
8#include <Columns/ColumnArray.h>
9#include <Columns/ColumnString.h>
10#include <Interpreters/Context.h>
11#include <Interpreters/EmbeddedDictionaries.h>
12#include <Functions/IFunctionImpl.h>
13#include <Functions/FunctionHelpers.h>
14#include <Dictionaries/Embedded/RegionsHierarchy.h>
15#include <Dictionaries/Embedded/RegionsHierarchies.h>
16#include <Dictionaries/Embedded/RegionsNames.h>
17#include <IO/WriteHelpers.h>
18#include <Common/config.h>
19#include <Common/typeid_cast.h>
20
21
22namespace DB
23{
24
25namespace ErrorCodes
26{
27 extern const int DICTIONARIES_WAS_NOT_LOADED;
28 extern const int BAD_ARGUMENTS;
29 extern const int ILLEGAL_COLUMN;
30 extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
31}
32
33/** Functions using Yandex.Metrica dictionaries
34 * - dictionaries of regions, operating systems, search engines.
35 *
36 * Climb up the tree to a certain level.
37 * regionToCity, regionToArea, regionToCountry, ...
38 *
39 * Convert values of a column
40 * regionToName
41 *
42 * Whether the first identifier is a descendant of the second.
43 * regionIn
44 *
45 * Get an array of region identifiers, consisting of the source and the parents chain. Order implementation defined.
46 * regionHierarchy
47 */
48
49
50struct RegionToCityImpl
51{
52 static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toCity(x); }
53};
54
55struct RegionToAreaImpl
56{
57 static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toArea(x); }
58};
59
60struct RegionToDistrictImpl
61{
62 static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toDistrict(x); }
63};
64
65struct RegionToCountryImpl
66{
67 static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toCountry(x); }
68};
69
70struct RegionToContinentImpl
71{
72 static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toContinent(x); }
73};
74
75struct RegionToTopContinentImpl
76{
77 static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toTopContinent(x); }
78};
79
80struct RegionToPopulationImpl
81{
82 static UInt32 apply(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.getPopulation(x); }
83};
84
85struct RegionInImpl
86{
87 static bool apply(UInt32 x, UInt32 y, const RegionsHierarchy & hierarchy) { return hierarchy.in(x, y); }
88};
89
90struct RegionHierarchyImpl
91{
92 static UInt32 toParent(UInt32 x, const RegionsHierarchy & hierarchy) { return hierarchy.toParent(x); }
93};
94
95
96/** Auxiliary thing, allowing to get from the dictionary a specific dictionary, corresponding to the point of view
97 * (the dictionary key passed as function argument).
98 * Example: when calling regionToCountry(x, 'ua'), a dictionary can be used, in which Crimea refers to Ukraine.
99 */
100struct RegionsHierarchyGetter
101{
102 using Src = const RegionsHierarchies;
103 using Dst = const RegionsHierarchy;
104
105 static Dst & get(Src & src, const std::string & key)
106 {
107 return src.get(key);
108 }
109};
110
111/** For dictionaries without key support. Doing nothing.
112 */
113template <typename Dict>
114struct IdentityDictionaryGetter
115{
116 using Src = const Dict;
117 using Dst = const Dict;
118
119 static Dst & get(Src & src, const std::string & key)
120 {
121 if (key.empty())
122 return src;
123 else
124 throw Exception("Dictionary doesn't support 'point of view' keys.", ErrorCodes::BAD_ARGUMENTS);
125 }
126};
127
128
129/// Converts an identifier using a dictionary.
130template <typename T, typename Transform, typename DictGetter, typename Name>
131class FunctionTransformWithDictionary : public IFunction
132{
133public:
134 static constexpr auto name = Name::name;
135 using base_type = FunctionTransformWithDictionary;
136
137private:
138 const std::shared_ptr<typename DictGetter::Src> owned_dict;
139
140public:
141 FunctionTransformWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
142 : owned_dict(owned_dict_)
143 {
144 if (!owned_dict)
145 throw Exception("Embedded dictionaries were not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
146 }
147
148 String getName() const override
149 {
150 return name;
151 }
152
153 bool isVariadic() const override { return true; }
154 size_t getNumberOfArguments() const override { return 0; }
155
156 DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
157 {
158 if (arguments.size() != 1 && arguments.size() != 2)
159 throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
160 + toString(arguments.size()) + ", should be 1 or 2.",
161 ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
162
163 if (arguments[0]->getName() != TypeName<T>::get())
164 throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName()
165 + " (must be " + String(TypeName<T>::get()) + ")",
166 ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
167
168 if (arguments.size() == 2 && arguments[1]->getName() != TypeName<String>::get())
169 throw Exception("Illegal type " + arguments[1]->getName() + " of the second ('point of view') argument of function " + getName()
170 + " (must be " + String(TypeName<T>::get()) + ")",
171 ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
172
173 return arguments[0];
174 }
175
176 bool useDefaultImplementationForConstants() const override { return true; }
177 ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
178
179 bool isDeterministic() const override { return false; }
180
181 void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
182 {
183 /// The dictionary key that defines the "point of view".
184 std::string dict_key;
185
186 if (arguments.size() == 2)
187 {
188 const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
189
190 if (!key_col)
191 throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
192 + " of second ('point of view') argument of function " + name
193 + ". Must be constant string.",
194 ErrorCodes::ILLEGAL_COLUMN);
195
196 dict_key = key_col->getValue<String>();
197 }
198
199 const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key);
200
201 if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
202 {
203 auto col_to = ColumnVector<T>::create();
204
205 const typename ColumnVector<T>::Container & vec_from = col_from->getData();
206 typename ColumnVector<T>::Container & vec_to = col_to->getData();
207 size_t size = vec_from.size();
208 vec_to.resize(size);
209
210 for (size_t i = 0; i < size; ++i)
211 vec_to[i] = Transform::apply(vec_from[i], dict);
212
213 block.getByPosition(result).column = std::move(col_to);
214 }
215 else
216 throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
217 + " of first argument of function " + name,
218 ErrorCodes::ILLEGAL_COLUMN);
219 }
220};
221
222
223/// Checks belonging using a dictionary.
224template <typename T, typename Transform, typename DictGetter, typename Name>
225class FunctionIsInWithDictionary : public IFunction
226{
227public:
228 static constexpr auto name = Name::name;
229 using base_type = FunctionIsInWithDictionary;
230
231private:
232 const std::shared_ptr<typename DictGetter::Src> owned_dict;
233
234public:
235 FunctionIsInWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
236 : owned_dict(owned_dict_)
237 {
238 if (!owned_dict)
239 throw Exception("Embedded dictionaries were not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
240 }
241
242 String getName() const override
243 {
244 return name;
245 }
246
247 bool isVariadic() const override { return true; }
248 size_t getNumberOfArguments() const override { return 0; }
249
250 DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
251 {
252 if (arguments.size() != 2 && arguments.size() != 3)
253 throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
254 + toString(arguments.size()) + ", should be 2 or 3.",
255 ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
256
257 if (arguments[0]->getName() != TypeName<T>::get())
258 throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
259 + " (must be " + String(TypeName<T>::get()) + ")",
260 ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
261
262 if (arguments[1]->getName() != TypeName<T>::get())
263 throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
264 + " (must be " + String(TypeName<T>::get()) + ")",
265 ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
266
267 if (arguments.size() == 3 && arguments[2]->getName() != TypeName<String>::get())
268 throw Exception("Illegal type " + arguments[2]->getName() + " of the third ('point of view') argument of function " + getName()
269 + " (must be " + String(TypeName<String>::get()) + ")",
270 ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
271
272 return std::make_shared<DataTypeUInt8>();
273 }
274
275 bool isDeterministic() const override { return false; }
276
277 void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
278 {
279 /// The dictionary key that defines the "point of view".
280 std::string dict_key;
281
282 if (arguments.size() == 3)
283 {
284 const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[2]).column.get());
285
286 if (!key_col)
287 throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
288 + " of third ('point of view') argument of function " + name
289 + ". Must be constant string.",
290 ErrorCodes::ILLEGAL_COLUMN);
291
292 dict_key = key_col->getValue<String>();
293 }
294
295 const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key);
296
297 const ColumnVector<T> * col_vec1 = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get());
298 const ColumnVector<T> * col_vec2 = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[1]).column.get());
299 const ColumnConst * col_const1 = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get());
300 const ColumnConst * col_const2 = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[1]).column.get());
301
302 if (col_vec1 && col_vec2)
303 {
304 auto col_to = ColumnUInt8::create();
305
306 const typename ColumnVector<T>::Container & vec_from1 = col_vec1->getData();
307 const typename ColumnVector<T>::Container & vec_from2 = col_vec2->getData();
308 typename ColumnUInt8::Container & vec_to = col_to->getData();
309 size_t size = vec_from1.size();
310 vec_to.resize(size);
311
312 for (size_t i = 0; i < size; ++i)
313 vec_to[i] = Transform::apply(vec_from1[i], vec_from2[i], dict);
314
315 block.getByPosition(result).column = std::move(col_to);
316 }
317 else if (col_vec1 && col_const2)
318 {
319 auto col_to = ColumnUInt8::create();
320
321 const typename ColumnVector<T>::Container & vec_from1 = col_vec1->getData();
322 const T const_from2 = col_const2->template getValue<T>();
323 typename ColumnUInt8::Container & vec_to = col_to->getData();
324 size_t size = vec_from1.size();
325 vec_to.resize(size);
326
327 for (size_t i = 0; i < size; ++i)
328 vec_to[i] = Transform::apply(vec_from1[i], const_from2, dict);
329
330 block.getByPosition(result).column = std::move(col_to);
331 }
332 else if (col_const1 && col_vec2)
333 {
334 auto col_to = ColumnUInt8::create();
335
336 const T const_from1 = col_const1->template getValue<T>();
337 const typename ColumnVector<T>::Container & vec_from2 = col_vec2->getData();
338 typename ColumnUInt8::Container & vec_to = col_to->getData();
339 size_t size = vec_from2.size();
340 vec_to.resize(size);
341
342 for (size_t i = 0; i < size; ++i)
343 vec_to[i] = Transform::apply(const_from1, vec_from2[i], dict);
344
345 block.getByPosition(result).column = std::move(col_to);
346 }
347 else if (col_const1 && col_const2)
348 {
349 block.getByPosition(result).column = DataTypeUInt8().createColumnConst(col_const1->size(),
350 toField(Transform::apply(col_const1->template getValue<T>(), col_const2->template getValue<T>(), dict)));
351 }
352 else
353 throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName()
354 + " and " + block.getByPosition(arguments[1]).column->getName()
355 + " of arguments of function " + name,
356 ErrorCodes::ILLEGAL_COLUMN);
357 }
358};
359
360
361/// Gets an array of identifiers consisting of the source and the parents chain.
362template <typename T, typename Transform, typename DictGetter, typename Name>
363class FunctionHierarchyWithDictionary : public IFunction
364{
365public:
366 static constexpr auto name = Name::name;
367 using base_type = FunctionHierarchyWithDictionary;
368
369private:
370 const std::shared_ptr<typename DictGetter::Src> owned_dict;
371
372public:
373 FunctionHierarchyWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
374 : owned_dict(owned_dict_)
375 {
376 if (!owned_dict)
377 throw Exception("Embedded dictionaries were not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
378 }
379
380 String getName() const override
381 {
382 return name;
383 }
384
385 bool isVariadic() const override { return true; }
386 size_t getNumberOfArguments() const override { return 0; }
387
388 DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
389 {
390 if (arguments.size() != 1 && arguments.size() != 2)
391 throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
392 + toString(arguments.size()) + ", should be 1 or 2.",
393 ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
394
395 if (arguments[0]->getName() != TypeName<T>::get())
396 throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName()
397 + " (must be " + String(TypeName<T>::get()) + ")",
398 ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
399
400 if (arguments.size() == 2 && arguments[1]->getName() != TypeName<String>::get())
401 throw Exception("Illegal type " + arguments[1]->getName() + " of the second ('point of view') argument of function " + getName()
402 + " (must be " + String(TypeName<String>::get()) + ")",
403 ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
404
405 return std::make_shared<DataTypeArray>(arguments[0]);
406 }
407
408 bool useDefaultImplementationForConstants() const override { return true; }
409 ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
410
411 bool isDeterministic() const override { return false; }
412
413 void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
414 {
415 /// The dictionary key that defines the "point of view".
416 std::string dict_key;
417
418 if (arguments.size() == 2)
419 {
420 const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
421
422 if (!key_col)
423 throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
424 + " of second ('point of view') argument of function " + name
425 + ". Must be constant string.",
426 ErrorCodes::ILLEGAL_COLUMN);
427
428 dict_key = key_col->getValue<String>();
429 }
430
431 const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key);
432
433 if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
434 {
435 auto col_values = ColumnVector<T>::create();
436 auto col_offsets = ColumnArray::ColumnOffsets::create();
437
438 auto & res_offsets = col_offsets->getData();
439 auto & res_values = col_values->getData();
440
441 const typename ColumnVector<T>::Container & vec_from = col_from->getData();
442 size_t size = vec_from.size();
443 res_offsets.resize(size);
444 res_values.reserve(size * 4);
445
446 for (size_t i = 0; i < size; ++i)
447 {
448 T cur = vec_from[i];
449 while (cur)
450 {
451 res_values.push_back(cur);
452 cur = Transform::toParent(cur, dict);
453 }
454 res_offsets[i] = res_values.size();
455 }
456
457 block.getByPosition(result).column = ColumnArray::create(std::move(col_values), std::move(col_offsets));
458 }
459 else
460 throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
461 + " of first argument of function " + name,
462 ErrorCodes::ILLEGAL_COLUMN);
463 }
464};
465
466
467struct NameRegionToCity { static constexpr auto name = "regionToCity"; };
468struct NameRegionToArea { static constexpr auto name = "regionToArea"; };
469struct NameRegionToDistrict { static constexpr auto name = "regionToDistrict"; };
470struct NameRegionToCountry { static constexpr auto name = "regionToCountry"; };
471struct NameRegionToContinent { static constexpr auto name = "regionToContinent"; };
472struct NameRegionToTopContinent { static constexpr auto name = "regionToTopContinent"; };
473struct NameRegionToPopulation { static constexpr auto name = "regionToPopulation"; };
474struct NameRegionHierarchy { static constexpr auto name = "regionHierarchy"; };
475struct NameRegionIn { static constexpr auto name = "regionIn"; };
476
477
478struct FunctionRegionToCity :
479 public FunctionTransformWithDictionary<UInt32, RegionToCityImpl, RegionsHierarchyGetter, NameRegionToCity>
480{
481 static FunctionPtr create(const Context & context)
482 {
483 return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
484 }
485};
486
487struct FunctionRegionToArea :
488 public FunctionTransformWithDictionary<UInt32, RegionToAreaImpl, RegionsHierarchyGetter, NameRegionToArea>
489{
490 static FunctionPtr create(const Context & context)
491 {
492 return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
493 }
494};
495
496struct FunctionRegionToDistrict :
497 public FunctionTransformWithDictionary<UInt32, RegionToDistrictImpl, RegionsHierarchyGetter, NameRegionToDistrict>
498{
499 static FunctionPtr create(const Context & context)
500 {
501 return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
502 }
503};
504
505struct FunctionRegionToCountry :
506 public FunctionTransformWithDictionary<UInt32, RegionToCountryImpl, RegionsHierarchyGetter, NameRegionToCountry>
507{
508 static FunctionPtr create(const Context & context)
509 {
510 return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
511 }
512};
513
514struct FunctionRegionToContinent :
515 public FunctionTransformWithDictionary<UInt32, RegionToContinentImpl, RegionsHierarchyGetter, NameRegionToContinent>
516{
517 static FunctionPtr create(const Context & context)
518 {
519 return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
520 }
521};
522
523struct FunctionRegionToTopContinent :
524 public FunctionTransformWithDictionary<UInt32, RegionToTopContinentImpl, RegionsHierarchyGetter, NameRegionToTopContinent>
525{
526 static FunctionPtr create(const Context & context)
527 {
528 return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
529 }
530};
531
532struct FunctionRegionToPopulation :
533 public FunctionTransformWithDictionary<UInt32, RegionToPopulationImpl, RegionsHierarchyGetter, NameRegionToPopulation>
534{
535 static FunctionPtr create(const Context & context)
536 {
537 return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
538 }
539};
540
541struct FunctionRegionIn :
542 public FunctionIsInWithDictionary<UInt32, RegionInImpl, RegionsHierarchyGetter, NameRegionIn>
543{
544 static FunctionPtr create(const Context & context)
545 {
546 return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
547 }
548};
549
550struct FunctionRegionHierarchy :
551 public FunctionHierarchyWithDictionary<UInt32, RegionHierarchyImpl, RegionsHierarchyGetter, NameRegionHierarchy>
552{
553 static FunctionPtr create(const Context & context)
554 {
555 return std::make_shared<base_type>(context.getEmbeddedDictionaries().getRegionsHierarchies());
556 }
557};
558
559
560/// Converts a region's numeric identifier to a name in the specified language using a dictionary.
561class FunctionRegionToName : public IFunction
562{
563public:
564 static constexpr auto name = "regionToName";
565 static FunctionPtr create(const Context & context)
566 {
567 return std::make_shared<FunctionRegionToName>(context.getEmbeddedDictionaries().getRegionsNames());
568 }
569
570private:
571 const MultiVersion<RegionsNames>::Version owned_dict;
572
573public:
574 FunctionRegionToName(const MultiVersion<RegionsNames>::Version & owned_dict_)
575 : owned_dict(owned_dict_)
576 {
577 if (!owned_dict)
578 throw Exception("Embedded dictionaries were not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
579 }
580
581 String getName() const override
582 {
583 return name;
584 }
585
586 bool isVariadic() const override { return true; }
587 size_t getNumberOfArguments() const override { return 0; }
588
589 /// For the purpose of query optimization, we assume this function to be injective
590 /// even in face of fact that there are many different cities named Moscow.
591 bool isInjective(const Block &) override { return true; }
592
593 DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
594 {
595 if (arguments.size() != 1 && arguments.size() != 2)
596 throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
597 + toString(arguments.size()) + ", should be 1 or 2.",
598 ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
599
600 if (arguments[0]->getName() != TypeName<UInt32>::get())
601 throw Exception("Illegal type " + arguments[0]->getName() + " of the first argument of function " + getName()
602 + " (must be " + String(TypeName<UInt32>::get()) + ")",
603 ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
604
605 if (arguments.size() == 2 && arguments[1]->getName() != TypeName<String>::get())
606 throw Exception("Illegal type " + arguments[0]->getName() + " of the second argument of function " + getName()
607 + " (must be " + String(TypeName<String>::get()) + ")",
608 ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
609
610 return std::make_shared<DataTypeString>();
611 }
612
613 bool useDefaultImplementationForConstants() const override { return true; }
614 ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
615
616 bool isDeterministic() const override { return false; }
617
618 void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
619 {
620 RegionsNames::Language language = RegionsNames::Language::ru;
621
622 /// If the result language is specified
623 if (arguments.size() == 2)
624 {
625 if (const ColumnConst * col_language = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()))
626 language = RegionsNames::getLanguageEnum(col_language->getValue<String>());
627 else
628 throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
629 + " of the second argument of function " + getName(),
630 ErrorCodes::ILLEGAL_COLUMN);
631 }
632
633 const RegionsNames & dict = *owned_dict;
634
635 if (const ColumnUInt32 * col_from = typeid_cast<const ColumnUInt32 *>(block.getByPosition(arguments[0]).column.get()))
636 {
637 auto col_to = ColumnString::create();
638
639 const ColumnUInt32::Container & region_ids = col_from->getData();
640
641 for (size_t i = 0; i < region_ids.size(); ++i)
642 {
643 const StringRef & name_ref = dict.getRegionName(region_ids[i], language);
644 col_to->insertDataWithTerminatingZero(name_ref.data, name_ref.size + 1);
645 }
646
647 block.getByPosition(result).column = std::move(col_to);
648 }
649 else
650 throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
651 + " of the first argument of function " + getName(),
652 ErrorCodes::ILLEGAL_COLUMN);
653 }
654};
655
656}
657