1#pragma once
2
3#include <memory>
4#include <Common/COW.h>
5#include <boost/noncopyable.hpp>
6#include <DataTypes/DataTypeCustom.h>
7
8
9namespace DB
10{
11
12class ReadBuffer;
13class WriteBuffer;
14
15class IDataType;
16struct FormatSettings;
17
18class IColumn;
19using ColumnPtr = COW<IColumn>::Ptr;
20using MutableColumnPtr = COW<IColumn>::MutablePtr;
21
22class Field;
23
24using DataTypePtr = std::shared_ptr<const IDataType>;
25using DataTypes = std::vector<DataTypePtr>;
26
27class ProtobufReader;
28class ProtobufWriter;
29
30
31/** Properties of data type.
32 * Contains methods for serialization/deserialization.
33 * Implementations of this interface represent a data type (example: UInt8)
34 * or parametric family of data types (example: Array(...)).
35 *
36 * DataType is totally immutable object. You can always share them.
37 */
38class IDataType : private boost::noncopyable
39{
40public:
41 IDataType();
42 virtual ~IDataType();
43
44 /// Compile time flag. If false, then if C++ types are the same, then SQL types are also the same.
45 /// Example: DataTypeString is not parametric: thus all instances of DataTypeString are the same SQL type.
46 /// Example: DataTypeFixedString is parametric: different instances of DataTypeFixedString may be different SQL types.
47 /// Place it in descendants:
48 /// static constexpr bool is_parametric = false;
49
50 /// Name of data type (examples: UInt64, Array(String)).
51 String getName() const;
52
53 /// Name of data type family (example: FixedString, Array).
54 virtual const char * getFamilyName() const = 0;
55
56 /// Data type id. It's used for runtime type checks.
57 virtual TypeIndex getTypeId() const = 0;
58
59 /** Binary serialization for range of values in column - for writing to disk/network, etc.
60 *
61 * Some data types are represented in multiple streams while being serialized.
62 * Example:
63 * - Arrays are represented as stream of all elements and stream of array sizes.
64 * - Nullable types are represented as stream of values (with unspecified values in place of NULLs) and stream of NULL flags.
65 *
66 * Different streams are identified by "path".
67 * If the data type require single stream (it's true for most of data types), the stream will have empty path.
68 * Otherwise, the path can have components like "array elements", "array sizes", etc.
69 *
70 * For multidimensional arrays, path can have arbiraty length.
71 * As an example, for 2-dimensional arrays of numbers we have at least three streams:
72 * - array sizes; (sizes of top level arrays)
73 * - array elements / array sizes; (sizes of second level (nested) arrays)
74 * - array elements / array elements; (the most deep elements, placed contiguously)
75 *
76 * Descendants must override either serializeBinaryBulk, deserializeBinaryBulk methods (for simple cases with single stream)
77 * or serializeBinaryBulkWithMultipleStreams, deserializeBinaryBulkWithMultipleStreams, enumerateStreams methods (for cases with multiple streams).
78 *
79 * Default implementations of ...WithMultipleStreams methods will call serializeBinaryBulk, deserializeBinaryBulk for single stream.
80 */
81
82 struct Substream
83 {
84 enum Type
85 {
86 ArrayElements,
87 ArraySizes,
88
89 NullableElements,
90 NullMap,
91
92 TupleElement,
93
94 DictionaryKeys,
95 DictionaryIndexes,
96 };
97 Type type;
98
99 /// Index of tuple element, starting at 1.
100 String tuple_element_name;
101
102 Substream(Type type_) : type(type_) {}
103 };
104
105 using SubstreamPath = std::vector<Substream>;
106
107 using StreamCallback = std::function<void(const SubstreamPath &)>;
108 virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
109 {
110 callback(path);
111 }
112 void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); }
113 void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); }
114
115 using OutputStreamGetter = std::function<WriteBuffer*(const SubstreamPath &)>;
116 using InputStreamGetter = std::function<ReadBuffer*(const SubstreamPath &)>;
117
118 struct SerializeBinaryBulkState
119 {
120 virtual ~SerializeBinaryBulkState() = default;
121 };
122 struct DeserializeBinaryBulkState
123 {
124 virtual ~DeserializeBinaryBulkState() = default;
125 };
126
127 using SerializeBinaryBulkStatePtr = std::shared_ptr<SerializeBinaryBulkState>;
128 using DeserializeBinaryBulkStatePtr = std::shared_ptr<DeserializeBinaryBulkState>;
129
130 struct SerializeBinaryBulkSettings
131 {
132 OutputStreamGetter getter;
133 SubstreamPath path;
134
135 size_t low_cardinality_max_dictionary_size = 0;
136 bool low_cardinality_use_single_dictionary_for_part = true;
137
138 bool position_independent_encoding = true;
139 };
140
141 struct DeserializeBinaryBulkSettings
142 {
143 InputStreamGetter getter;
144 SubstreamPath path;
145
146 /// True if continue reading from previous positions in file. False if made fseek to the start of new granule.
147 bool continuous_reading = true;
148
149 bool position_independent_encoding = true;
150 /// If not zero, may be used to avoid reallocations while reading column of String type.
151 double avg_value_size_hint = 0;
152 };
153
154 /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark.
155 virtual void serializeBinaryBulkStatePrefix(
156 SerializeBinaryBulkSettings & /*settings*/,
157 SerializeBinaryBulkStatePtr & /*state*/) const {}
158
159 /// Call after serializeBinaryBulkWithMultipleStreams chain to finish serialization.
160 virtual void serializeBinaryBulkStateSuffix(
161 SerializeBinaryBulkSettings & /*settings*/,
162 SerializeBinaryBulkStatePtr & /*state*/) const {}
163
164 /// Call before before deserializeBinaryBulkWithMultipleStreams chain to get DeserializeBinaryBulkStatePtr.
165 virtual void deserializeBinaryBulkStatePrefix(
166 DeserializeBinaryBulkSettings & /*settings*/,
167 DeserializeBinaryBulkStatePtr & /*state*/) const {}
168
169 /** 'offset' and 'limit' are used to specify range.
170 * limit = 0 - means no limit.
171 * offset must be not greater than size of column.
172 * offset + limit could be greater than size of column
173 * - in that case, column is serialized till the end.
174 */
175 virtual void serializeBinaryBulkWithMultipleStreams(
176 const IColumn & column,
177 size_t offset,
178 size_t limit,
179 SerializeBinaryBulkSettings & settings,
180 SerializeBinaryBulkStatePtr & /*state*/) const
181 {
182 if (WriteBuffer * stream = settings.getter(settings.path))
183 serializeBinaryBulk(column, *stream, offset, limit);
184 }
185
186 /// Read no more than limit values and append them into column.
187 virtual void deserializeBinaryBulkWithMultipleStreams(
188 IColumn & column,
189 size_t limit,
190 DeserializeBinaryBulkSettings & settings,
191 DeserializeBinaryBulkStatePtr & /*state*/) const
192 {
193 if (ReadBuffer * stream = settings.getter(settings.path))
194 deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint);
195 }
196
197 /** Override these methods for data types that require just single stream (most of data types).
198 */
199 virtual void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const;
200 virtual void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const;
201
202 /** Serialization/deserialization of individual values.
203 *
204 * These are helper methods for implementation of various formats to input/output for user (like CSV, JSON, etc.).
205 * There is no one-to-one correspondence between formats and these methods.
206 * For example, TabSeparated and Pretty formats could use same helper method serializeTextEscaped.
207 *
208 * For complex data types (like arrays) binary serde for individual values may differ from bulk serde.
209 * For example, if you serialize single array, it will be represented as its size and elements in single contiguous stream,
210 * but if you bulk serialize column with arrays, then sizes and elements will be written to separate streams.
211 */
212
213 /// There is two variants for binary serde. First variant work with Field.
214 virtual void serializeBinary(const Field & field, WriteBuffer & ostr) const = 0;
215 virtual void deserializeBinary(Field & field, ReadBuffer & istr) const = 0;
216
217 /// Other variants takes a column, to avoid creating temporary Field object.
218 /// Column must be non-constant.
219
220 /// Serialize one value of a column at specified row number.
221 virtual void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0;
222 /// Deserialize one value and insert into a column.
223 /// If method will throw an exception, then column will be in same state as before call to method.
224 virtual void deserializeBinary(IColumn & column, ReadBuffer & istr) const = 0;
225
226 /** Serialize to a protobuf. */
227 virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const = 0;
228 virtual void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const = 0;
229
230 /** Text serialization with escaping but without quoting.
231 */
232 void serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
233
234 void deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
235
236 /** Text serialization as a literal that may be inserted into a query.
237 */
238 void serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
239
240 void deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
241
242 /** Text serialization for the CSV format.
243 */
244 void serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
245 void deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
246
247 /** Text serialization for displaying on a terminal or saving into a text file, and the like.
248 * Without escaping or quoting.
249 */
250 void serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
251
252 /** Text deserialization in case when buffer contains only one value, without any escaping and delimiters.
253 */
254 void deserializeAsWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
255
256 /** Text serialization intended for using in JSON format.
257 */
258 void serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
259 void deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
260
261 /** Text serialization for putting into the XML format.
262 */
263 void serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
264
265protected:
266 virtual String doGetName() const;
267
268 /// Default implementations of text serialization in case of 'custom_text_serialization' is not set.
269
270 virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
271 virtual void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
272 virtual void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
273 virtual void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
274 virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
275 virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
276 virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
277 virtual void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
278 virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
279 virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
280 virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
281 {
282 serializeText(column, row_num, ostr, settings);
283 }
284
285public:
286 /** Create empty column for corresponding type.
287 */
288 virtual MutableColumnPtr createColumn() const = 0;
289
290 /** Create ColumnConst for corresponding type, with specified size and value.
291 */
292 ColumnPtr createColumnConst(size_t size, const Field & field) const;
293 ColumnPtr createColumnConstWithDefaultValue(size_t size) const;
294
295 /** Get default value of data type.
296 * It is the "default" default, regardless the fact that a table could contain different user-specified default.
297 */
298 virtual Field getDefault() const = 0;
299
300 /** The data type can be promoted in order to try to avoid overflows.
301 * Data types which can be promoted are typically Number or Decimal data types.
302 */
303 virtual bool canBePromoted() const { return false; }
304
305 /** Return the promoted numeric data type of the current data type. Throw an exception if `canBePromoted() == false`.
306 */
307 virtual DataTypePtr promoteNumericType() const;
308
309 /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault.
310 * This should be overriden if data type default value differs from column default value (example: Enum data types).
311 */
312 virtual void insertDefaultInto(IColumn & column) const;
313
314 /// Checks that two instances belong to the same type
315 virtual bool equals(const IDataType & rhs) const = 0;
316
317
318 /// Various properties on behaviour of data type.
319
320 /** The data type is dependent on parameters and types with different parameters are different.
321 * Examples: FixedString(N), Tuple(T1, T2), Nullable(T).
322 * Otherwise all instances of the same class are the same types.
323 */
324 virtual bool isParametric() const = 0;
325
326 /** The data type is dependent on parameters and at least one of them is another type.
327 * Examples: Tuple(T1, T2), Nullable(T). But FixedString(N) is not.
328 */
329 virtual bool haveSubtypes() const = 0;
330
331 /** Can appear in table definition.
332 * Counterexamples: Interval, Nothing.
333 */
334 virtual bool cannotBeStoredInTables() const { return false; }
335
336 /** In text formats that render "pretty" tables,
337 * is it better to align value right in table cell.
338 * Examples: numbers, even nullable.
339 */
340 virtual bool shouldAlignRightInPrettyFormats() const { return false; }
341
342 /** Does formatted value in any text format can contain anything but valid UTF8 sequences.
343 * Example: String (because it can contain arbitrary bytes).
344 * Counterexamples: numbers, Date, DateTime.
345 * For Enum, it depends.
346 */
347 virtual bool textCanContainOnlyValidUTF8() const { return false; }
348
349 /** Is it possible to compare for less/greater, to calculate min/max?
350 * Not necessarily totally comparable. For example, floats are comparable despite the fact that NaNs compares to nothing.
351 * The same for nullable of comparable types: they are comparable (but not totally-comparable).
352 */
353 virtual bool isComparable() const { return false; }
354
355 /** Does it make sense to use this type with COLLATE modifier in ORDER BY.
356 * Example: String, but not FixedString.
357 */
358 virtual bool canBeComparedWithCollation() const { return false; }
359
360 /** If the type is totally comparable (Ints, Date, DateTime, not nullable, not floats)
361 * and "simple" enough (not String, FixedString) to be used as version number
362 * (to select rows with maximum version).
363 */
364 virtual bool canBeUsedAsVersion() const { return false; }
365
366 /** Values of data type can be summed (possibly with overflow, within the same data type).
367 * Example: numbers, even nullable. Not Date/DateTime. Not Enum.
368 * Enums can be passed to aggregate function 'sum', but the result is Int64, not Enum, so they are not summable.
369 */
370 virtual bool isSummable() const { return false; }
371
372 /** Can be used in operations like bit and, bit shift, bit not, etc.
373 */
374 virtual bool canBeUsedInBitOperations() const { return false; }
375
376 /** Can be used in boolean context (WHERE, HAVING).
377 * UInt8, maybe nullable.
378 */
379 virtual bool canBeUsedInBooleanContext() const { return false; }
380
381 /** Numbers, Enums, Date, DateTime. Not nullable.
382 */
383 virtual bool isValueRepresentedByNumber() const { return false; }
384
385 /** Integers, Enums, Date, DateTime. Not nullable.
386 */
387 virtual bool isValueRepresentedByInteger() const { return false; }
388
389 /** Unsigned Integers, Date, DateTime. Not nullable.
390 */
391 virtual bool isValueRepresentedByUnsignedInteger() const { return false; }
392
393 /** Values are unambiguously identified by contents of contiguous memory region,
394 * that can be obtained by IColumn::getDataAt method.
395 * Examples: numbers, Date, DateTime, String, FixedString,
396 * and Arrays of numbers, Date, DateTime, FixedString, Enum, but not String.
397 * (because Array(String) values became ambiguous if you concatenate Strings).
398 * Counterexamples: Nullable, Tuple.
399 */
400 virtual bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const { return false; }
401
402 virtual bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const
403 {
404 return isValueRepresentedByNumber();
405 }
406
407 /** Example: numbers, Date, DateTime, FixedString, Enum... Nullable and Tuple of such types.
408 * Counterexamples: String, Array.
409 * It's Ok to return false for AggregateFunction despite the fact that some of them have fixed size state.
410 */
411 virtual bool haveMaximumSizeOfValue() const { return false; }
412
413 /** Size in amount of bytes in memory. Throws an exception if not haveMaximumSizeOfValue.
414 */
415 virtual size_t getMaximumSizeOfValueInMemory() const { return getSizeOfValueInMemory(); }
416
417 /** Throws an exception if value is not of fixed size.
418 */
419 virtual size_t getSizeOfValueInMemory() const;
420
421 /** Integers (not floats), Enum, String, FixedString.
422 */
423 virtual bool isCategorial() const { return false; }
424
425 virtual bool isNullable() const { return false; }
426
427 /** Is this type can represent only NULL value? (It also implies isNullable)
428 */
429 virtual bool onlyNull() const { return false; }
430
431 /** If this data type cannot be wrapped in Nullable data type.
432 */
433 virtual bool canBeInsideNullable() const { return false; }
434
435 virtual bool lowCardinality() const { return false; }
436
437 /// Strings, Numbers, Date, DateTime, Nullable
438 virtual bool canBeInsideLowCardinality() const { return false; }
439
440 /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
441 static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint);
442
443 static String getFileNameForStream(const String & column_name, const SubstreamPath & path);
444
445private:
446 friend class DataTypeFactory;
447 /** Customize this DataType
448 */
449 void setCustomization(DataTypeCustomDescPtr custom_desc_) const;
450
451private:
452 /** This is mutable to allow setting custom name and serialization on `const IDataType` post construction.
453 */
454 mutable DataTypeCustomNamePtr custom_name;
455 mutable DataTypeCustomTextSerializationPtr custom_text_serialization;
456
457public:
458 const IDataTypeCustomName * getCustomName() const { return custom_name.get(); }
459};
460
461
462/// Some sugar to check data type of IDataType
463struct WhichDataType
464{
465 TypeIndex idx;
466
467 WhichDataType(TypeIndex idx_ = TypeIndex::Nothing)
468 : idx(idx_)
469 {}
470
471 WhichDataType(const IDataType & data_type)
472 : idx(data_type.getTypeId())
473 {}
474
475 WhichDataType(const IDataType * data_type)
476 : idx(data_type->getTypeId())
477 {}
478
479 WhichDataType(const DataTypePtr & data_type)
480 : idx(data_type->getTypeId())
481 {}
482
483 bool isUInt8() const { return idx == TypeIndex::UInt8; }
484 bool isUInt16() const { return idx == TypeIndex::UInt16; }
485 bool isUInt32() const { return idx == TypeIndex::UInt32; }
486 bool isUInt64() const { return idx == TypeIndex::UInt64; }
487 bool isUInt128() const { return idx == TypeIndex::UInt128; }
488 bool isUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64() || isUInt128(); }
489 bool isNativeUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64(); }
490
491 bool isInt8() const { return idx == TypeIndex::Int8; }
492 bool isInt16() const { return idx == TypeIndex::Int16; }
493 bool isInt32() const { return idx == TypeIndex::Int32; }
494 bool isInt64() const { return idx == TypeIndex::Int64; }
495 bool isInt128() const { return idx == TypeIndex::Int128; }
496 bool isInt() const { return isInt8() || isInt16() || isInt32() || isInt64() || isInt128(); }
497 bool isNativeInt() const { return isInt8() || isInt16() || isInt32() || isInt64(); }
498
499 bool isDecimal32() const { return idx == TypeIndex::Decimal32; }
500 bool isDecimal64() const { return idx == TypeIndex::Decimal64; }
501 bool isDecimal128() const { return idx == TypeIndex::Decimal128; }
502 bool isDecimal() const { return isDecimal32() || isDecimal64() || isDecimal128(); }
503
504 bool isFloat32() const { return idx == TypeIndex::Float32; }
505 bool isFloat64() const { return idx == TypeIndex::Float64; }
506 bool isFloat() const { return isFloat32() || isFloat64(); }
507
508 bool isEnum8() const { return idx == TypeIndex::Enum8; }
509 bool isEnum16() const { return idx == TypeIndex::Enum16; }
510 bool isEnum() const { return isEnum8() || isEnum16(); }
511
512 bool isDate() const { return idx == TypeIndex::Date; }
513 bool isDateTime() const { return idx == TypeIndex::DateTime; }
514 bool isDateTime64() const { return idx == TypeIndex::DateTime64; }
515 bool isDateOrDateTime() const { return isDate() || isDateTime() || isDateTime64(); }
516
517 bool isString() const { return idx == TypeIndex::String; }
518 bool isFixedString() const { return idx == TypeIndex::FixedString; }
519 bool isStringOrFixedString() const { return isString() || isFixedString(); }
520
521 bool isUUID() const { return idx == TypeIndex::UUID; }
522 bool isArray() const { return idx == TypeIndex::Array; }
523 bool isTuple() const { return idx == TypeIndex::Tuple; }
524 bool isSet() const { return idx == TypeIndex::Set; }
525 bool isInterval() const { return idx == TypeIndex::Interval; }
526
527 bool isNothing() const { return idx == TypeIndex::Nothing; }
528 bool isNullable() const { return idx == TypeIndex::Nullable; }
529 bool isFunction() const { return idx == TypeIndex::Function; }
530 bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; }
531};
532
533/// IDataType helpers (alternative for IDataType virtual methods with single point of truth)
534
535inline bool isDate(const DataTypePtr & data_type) { return WhichDataType(data_type).isDate(); }
536inline bool isDateOrDateTime(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateOrDateTime(); }
537inline bool isDateTime(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateTime(); }
538inline bool isDateTime64(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateTime64(); }
539inline bool isEnum(const DataTypePtr & data_type) { return WhichDataType(data_type).isEnum(); }
540inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data_type).isDecimal(); }
541inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); }
542inline bool isArray(const DataTypePtr & data_type) { return WhichDataType(data_type).isArray(); }
543
544template <typename T>
545inline bool isUInt8(const T & data_type)
546{
547 return WhichDataType(data_type).isUInt8();
548}
549
550template <typename T>
551inline bool isUnsignedInteger(const T & data_type)
552{
553 return WhichDataType(data_type).isUInt();
554}
555
556template <typename T>
557inline bool isInteger(const T & data_type)
558{
559 WhichDataType which(data_type);
560 return which.isInt() || which.isUInt();
561}
562
563template <typename T>
564inline bool isFloat(const T & data_type)
565{
566 WhichDataType which(data_type);
567 return which.isFloat();
568}
569
570template <typename T>
571inline bool isNativeInteger(const T & data_type)
572{
573 WhichDataType which(data_type);
574 return which.isNativeInt() || which.isNativeUInt();
575}
576
577
578template <typename T>
579inline bool isNativeNumber(const T & data_type)
580{
581 WhichDataType which(data_type);
582 return which.isNativeInt() || which.isNativeUInt() || which.isFloat();
583}
584
585template <typename T>
586inline bool isNumber(const T & data_type)
587{
588 WhichDataType which(data_type);
589 return which.isInt() || which.isUInt() || which.isFloat() || which.isDecimal();
590}
591
592template <typename T>
593inline bool isColumnedAsNumber(const T & data_type)
594{
595 WhichDataType which(data_type);
596 return which.isInt() || which.isUInt() || which.isFloat() || which.isDateOrDateTime() || which.isUUID();
597}
598
599template <typename T>
600inline bool isColumnedAsDecimal(const T & data_type)
601{
602 WhichDataType which(data_type);
603 return which.isDecimal() || which.isDateTime64();
604}
605
606template <typename T>
607inline bool isString(const T & data_type)
608{
609 return WhichDataType(data_type).isString();
610}
611
612template <typename T>
613inline bool isFixedString(const T & data_type)
614{
615 return WhichDataType(data_type).isFixedString();
616}
617
618template <typename T>
619inline bool isStringOrFixedString(const T & data_type)
620{
621 return WhichDataType(data_type).isStringOrFixedString();
622}
623
624
625inline bool isNotDecimalButComparableToDecimal(const DataTypePtr & data_type)
626{
627 WhichDataType which(data_type);
628 return which.isInt() || which.isUInt();
629}
630
631inline bool isCompilableType(const DataTypePtr & data_type)
632{
633 return data_type->isValueRepresentedByNumber() && !isDecimal(data_type);
634}
635
636template <typename DataType> constexpr bool IsDataTypeDecimal = false;
637template <typename DataType> constexpr bool IsDataTypeNumber = false;
638template <typename DataType> constexpr bool IsDataTypeDateOrDateTime = false;
639
640template <typename DataType> constexpr bool IsDataTypeDecimalOrNumber = IsDataTypeDecimal<DataType> || IsDataTypeNumber<DataType>;
641
642template <typename T>
643class DataTypeDecimal;
644
645template <typename T>
646class DataTypeNumber;
647
648class DataTypeDate;
649class DataTypeDateTime;
650class DataTypeDateTime64;
651
652template <typename T> constexpr bool IsDataTypeDecimal<DataTypeDecimal<T>> = true;
653template <> inline constexpr bool IsDataTypeDecimal<DataTypeDateTime64> = true;
654
655template <typename T> constexpr bool IsDataTypeNumber<DataTypeNumber<T>> = true;
656
657template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDate> = true;
658template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime> = true;
659template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime64> = true;
660
661}
662
663