mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
merge remote-tracking branch 'origin' into integration-6
This commit is contained in:
commit
2ae7d72dc9
@ -18,6 +18,8 @@
|
||||
|
||||
#define DATE_LUT_MAX (0xFFFFFFFFU - 86400)
|
||||
#define DATE_LUT_MAX_DAY_NUM 0xFFFF
|
||||
/// Max int value of Date32, DATE LUT cache size minus daynum_offset_epoch
|
||||
#define DATE_LUT_MAX_EXTEND_DAY_NUM (DATE_LUT_SIZE - 16436)
|
||||
|
||||
/// A constant to add to time_t so every supported time point becomes non-negative and still has the same remainder of division by 3600.
|
||||
/// If we treat "remainder of division" operation in the sense of modular arithmetic (not like in C++).
|
||||
@ -270,6 +272,8 @@ public:
|
||||
auto getOffsetAtStartOfEpoch() const { return offset_at_start_of_epoch; }
|
||||
auto getTimeOffsetAtStartOfLUT() const { return offset_at_start_of_lut; }
|
||||
|
||||
auto getDayNumOffsetEpoch() const { return daynum_offset_epoch; }
|
||||
|
||||
/// All functions below are thread-safe; arguments are not checked.
|
||||
|
||||
inline ExtendedDayNum toDayNum(ExtendedDayNum d) const
|
||||
@ -926,15 +930,17 @@ public:
|
||||
{
|
||||
if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
|
||||
return LUTIndex(0);
|
||||
|
||||
return LUTIndex{years_months_lut[(year - DATE_LUT_MIN_YEAR) * 12 + month - 1] + day_of_month - 1};
|
||||
auto year_lut_index = (year - DATE_LUT_MIN_YEAR) * 12 + month - 1;
|
||||
UInt32 index = years_months_lut[year_lut_index].toUnderType() + day_of_month - 1;
|
||||
/// When date is out of range, default value is DATE_LUT_SIZE - 1 (2283-11-11)
|
||||
return LUTIndex{std::min(index, static_cast<UInt32>(DATE_LUT_SIZE - 1))};
|
||||
}
|
||||
|
||||
/// Create DayNum from year, month, day of month.
|
||||
inline ExtendedDayNum makeDayNum(Int16 year, UInt8 month, UInt8 day_of_month) const
|
||||
inline ExtendedDayNum makeDayNum(Int16 year, UInt8 month, UInt8 day_of_month, Int32 default_error_day_num = 0) const
|
||||
{
|
||||
if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
|
||||
return ExtendedDayNum(0);
|
||||
return ExtendedDayNum(default_error_day_num);
|
||||
|
||||
return toDayNum(makeLUTIndex(year, month, day_of_month));
|
||||
}
|
||||
@ -1091,9 +1097,9 @@ public:
|
||||
return lut[new_index].date + time;
|
||||
}
|
||||
|
||||
inline NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int64 delta) const
|
||||
inline NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int32 delta) const
|
||||
{
|
||||
return addDays(t, delta * 7);
|
||||
return addDays(t, static_cast<Int64>(delta) * 7);
|
||||
}
|
||||
|
||||
inline UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const
|
||||
@ -1158,14 +1164,14 @@ public:
|
||||
return toDayNum(addMonthsIndex(d, delta));
|
||||
}
|
||||
|
||||
inline Time NO_SANITIZE_UNDEFINED addQuarters(Time t, Int64 delta) const
|
||||
inline Time NO_SANITIZE_UNDEFINED addQuarters(Time t, Int32 delta) const
|
||||
{
|
||||
return addMonths(t, delta * 3);
|
||||
return addMonths(t, static_cast<Int64>(delta) * 3);
|
||||
}
|
||||
|
||||
inline ExtendedDayNum addQuarters(ExtendedDayNum d, Int64 delta) const
|
||||
inline ExtendedDayNum addQuarters(ExtendedDayNum d, Int32 delta) const
|
||||
{
|
||||
return addMonths(d, delta * 3);
|
||||
return addMonths(d, static_cast<Int64>(delta) * 3);
|
||||
}
|
||||
|
||||
template <typename DateOrTime>
|
||||
|
@ -70,6 +70,14 @@ public:
|
||||
m_day = values.day_of_month;
|
||||
}
|
||||
|
||||
explicit LocalDate(ExtendedDayNum day_num)
|
||||
{
|
||||
const auto & values = DateLUT::instance().getValues(day_num);
|
||||
m_year = values.year;
|
||||
m_month = values.month;
|
||||
m_day = values.day_of_month;
|
||||
}
|
||||
|
||||
LocalDate(unsigned short year_, unsigned char month_, unsigned char day_)
|
||||
: m_year(year_), m_month(month_), m_day(day_)
|
||||
{
|
||||
@ -98,6 +106,12 @@ public:
|
||||
return DayNum(lut.makeDayNum(m_year, m_month, m_day).toUnderType());
|
||||
}
|
||||
|
||||
ExtendedDayNum getExtenedDayNum() const
|
||||
{
|
||||
const auto & lut = DateLUT::instance();
|
||||
return ExtendedDayNum (lut.makeDayNum(m_year, m_month, m_day).toUnderType());
|
||||
}
|
||||
|
||||
operator DayNum() const
|
||||
{
|
||||
return getDayNum();
|
||||
|
2
contrib/rocksdb
vendored
2
contrib/rocksdb
vendored
@ -1 +1 @@
|
||||
Subproject commit 6ff0adefdc84dac44e78804f7ca4122fe992cf8d
|
||||
Subproject commit dac0e9a68080c837d6b6223921f3fc151abbfcdc
|
@ -70,11 +70,6 @@ else()
|
||||
endif()
|
||||
endif()
|
||||
|
||||
set(BUILD_VERSION_CC rocksdb_build_version.cc)
|
||||
add_library(rocksdb_build_version OBJECT ${BUILD_VERSION_CC})
|
||||
|
||||
target_include_directories(rocksdb_build_version PRIVATE "${ROCKSDB_SOURCE_DIR}/util")
|
||||
|
||||
include(CheckCCompilerFlag)
|
||||
if(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64")
|
||||
CHECK_C_COMPILER_FLAG("-mcpu=power9" HAS_POWER9)
|
||||
@ -243,272 +238,293 @@ find_package(Threads REQUIRED)
|
||||
# Main library source code
|
||||
|
||||
set(SOURCES
|
||||
"${ROCKSDB_SOURCE_DIR}/cache/cache.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/cache/clock_cache.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/cache/lru_cache.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/arena_wrapped_db_iter.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_addition.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_builder.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_cache.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_garbage.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_meta.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_format.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_sequential_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_writer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/builder.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/c.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/column_family.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/compacted_db_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/compaction/compaction.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_iterator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_job.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_fifo.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_level.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_universal.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/compaction/sst_partitioner.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/convenience.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_filesnapshot.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_write.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_compaction_flush.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_files.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_open.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_debug.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_experimental.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_readonly.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_secondary.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_info_dumper.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/db_iter.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/dbformat.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/error_handler.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/event_helpers.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/experimental.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/external_sst_file_ingestion_job.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/file_indexer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/flush_job.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/flush_scheduler.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/forward_iterator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/import_column_family_job.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/internal_stats.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/logs_with_prep_tracker.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/log_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/log_writer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/malloc_stats.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/memtable.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/memtable_list.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/merge_helper.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/merge_operator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/output_validator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/periodic_work_scheduler.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/range_del_aggregator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/range_tombstone_fragmenter.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/repair.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/snapshot_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/table_cache.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/table_properties_collector.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/transaction_log_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/trim_history_scheduler.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/version_builder.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/version_edit.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/version_edit_handler.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/version_set.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/wal_edit.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/wal_manager.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/write_batch.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/write_batch_base.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/write_controller.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/db/write_thread.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/env/env.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/env/env_chroot.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/env/env_encryption.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/env/env_hdfs.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/env/file_system.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/env/mock_env.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/delete_scheduler.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/file_prefetch_buffer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/file_util.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/filename.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/random_access_file_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/read_write_util.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/readahead_raf.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/sequence_file_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/sst_file_manager_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/file/writable_file_writer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/logging/auto_roll_logger.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/logging/event_logger.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/logging/log_buffer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memory/arena.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memory/concurrent_arena.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memory/jemalloc_nodump_allocator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memory/memkind_kmem_allocator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memtable/alloc_tracker.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memtable/hash_linklist_rep.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memtable/hash_skiplist_rep.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memtable/skiplistrep.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memtable/vectorrep.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/memtable/write_buffer_manager.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/histogram.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/histogram_windowing.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/in_memory_stats_history.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/instrumented_mutex.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/iostats_context.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/perf_context.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/perf_level.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/persistent_stats_history.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/statistics.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_updater.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_util.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_util_debug.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/options/cf_options.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/options/configurable.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/options/customizable.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/options/db_options.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/options/options.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/options/options_helper.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/options/options_parser.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/port/stack_trace.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/adaptive/adaptive_table_factory.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/binary_search_index_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/block.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_filter_block.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_builder.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_factory.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_iterator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/block_builder.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefetcher.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefix_index.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/data_block_hash_index.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/data_block_footer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/filter_block_reader_common.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/filter_policy.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/flush_block_policy.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/full_filter_block.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/hash_index_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/index_builder.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/index_reader_common.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/parsed_full_filter_block.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/partitioned_filter_block.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/partitioned_index_iterator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/partitioned_index_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/reader_common.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_based/uncompression_dict_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/block_fetcher.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/cuckoo/cuckoo_table_builder.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/cuckoo/cuckoo_table_factory.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/cuckoo/cuckoo_table_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/format.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/get_context.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/iterator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/merging_iterator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/meta_blocks.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/persistent_cache_helper.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_bloom.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_builder.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_factory.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_index.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_key_coding.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/sst_file_dumper.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/sst_file_reader.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/sst_file_writer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/table_factory.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/table_properties.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/table/two_level_iterator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/test_util/sync_point.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/test_util/sync_point_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/test_util/testutil.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/test_util/transaction_test_util.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/tools/block_cache_analyzer/block_cache_trace_analyzer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/tools/dump/db_dump_tool.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/tools/io_tracer_parser_tool.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/tools/ldb_cmd.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/tools/ldb_tool.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/tools/sst_dump_tool.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/tools/trace_analyzer_tool.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/trace_replay/trace_replay.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/trace_replay/block_cache_tracer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/trace_replay/io_tracer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/coding.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/compaction_job_stats_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/comparator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/compression_context_cache.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/concurrent_task_limiter_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/crc32c.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/dynamic_bloom.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/hash.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/murmurhash.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/random.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/rate_limiter.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/slice.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/file_checksum_helper.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/status.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/string_util.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/thread_local.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/threadpool_imp.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/util/xxhash.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/backupable/backupable_db.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_compaction_filter.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl_filesnapshot.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_dump_tool.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_file.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/cassandra/cassandra_compaction_filter.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/cassandra/format.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/cassandra/merge_operator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/checkpoint/checkpoint_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters/remove_emptyvalue_compactionfilter.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/debug.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/env_mirror.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/env_timed.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_env.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_fs.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/leveldb_options/leveldb_options.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/memory/memory_util.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/bytesxor.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/max.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/put.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/sortlist.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/string_append/stringappend.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/string_append/stringappend2.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/uint64add.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/object_registry.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/option_change_migration/option_change_migration.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/options/options_util.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/block_cache_tier.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/block_cache_tier_file.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/block_cache_tier_metadata.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/persistent_cache_tier.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/volatile_tier_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/cache_simulator.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/sim_cache.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_on_deletion_collector.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/trace/file_trace_reader_writer.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/lock_manager.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_tracker.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_manager.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/optimistic_transaction_db_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/optimistic_transaction.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/pessimistic_transaction.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/pessimistic_transaction_db.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/snapshot_checker.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_base.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_db_mutex_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_util.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn_db.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn_db.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/ttl/db_ttl_impl.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index.cc"
|
||||
"${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index_internal.cc"
|
||||
$<TARGET_OBJECTS:rocksdb_build_version>)
|
||||
${ROCKSDB_SOURCE_DIR}/cache/cache.cc
|
||||
${ROCKSDB_SOURCE_DIR}/cache/cache_entry_roles.cc
|
||||
${ROCKSDB_SOURCE_DIR}/cache/clock_cache.cc
|
||||
${ROCKSDB_SOURCE_DIR}/cache/lru_cache.cc
|
||||
${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/arena_wrapped_db_iter.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_fetcher.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_addition.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_builder.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_cache.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_garbage.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_meta.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_garbage_meter.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_format.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_sequential_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_writer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/builder.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/c.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/column_family.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/compaction/compaction.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_iterator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_job.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_fifo.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_level.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_universal.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/compaction/sst_partitioner.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/convenience.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_filesnapshot.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/compacted_db_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_write.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_compaction_flush.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_files.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_open.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_debug.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_experimental.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_readonly.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_secondary.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_info_dumper.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/db_iter.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/dbformat.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/error_handler.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/event_helpers.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/experimental.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/external_sst_file_ingestion_job.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/file_indexer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/flush_job.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/flush_scheduler.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/forward_iterator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/import_column_family_job.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/internal_stats.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/logs_with_prep_tracker.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/log_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/log_writer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/malloc_stats.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/memtable.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/memtable_list.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/merge_helper.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/merge_operator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/output_validator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/periodic_work_scheduler.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/range_del_aggregator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/range_tombstone_fragmenter.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/repair.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/snapshot_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/table_cache.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/table_properties_collector.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/transaction_log_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/trim_history_scheduler.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/version_builder.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/version_edit.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/version_edit_handler.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/version_set.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/wal_edit.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/wal_manager.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/write_batch.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/write_batch_base.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/write_controller.cc
|
||||
${ROCKSDB_SOURCE_DIR}/db/write_thread.cc
|
||||
${ROCKSDB_SOURCE_DIR}/env/composite_env.cc
|
||||
${ROCKSDB_SOURCE_DIR}/env/env.cc
|
||||
${ROCKSDB_SOURCE_DIR}/env/env_chroot.cc
|
||||
${ROCKSDB_SOURCE_DIR}/env/env_encryption.cc
|
||||
${ROCKSDB_SOURCE_DIR}/env/env_hdfs.cc
|
||||
${ROCKSDB_SOURCE_DIR}/env/file_system.cc
|
||||
${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/env/fs_remap.cc
|
||||
${ROCKSDB_SOURCE_DIR}/env/mock_env.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/delete_scheduler.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/file_prefetch_buffer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/file_util.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/filename.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/line_file_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/random_access_file_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/read_write_util.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/readahead_raf.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/sequence_file_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/sst_file_manager_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/file/writable_file_writer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/logging/auto_roll_logger.cc
|
||||
${ROCKSDB_SOURCE_DIR}/logging/event_logger.cc
|
||||
${ROCKSDB_SOURCE_DIR}/logging/log_buffer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memory/arena.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memory/concurrent_arena.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memory/jemalloc_nodump_allocator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memory/memkind_kmem_allocator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memtable/alloc_tracker.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memtable/hash_linklist_rep.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memtable/hash_skiplist_rep.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memtable/skiplistrep.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memtable/vectorrep.cc
|
||||
${ROCKSDB_SOURCE_DIR}/memtable/write_buffer_manager.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/histogram.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/histogram_windowing.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/in_memory_stats_history.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/instrumented_mutex.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/iostats_context.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/perf_context.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/perf_level.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/persistent_stats_history.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/statistics.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_updater.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_util.cc
|
||||
${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_util_debug.cc
|
||||
${ROCKSDB_SOURCE_DIR}/options/cf_options.cc
|
||||
${ROCKSDB_SOURCE_DIR}/options/configurable.cc
|
||||
${ROCKSDB_SOURCE_DIR}/options/customizable.cc
|
||||
${ROCKSDB_SOURCE_DIR}/options/db_options.cc
|
||||
${ROCKSDB_SOURCE_DIR}/options/options.cc
|
||||
${ROCKSDB_SOURCE_DIR}/options/options_helper.cc
|
||||
${ROCKSDB_SOURCE_DIR}/options/options_parser.cc
|
||||
${ROCKSDB_SOURCE_DIR}/port/stack_trace.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/adaptive/adaptive_table_factory.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/binary_search_index_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/block.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_filter_block.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_builder.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_factory.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_iterator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/block_builder.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefetcher.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefix_index.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/data_block_hash_index.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/data_block_footer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/filter_block_reader_common.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/filter_policy.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/flush_block_policy.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/full_filter_block.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/hash_index_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/index_builder.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/index_reader_common.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/parsed_full_filter_block.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/partitioned_filter_block.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/partitioned_index_iterator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/partitioned_index_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/reader_common.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_based/uncompression_dict_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/block_fetcher.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/cuckoo/cuckoo_table_builder.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/cuckoo/cuckoo_table_factory.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/cuckoo/cuckoo_table_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/format.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/get_context.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/iterator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/merging_iterator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/meta_blocks.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/persistent_cache_helper.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_bloom.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_builder.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_factory.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_index.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_key_coding.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/sst_file_dumper.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/sst_file_reader.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/sst_file_writer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/table_factory.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/table_properties.cc
|
||||
${ROCKSDB_SOURCE_DIR}/table/two_level_iterator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/test_util/sync_point.cc
|
||||
${ROCKSDB_SOURCE_DIR}/test_util/sync_point_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/test_util/testutil.cc
|
||||
${ROCKSDB_SOURCE_DIR}/test_util/transaction_test_util.cc
|
||||
${ROCKSDB_SOURCE_DIR}/tools/block_cache_analyzer/block_cache_trace_analyzer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/tools/dump/db_dump_tool.cc
|
||||
${ROCKSDB_SOURCE_DIR}/tools/io_tracer_parser_tool.cc
|
||||
${ROCKSDB_SOURCE_DIR}/tools/ldb_cmd.cc
|
||||
${ROCKSDB_SOURCE_DIR}/tools/ldb_tool.cc
|
||||
${ROCKSDB_SOURCE_DIR}/tools/sst_dump_tool.cc
|
||||
${ROCKSDB_SOURCE_DIR}/tools/trace_analyzer_tool.cc
|
||||
${ROCKSDB_SOURCE_DIR}/trace_replay/trace_replay.cc
|
||||
${ROCKSDB_SOURCE_DIR}/trace_replay/block_cache_tracer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/trace_replay/io_tracer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/coding.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/compaction_job_stats_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/comparator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/compression_context_cache.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/concurrent_task_limiter_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/crc32c.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/dynamic_bloom.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/hash.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/murmurhash.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/random.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/rate_limiter.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/ribbon_config.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/slice.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/file_checksum_helper.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/status.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/string_util.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/thread_local.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/threadpool_imp.cc
|
||||
${ROCKSDB_SOURCE_DIR}/util/xxhash.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/backupable/backupable_db.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_compaction_filter.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl_filesnapshot.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_dump_tool.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_file.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/cassandra/cassandra_compaction_filter.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/cassandra/format.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/cassandra/merge_operator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/checkpoint/checkpoint_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters/remove_emptyvalue_compactionfilter.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/debug.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/env_mirror.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/env_timed.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_env.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_fs.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/leveldb_options/leveldb_options.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/memory/memory_util.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/bytesxor.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/max.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/put.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/sortlist.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/string_append/stringappend.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/string_append/stringappend2.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/uint64add.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/object_registry.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/option_change_migration/option_change_migration.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/options/options_util.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/block_cache_tier.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/block_cache_tier_file.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/block_cache_tier_metadata.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/persistent_cache_tier.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/volatile_tier_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/cache_simulator.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/sim_cache.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_on_deletion_collector.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/trace/file_trace_reader_writer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/lock_manager.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_tracker.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_manager.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/range_tree_lock_manager.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/range_tree_lock_tracker.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/optimistic_transaction_db_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/optimistic_transaction.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/pessimistic_transaction.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/pessimistic_transaction_db.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/snapshot_checker.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_base.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_db_mutex_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_util.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn_db.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn_db.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/ttl/db_ttl_impl.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index_internal.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/keyrange.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/lock_request.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/locktree.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/manager.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/range_buffer.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/treenode.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/txnid_set.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/wfg.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/standalone_port.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/util/dbt.cc
|
||||
${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc
|
||||
rocksdb_build_version.cc)
|
||||
|
||||
if(HAVE_SSE42 AND NOT MSVC)
|
||||
set_source_files_properties(
|
||||
|
@ -1,3 +1,62 @@
|
||||
const char* rocksdb_build_git_sha = "rocksdb_build_git_sha:0";
|
||||
const char* rocksdb_build_git_date = "rocksdb_build_git_date:2000-01-01";
|
||||
const char* rocksdb_build_compile_date = "2000-01-01";
|
||||
// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
|
||||
/// This file was edited for ClickHouse.
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include "rocksdb/version.h"
|
||||
#include "util/string_util.h"
|
||||
|
||||
// The build script may replace these values with real values based
|
||||
// on whether or not GIT is available and the platform settings
|
||||
static const std::string rocksdb_build_git_sha = "rocksdb_build_git_sha:0";
|
||||
static const std::string rocksdb_build_git_tag = "rocksdb_build_git_tag:master";
|
||||
static const std::string rocksdb_build_date = "rocksdb_build_date:2000-01-01";
|
||||
|
||||
namespace ROCKSDB_NAMESPACE {
|
||||
static void AddProperty(std::unordered_map<std::string, std::string> *props, const std::string& name) {
|
||||
size_t colon = name.find(":");
|
||||
if (colon != std::string::npos && colon > 0 && colon < name.length() - 1) {
|
||||
// If we found a "@:", then this property was a build-time substitution that failed. Skip it
|
||||
size_t at = name.find("@", colon);
|
||||
if (at != colon + 1) {
|
||||
// Everything before the colon is the name, after is the value
|
||||
(*props)[name.substr(0, colon)] = name.substr(colon + 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static std::unordered_map<std::string, std::string>* LoadPropertiesSet() {
|
||||
auto * properties = new std::unordered_map<std::string, std::string>();
|
||||
AddProperty(properties, rocksdb_build_git_sha);
|
||||
AddProperty(properties, rocksdb_build_git_tag);
|
||||
AddProperty(properties, rocksdb_build_date);
|
||||
return properties;
|
||||
}
|
||||
|
||||
const std::unordered_map<std::string, std::string>& GetRocksBuildProperties() {
|
||||
static std::unique_ptr<std::unordered_map<std::string, std::string>> props(LoadPropertiesSet());
|
||||
return *props;
|
||||
}
|
||||
|
||||
std::string GetRocksVersionAsString(bool with_patch) {
|
||||
std::string version = ToString(ROCKSDB_MAJOR) + "." + ToString(ROCKSDB_MINOR);
|
||||
if (with_patch) {
|
||||
return version + "." + ToString(ROCKSDB_PATCH);
|
||||
} else {
|
||||
return version;
|
||||
}
|
||||
}
|
||||
|
||||
std::string GetRocksBuildInfoAsString(const std::string& program, bool verbose) {
|
||||
std::string info = program + " (RocksDB) " + GetRocksVersionAsString(true);
|
||||
if (verbose) {
|
||||
for (const auto& it : GetRocksBuildProperties()) {
|
||||
info.append("\n ");
|
||||
info.append(it.first);
|
||||
info.append(": ");
|
||||
info.append(it.second);
|
||||
}
|
||||
}
|
||||
return info;
|
||||
}
|
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
|
@ -58,11 +58,11 @@ function start()
|
||||
echo "Cannot start clickhouse-server"
|
||||
cat /var/log/clickhouse-server/stdout.log
|
||||
tail -n1000 /var/log/clickhouse-server/stderr.log
|
||||
tail -n1000 /var/log/clickhouse-server/clickhouse-server.log
|
||||
tail -n100000 /var/log/clickhouse-server/clickhouse-server.log | grep -F -v '<Warning> RaftInstance:' -e '<Information> RaftInstance' | tail -n1000
|
||||
break
|
||||
fi
|
||||
# use root to match with current uid
|
||||
clickhouse start --user root >/var/log/clickhouse-server/stdout.log 2>/var/log/clickhouse-server/stderr.log
|
||||
clickhouse start --user root >/var/log/clickhouse-server/stdout.log 2>>/var/log/clickhouse-server/stderr.log
|
||||
sleep 0.5
|
||||
counter=$((counter + 1))
|
||||
done
|
||||
@ -125,7 +125,7 @@ zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log*
|
||||
# Sanitizer asserts
|
||||
zgrep -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
|
||||
zgrep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
|
||||
zgrep -Fav "ASan doesn't fully support makecontext/swapcontext functions" > /dev/null \
|
||||
zgrep -Fav "ASan doesn't fully support makecontext/swapcontext functions" /test_output/tmp > /dev/null \
|
||||
&& echo -e 'Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/test_results.tsv \
|
||||
|| echo -e 'No sanitizer asserts\tOK' >> /test_output/test_results.tsv
|
||||
rm -f /test_output/tmp
|
||||
|
@ -37,6 +37,14 @@ Also, it accepts the following settings:
|
||||
|
||||
- `max_delay_to_insert` - max delay of inserting data into Distributed table in seconds, if there are a lot of pending bytes for async send. Default 60.
|
||||
|
||||
- `monitor_batch_inserts` - same as [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts)
|
||||
|
||||
- `monitor_split_batch_on_failure` - same as [distributed_directory_monitor_split_batch_on_failure](../../../operations/settings/settings.md#distributed_directory_monitor_split_batch_on_failure)
|
||||
|
||||
- `monitor_sleep_time_ms` - same as [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms)
|
||||
|
||||
- `monitor_max_sleep_time_ms` - same as [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms)
|
||||
|
||||
!!! note "Note"
|
||||
|
||||
**Durability settings** (`fsync_...`):
|
||||
|
@ -189,7 +189,7 @@ CREATE TABLE codec_example
|
||||
dt Date CODEC(ZSTD),
|
||||
ts DateTime CODEC(LZ4HC),
|
||||
float_value Float32 CODEC(NONE),
|
||||
double_value Float64 CODEC(LZ4HC(9))
|
||||
double_value Float64 CODEC(LZ4HC(9)),
|
||||
value Float32 CODEC(Delta, ZSTD)
|
||||
)
|
||||
ENGINE = <Engine>
|
||||
|
@ -26,6 +26,7 @@
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
#include <Poco/String.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <common/find_symbols.h>
|
||||
#include <common/LineReader.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
@ -301,26 +302,9 @@ private:
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
bool print_stack_trace = config().getBool("stacktrace", false);
|
||||
bool print_stack_trace = config().getBool("stacktrace", false) && e.code() != ErrorCodes::NETWORK_ERROR;
|
||||
|
||||
std::string text = e.displayText();
|
||||
|
||||
/** If exception is received from server, then stack trace is embedded in message.
|
||||
* If exception is thrown on client, then stack trace is in separate field.
|
||||
*/
|
||||
|
||||
auto embedded_stack_trace_pos = text.find("Stack trace");
|
||||
if (std::string::npos != embedded_stack_trace_pos && !print_stack_trace)
|
||||
text.resize(embedded_stack_trace_pos);
|
||||
|
||||
std::cerr << "Code: " << e.code() << ". " << text << std::endl << std::endl;
|
||||
|
||||
/// Don't print the stack trace on the client if it was logged on the server.
|
||||
/// Also don't print the stack trace in case of network errors.
|
||||
if (print_stack_trace && e.code() != ErrorCodes::NETWORK_ERROR && std::string::npos == embedded_stack_trace_pos)
|
||||
{
|
||||
std::cerr << "Stack trace:" << std::endl << e.getStackTraceString();
|
||||
}
|
||||
std::cerr << getExceptionMessage(e, print_stack_trace, true) << std::endl << std::endl;
|
||||
|
||||
/// If exception code isn't zero, we should return non-zero return code anyway.
|
||||
return e.code() ? e.code() : -1;
|
||||
@ -487,6 +471,52 @@ private:
|
||||
}
|
||||
#endif
|
||||
|
||||
/// Make query to get all server warnings
|
||||
std::vector<String> loadWarningMessages()
|
||||
{
|
||||
std::vector<String> messages;
|
||||
connection->sendQuery(connection_parameters.timeouts, "SELECT message FROM system.warnings", "" /* query_id */, QueryProcessingStage::Complete);
|
||||
while (true)
|
||||
{
|
||||
Packet packet = connection->receivePacket();
|
||||
switch (packet.type)
|
||||
{
|
||||
case Protocol::Server::Data:
|
||||
if (packet.block)
|
||||
{
|
||||
const ColumnString & column = typeid_cast<const ColumnString &>(*packet.block.getByPosition(0).column);
|
||||
|
||||
size_t rows = packet.block.rows();
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
messages.emplace_back(column.getDataAt(i).toString());
|
||||
}
|
||||
continue;
|
||||
|
||||
case Protocol::Server::Progress:
|
||||
continue;
|
||||
case Protocol::Server::ProfileInfo:
|
||||
continue;
|
||||
case Protocol::Server::Totals:
|
||||
continue;
|
||||
case Protocol::Server::Extremes:
|
||||
continue;
|
||||
case Protocol::Server::Log:
|
||||
continue;
|
||||
|
||||
case Protocol::Server::Exception:
|
||||
packet.exception->rethrow();
|
||||
return messages;
|
||||
|
||||
case Protocol::Server::EndOfStream:
|
||||
return messages;
|
||||
|
||||
default:
|
||||
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}",
|
||||
packet.type, connection->getDescription());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
int mainImpl()
|
||||
{
|
||||
UseSSL use_ssl;
|
||||
@ -565,6 +595,26 @@ private:
|
||||
suggest->load(connection_parameters, config().getInt("suggestion_limit"));
|
||||
}
|
||||
|
||||
/// Load Warnings at the beginning of connection
|
||||
if (!config().has("no-warnings"))
|
||||
{
|
||||
try
|
||||
{
|
||||
std::vector<String> messages = loadWarningMessages();
|
||||
if (!messages.empty())
|
||||
{
|
||||
std::cout << "Warnings:" << std::endl;
|
||||
for (const auto & message : messages)
|
||||
std::cout << "* " << message << std::endl;
|
||||
}
|
||||
std::cout << std::endl;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Ignore exception
|
||||
}
|
||||
}
|
||||
|
||||
/// Load command history if present.
|
||||
if (config().has("history_file"))
|
||||
history_file = config().getString("history_file");
|
||||
@ -633,17 +683,10 @@ private:
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
// We don't need to handle the test hints in the interactive
|
||||
// mode.
|
||||
std::cerr << std::endl
|
||||
<< "Exception on client:" << std::endl
|
||||
<< "Code: " << e.code() << ". " << e.displayText() << std::endl;
|
||||
|
||||
if (config().getBool("stacktrace", false))
|
||||
std::cerr << "Stack trace:" << std::endl << e.getStackTraceString() << std::endl;
|
||||
|
||||
std::cerr << std::endl;
|
||||
/// We don't need to handle the test hints in the interactive mode.
|
||||
|
||||
bool print_stack_trace = config().getBool("stacktrace", false);
|
||||
std::cerr << "Exception on client:" << std::endl << getExceptionMessage(e, print_stack_trace, true) << std::endl << std::endl;
|
||||
client_exception = std::make_unique<Exception>(e);
|
||||
}
|
||||
|
||||
@ -940,18 +983,11 @@ private:
|
||||
{
|
||||
if (server_exception)
|
||||
{
|
||||
std::string text = server_exception->displayText();
|
||||
auto embedded_stack_trace_pos = text.find("Stack trace");
|
||||
if (std::string::npos != embedded_stack_trace_pos && !config().getBool("stacktrace", false))
|
||||
{
|
||||
text.resize(embedded_stack_trace_pos);
|
||||
}
|
||||
bool print_stack_trace = config().getBool("stacktrace", false);
|
||||
std::cerr << "Received exception from server (version " << server_version << "):" << std::endl
|
||||
<< "Code: " << server_exception->code() << ". " << text << std::endl;
|
||||
<< getExceptionMessage(*server_exception, print_stack_trace, true) << std::endl;
|
||||
if (is_interactive)
|
||||
{
|
||||
std::cerr << std::endl;
|
||||
}
|
||||
}
|
||||
|
||||
if (client_exception)
|
||||
@ -1410,8 +1446,7 @@ private:
|
||||
{
|
||||
// Just report it, we'll terminate below.
|
||||
fmt::print(stderr,
|
||||
"Error while reconnecting to the server: Code: {}: {}\n",
|
||||
getCurrentExceptionCode(),
|
||||
"Error while reconnecting to the server: {}\n",
|
||||
getCurrentExceptionMessage(true));
|
||||
|
||||
assert(!connection->isConnected());
|
||||
@ -2529,6 +2564,7 @@ public:
|
||||
("opentelemetry-traceparent", po::value<std::string>(), "OpenTelemetry traceparent header as described by W3C Trace Context recommendation")
|
||||
("opentelemetry-tracestate", po::value<std::string>(), "OpenTelemetry tracestate header as described by W3C Trace Context recommendation")
|
||||
("history_file", po::value<std::string>(), "path to history file")
|
||||
("no-warnings", "disable warnings when client connects to server")
|
||||
;
|
||||
|
||||
Settings cmd_settings;
|
||||
@ -2596,8 +2632,7 @@ public:
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
std::string text = e.displayText();
|
||||
std::cerr << "Code: " << e.code() << ". " << text << std::endl;
|
||||
std::cerr << getExceptionMessage(e, false) << std::endl;
|
||||
std::cerr << "Table №" << i << std::endl << std::endl;
|
||||
/// Avoid the case when error exit code can possibly overflow to normal (zero).
|
||||
auto exit_code = e.code() % 256;
|
||||
@ -2689,6 +2724,8 @@ public:
|
||||
config().setBool("highlight", options["highlight"].as<bool>());
|
||||
if (options.count("history_file"))
|
||||
config().setString("history_file", options["history_file"].as<std::string>());
|
||||
if (options.count("no-warnings"))
|
||||
config().setBool("no-warnings", true);
|
||||
|
||||
if ((query_fuzzer_runs = options["query-fuzzer-runs"].as<int>()))
|
||||
{
|
||||
@ -2740,8 +2777,7 @@ int mainEntryClickHouseClient(int argc, char ** argv)
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::string text = e.displayText();
|
||||
std::cerr << "Code: " << e.code() << ". " << text << std::endl;
|
||||
std::cerr << DB::getExceptionMessage(e, false) << std::endl;
|
||||
return 1;
|
||||
}
|
||||
catch (...)
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionSequenceMatch.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <common/range.h>
|
||||
|
@ -459,6 +459,8 @@ public:
|
||||
explicit FieldVisitorMax(const Field & rhs_) : rhs(rhs_) {}
|
||||
|
||||
bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (NegativeInfinity &) const { throw Exception("Cannot compare -Inf", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (PositiveInfinity &) const { throw Exception("Cannot compare +Inf", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot compare AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
||||
bool operator() (Array & x) const { return compareImpl<Array>(x); }
|
||||
@ -494,6 +496,8 @@ public:
|
||||
explicit FieldVisitorMin(const Field & rhs_) : rhs(rhs_) {}
|
||||
|
||||
bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (NegativeInfinity &) const { throw Exception("Cannot compare -Inf", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (PositiveInfinity &) const { throw Exception("Cannot compare +Inf", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot sum AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
||||
bool operator() (Array & x) const { return compareImpl<Array>(x); }
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
@ -49,6 +50,8 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDate::FieldType, Data>>(argument_types);
|
||||
else if (which.isDate32())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDate32::FieldType, Data>>(argument_types);
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDateTime::FieldType, Data>>(argument_types);
|
||||
else if (which.isStringOrFixedString())
|
||||
@ -95,6 +98,8 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDate::FieldType, Data<DataTypeDate::FieldType>>>(argument_types);
|
||||
else if (which.isDate32())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDate32::FieldType, Data<DataTypeDate32::FieldType>>>(argument_types);
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDateTime::FieldType, Data<DataTypeDateTime::FieldType>>>(argument_types);
|
||||
else if (which.isStringOrFixedString())
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <functional>
|
||||
@ -51,6 +52,8 @@ namespace
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate::FieldType>>(argument_types, params);
|
||||
else if (which.isDate32())
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate32::FieldType>>(argument_types, params);
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDateTime::FieldType>>(argument_types, params);
|
||||
else if (which.isStringOrFixedString())
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionUniqUpTo.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
@ -61,6 +62,8 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDate::FieldType>>(threshold, argument_types, params);
|
||||
else if (which.isDate32())
|
||||
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDate32::FieldType>>(threshold, argument_types, params);
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDateTime::FieldType>>(threshold, argument_types, params);
|
||||
else if (which.isStringOrFixedString())
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <common/range.h>
|
||||
|
@ -353,6 +353,11 @@ bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLoc
|
||||
if (offset_states[location.offset].active_connection_count == 0 && !offset_states[location.offset].next_replica_in_process)
|
||||
throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT);
|
||||
}
|
||||
else if (std::holds_alternative<std::exception_ptr>(res))
|
||||
{
|
||||
finishProcessReplica(replica_state, true);
|
||||
std::rethrow_exception(std::move(std::get<std::exception_ptr>(res)));
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
}
|
||||
|
||||
/// Resume packet receiving.
|
||||
std::variant<int, Packet, Poco::Timespan> resume()
|
||||
std::variant<int, Packet, Poco::Timespan, std::exception_ptr> resume()
|
||||
{
|
||||
/// If there is no pending data, check receive timeout.
|
||||
if (!connection->hasReadPendingData() && !checkReceiveTimeout())
|
||||
@ -43,7 +43,7 @@ public:
|
||||
/// Resume fiber.
|
||||
fiber = std::move(fiber).resume();
|
||||
if (exception)
|
||||
std::rethrow_exception(std::move(exception));
|
||||
return std::move(exception);
|
||||
|
||||
if (is_read_in_process)
|
||||
return epoll.getFileDescriptor();
|
||||
|
@ -546,97 +546,54 @@ namespace
|
||||
{
|
||||
|
||||
/// The following function implements a slightly more general version
|
||||
/// of getExtremes() than the implementation from ColumnVector.
|
||||
/// of getExtremes() than the implementation from Not-Null IColumns.
|
||||
/// It takes into account the possible presence of nullable values.
|
||||
template <typename T>
|
||||
void getExtremesFromNullableContent(const ColumnVector<T> & col, const NullMap & null_map, Field & min, Field & max)
|
||||
void getExtremesWithNulls(const IColumn & nested_column, const NullMap & null_array, Field & min, Field & max, bool null_last = false)
|
||||
{
|
||||
const auto & data = col.getData();
|
||||
size_t size = data.size();
|
||||
|
||||
if (size == 0)
|
||||
size_t number_of_nulls = 0;
|
||||
size_t n = null_array.size();
|
||||
NullMap not_null_array(n);
|
||||
for (auto i = 0ul; i < n; ++i)
|
||||
{
|
||||
min = Null();
|
||||
max = Null();
|
||||
return;
|
||||
}
|
||||
|
||||
bool has_not_null = false;
|
||||
bool has_not_nan = false;
|
||||
|
||||
T cur_min = 0;
|
||||
T cur_max = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const T x = data[i];
|
||||
|
||||
if (null_map[i])
|
||||
continue;
|
||||
|
||||
if (!has_not_null)
|
||||
if (null_array[i])
|
||||
{
|
||||
cur_min = x;
|
||||
cur_max = x;
|
||||
has_not_null = true;
|
||||
has_not_nan = !isNaN(x);
|
||||
continue;
|
||||
++number_of_nulls;
|
||||
not_null_array[i] = 0;
|
||||
}
|
||||
|
||||
if (isNaN(x))
|
||||
continue;
|
||||
|
||||
if (!has_not_nan)
|
||||
else
|
||||
{
|
||||
cur_min = x;
|
||||
cur_max = x;
|
||||
has_not_nan = true;
|
||||
continue;
|
||||
not_null_array[i] = 1;
|
||||
}
|
||||
|
||||
if (x < cur_min)
|
||||
cur_min = x;
|
||||
else if (x > cur_max)
|
||||
cur_max = x;
|
||||
}
|
||||
|
||||
if (has_not_null)
|
||||
if (number_of_nulls == 0)
|
||||
{
|
||||
min = cur_min;
|
||||
max = cur_max;
|
||||
nested_column.getExtremes(min, max);
|
||||
}
|
||||
else if (number_of_nulls == n)
|
||||
{
|
||||
min = PositiveInfinity();
|
||||
max = PositiveInfinity();
|
||||
}
|
||||
else
|
||||
{
|
||||
auto filtered_column = nested_column.filter(not_null_array, -1);
|
||||
filtered_column->getExtremes(min, max);
|
||||
if (null_last)
|
||||
max = PositiveInfinity();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
void ColumnNullable::getExtremes(Field & min, Field & max) const
|
||||
{
|
||||
min = Null();
|
||||
max = Null();
|
||||
getExtremesWithNulls(getNestedColumn(), getNullMapData(), min, max);
|
||||
}
|
||||
|
||||
const auto & null_map_data = getNullMapData();
|
||||
|
||||
if (const auto * col_i8 = typeid_cast<const ColumnInt8 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<Int8>(*col_i8, null_map_data, min, max);
|
||||
else if (const auto * col_i16 = typeid_cast<const ColumnInt16 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<Int16>(*col_i16, null_map_data, min, max);
|
||||
else if (const auto * col_i32 = typeid_cast<const ColumnInt32 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<Int32>(*col_i32, null_map_data, min, max);
|
||||
else if (const auto * col_i64 = typeid_cast<const ColumnInt64 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<Int64>(*col_i64, null_map_data, min, max);
|
||||
else if (const auto * col_u8 = typeid_cast<const ColumnUInt8 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<UInt8>(*col_u8, null_map_data, min, max);
|
||||
else if (const auto * col_u16 = typeid_cast<const ColumnUInt16 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<UInt16>(*col_u16, null_map_data, min, max);
|
||||
else if (const auto * col_u32 = typeid_cast<const ColumnUInt32 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<UInt32>(*col_u32, null_map_data, min, max);
|
||||
else if (const auto * col_u64 = typeid_cast<const ColumnUInt64 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<UInt64>(*col_u64, null_map_data, min, max);
|
||||
else if (const auto * col_f32 = typeid_cast<const ColumnFloat32 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<Float32>(*col_f32, null_map_data, min, max);
|
||||
else if (const auto * col_f64 = typeid_cast<const ColumnFloat64 *>(nested_column.get()))
|
||||
getExtremesFromNullableContent<Float64>(*col_f64, null_map_data, min, max);
|
||||
void ColumnNullable::getExtremesNullLast(Field & min, Field & max) const
|
||||
{
|
||||
getExtremesWithNulls(getNestedColumn(), getNullMapData(), min, max, true);
|
||||
}
|
||||
|
||||
|
||||
|
@ -111,6 +111,8 @@ public:
|
||||
void updateWeakHash32(WeakHash32 & hash) const override;
|
||||
void updateHashFast(SipHash & hash) const override;
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
// Special function for nullable minmax index
|
||||
void getExtremesNullLast(Field & min, Field & max) const;
|
||||
|
||||
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override
|
||||
{
|
||||
|
@ -313,7 +313,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
|
||||
try
|
||||
{
|
||||
stream << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
||||
<< ", e.displayText() = " << e.displayText()
|
||||
<< ", " << e.displayText()
|
||||
<< (with_stacktrace ? ", Stack trace (when copying this message, always include the lines below):\n\n" + getExceptionStackTraceString(e) : "")
|
||||
<< (with_extra_info ? getExtraExceptionInfo(e) : "")
|
||||
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
|
||||
@ -433,7 +433,12 @@ std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool
|
||||
}
|
||||
}
|
||||
|
||||
stream << "Code: " << e.code() << ", e.displayText() = " << text;
|
||||
stream << "Code: " << e.code() << ". " << text;
|
||||
|
||||
if (!text.empty() && text.back() != '.')
|
||||
stream << '.';
|
||||
|
||||
stream << " (" << ErrorCodes::getName(e.code()) << ")";
|
||||
|
||||
if (with_stacktrace && !has_embedded_stack_trace)
|
||||
stream << ", Stack trace (when copying this message, always include the lines below):\n\n" << e.getStackTraceString();
|
||||
|
@ -26,6 +26,16 @@ public:
|
||||
throw Exception("Cannot convert NULL to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const NegativeInfinity &) const
|
||||
{
|
||||
throw Exception("Cannot convert -Inf to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const PositiveInfinity &) const
|
||||
{
|
||||
throw Exception("Cannot convert +Inf to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const String &) const
|
||||
{
|
||||
throw Exception("Cannot convert String to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
|
@ -25,6 +25,8 @@ static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const Null &) const { return "NULL"; }
|
||||
String FieldVisitorDump::operator() (const NegativeInfinity &) const { return "-Inf"; }
|
||||
String FieldVisitorDump::operator() (const PositiveInfinity &) const { return "+Inf"; }
|
||||
String FieldVisitorDump::operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); }
|
||||
String FieldVisitorDump::operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); }
|
||||
String FieldVisitorDump::operator() (const Float64 & x) const { return formatQuotedWithPrefix(x, "Float64_"); }
|
||||
|
@ -10,6 +10,8 @@ class FieldVisitorDump : public StaticVisitor<String>
|
||||
{
|
||||
public:
|
||||
String operator() (const Null & x) const;
|
||||
String operator() (const NegativeInfinity & x) const;
|
||||
String operator() (const PositiveInfinity & x) const;
|
||||
String operator() (const UInt64 & x) const;
|
||||
String operator() (const UInt128 & x) const;
|
||||
String operator() (const UInt256 & x) const;
|
||||
|
@ -14,6 +14,18 @@ void FieldVisitorHash::operator() (const Null &) const
|
||||
hash.update(type);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const NegativeInfinity &) const
|
||||
{
|
||||
UInt8 type = Field::Types::NegativeInfinity;
|
||||
hash.update(type);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const PositiveInfinity &) const
|
||||
{
|
||||
UInt8 type = Field::Types::PositiveInfinity;
|
||||
hash.update(type);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UInt64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UInt64;
|
||||
|
@ -16,6 +16,8 @@ public:
|
||||
FieldVisitorHash(SipHash & hash_);
|
||||
|
||||
void operator() (const Null & x) const;
|
||||
void operator() (const NegativeInfinity & x) const;
|
||||
void operator() (const PositiveInfinity & x) const;
|
||||
void operator() (const UInt64 & x) const;
|
||||
void operator() (const UInt128 & x) const;
|
||||
void operator() (const UInt256 & x) const;
|
||||
|
@ -22,6 +22,8 @@ bool FieldVisitorSum::operator() (UInt64 & x) const
|
||||
bool FieldVisitorSum::operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }
|
||||
|
||||
bool FieldVisitorSum::operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (NegativeInfinity &) const { throw Exception("Cannot sum -Inf", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (PositiveInfinity &) const { throw Exception("Cannot sum +Inf", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (Tuple &) const { throw Exception("Cannot sum Tuples", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
@ -21,6 +21,8 @@ public:
|
||||
bool operator() (UInt64 & x) const;
|
||||
bool operator() (Float64 & x) const;
|
||||
bool operator() (Null &) const;
|
||||
bool operator() (NegativeInfinity & x) const;
|
||||
bool operator() (PositiveInfinity & x) const;
|
||||
bool operator() (String &) const;
|
||||
bool operator() (Array &) const;
|
||||
bool operator() (Tuple &) const;
|
||||
|
@ -53,6 +53,8 @@ static String formatFloat(const Float64 x)
|
||||
|
||||
|
||||
String FieldVisitorToString::operator() (const Null &) const { return "NULL"; }
|
||||
String FieldVisitorToString::operator() (const NegativeInfinity &) const { return "-Inf"; }
|
||||
String FieldVisitorToString::operator() (const PositiveInfinity &) const { return "+Inf"; }
|
||||
String FieldVisitorToString::operator() (const UInt64 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Int64 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Float64 & x) const { return formatFloat(x); }
|
||||
|
@ -10,6 +10,8 @@ class FieldVisitorToString : public StaticVisitor<String>
|
||||
{
|
||||
public:
|
||||
String operator() (const Null & x) const;
|
||||
String operator() (const NegativeInfinity & x) const;
|
||||
String operator() (const PositiveInfinity & x) const;
|
||||
String operator() (const UInt64 & x) const;
|
||||
String operator() (const UInt128 & x) const;
|
||||
String operator() (const UInt256 & x) const;
|
||||
|
@ -7,6 +7,8 @@ namespace DB
|
||||
{
|
||||
|
||||
void FieldVisitorWriteBinary::operator() (const Null &, WriteBuffer &) const { }
|
||||
void FieldVisitorWriteBinary::operator() (const NegativeInfinity &, WriteBuffer &) const { }
|
||||
void FieldVisitorWriteBinary::operator() (const PositiveInfinity &, WriteBuffer &) const { }
|
||||
void FieldVisitorWriteBinary::operator() (const UInt64 & x, WriteBuffer & buf) const { writeVarUInt(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Int64 & x, WriteBuffer & buf) const { writeVarInt(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Float64 & x, WriteBuffer & buf) const { writeFloatBinary(x, buf); }
|
||||
|
@ -9,6 +9,8 @@ class FieldVisitorWriteBinary
|
||||
{
|
||||
public:
|
||||
void operator() (const Null & x, WriteBuffer & buf) const;
|
||||
void operator() (const NegativeInfinity & x, WriteBuffer & buf) const;
|
||||
void operator() (const PositiveInfinity & x, WriteBuffer & buf) const;
|
||||
void operator() (const UInt64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UInt128 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UInt256 & x, WriteBuffer & buf) const;
|
||||
|
@ -26,8 +26,12 @@ public:
|
||||
template <typename T, typename U>
|
||||
bool operator() (const T & l, const U & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>)
|
||||
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>
|
||||
|| std::is_same_v<T, NegativeInfinity> || std::is_same_v<T, PositiveInfinity>
|
||||
|| std::is_same_v<U, NegativeInfinity> || std::is_same_v<U, PositiveInfinity>)
|
||||
{
|
||||
return std::is_same_v<T, U>;
|
||||
}
|
||||
else
|
||||
{
|
||||
if constexpr (std::is_same_v<T, U>)
|
||||
@ -77,6 +81,10 @@ public:
|
||||
{
|
||||
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>)
|
||||
return false;
|
||||
else if constexpr (std::is_same_v<T, NegativeInfinity> || std::is_same_v<U, PositiveInfinity>)
|
||||
return !std::is_same_v<T, U>;
|
||||
else if constexpr (std::is_same_v<U, NegativeInfinity> || std::is_same_v<T, PositiveInfinity>)
|
||||
return false;
|
||||
else
|
||||
{
|
||||
if constexpr (std::is_same_v<T, U>)
|
||||
|
@ -248,6 +248,9 @@
|
||||
M(S3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \
|
||||
M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \
|
||||
M(QueryMemoryLimitExceeded, "Number of times when memory limit exceeded for query.") \
|
||||
\
|
||||
M(SleepFunctionCalls, "Number of times a sleep function (sleep, sleepEachRow) has been called.") \
|
||||
M(SleepFunctionMicroseconds, "Time spent sleeping due to a sleep function call.") \
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -62,6 +62,8 @@ void ExternalResultDescription::init(const Block & sample_block_)
|
||||
types.emplace_back(ValueType::vtString, is_nullable);
|
||||
else if (which.isDate())
|
||||
types.emplace_back(ValueType::vtDate, is_nullable);
|
||||
else if (which.isDate32())
|
||||
types.emplace_back(ValueType::vtDate32, is_nullable);
|
||||
else if (which.isDateTime())
|
||||
types.emplace_back(ValueType::vtDateTime, is_nullable);
|
||||
else if (which.isUUID())
|
||||
|
@ -26,6 +26,7 @@ struct ExternalResultDescription
|
||||
vtEnum16,
|
||||
vtString,
|
||||
vtDate,
|
||||
vtDate32,
|
||||
vtDateTime,
|
||||
vtUUID,
|
||||
vtDateTime64,
|
||||
|
@ -455,6 +455,16 @@ inline void writeText(const Null &, WriteBuffer & buf)
|
||||
writeText(std::string("NULL"), buf);
|
||||
}
|
||||
|
||||
inline void writeText(const NegativeInfinity &, WriteBuffer & buf)
|
||||
{
|
||||
writeText(std::string("-Inf"), buf);
|
||||
}
|
||||
|
||||
inline void writeText(const PositiveInfinity &, WriteBuffer & buf)
|
||||
{
|
||||
writeText(std::string("+Inf"), buf);
|
||||
}
|
||||
|
||||
String toString(const Field & x)
|
||||
{
|
||||
return Field::dispatch(
|
||||
|
@ -218,6 +218,8 @@ template <> struct NearestFieldTypeImpl<Tuple> { using Type = Tuple; };
|
||||
template <> struct NearestFieldTypeImpl<Map> { using Type = Map; };
|
||||
template <> struct NearestFieldTypeImpl<bool> { using Type = UInt64; };
|
||||
template <> struct NearestFieldTypeImpl<Null> { using Type = Null; };
|
||||
template <> struct NearestFieldTypeImpl<NegativeInfinity> { using Type = NegativeInfinity; };
|
||||
template <> struct NearestFieldTypeImpl<PositiveInfinity> { using Type = PositiveInfinity; };
|
||||
|
||||
template <> struct NearestFieldTypeImpl<AggregateFunctionStateData> { using Type = AggregateFunctionStateData; };
|
||||
|
||||
@ -269,6 +271,10 @@ public:
|
||||
Int256 = 25,
|
||||
Map = 26,
|
||||
UUID = 27,
|
||||
|
||||
// Special types for index analysis
|
||||
NegativeInfinity = 254,
|
||||
PositiveInfinity = 255,
|
||||
};
|
||||
|
||||
static const char * toString(Which which)
|
||||
@ -276,6 +282,8 @@ public:
|
||||
switch (which)
|
||||
{
|
||||
case Null: return "Null";
|
||||
case NegativeInfinity: return "-Inf";
|
||||
case PositiveInfinity: return "+Inf";
|
||||
case UInt64: return "UInt64";
|
||||
case UInt128: return "UInt128";
|
||||
case UInt256: return "UInt256";
|
||||
@ -404,7 +412,10 @@ public:
|
||||
Types::Which getType() const { return which; }
|
||||
const char * getTypeName() const { return Types::toString(which); }
|
||||
|
||||
bool isNull() const { return which == Types::Null; }
|
||||
// Non-valued field are all denoted as Null
|
||||
bool isNull() const { return which == Types::Null || which == Types::NegativeInfinity || which == Types::PositiveInfinity; }
|
||||
bool isNegativeInfinity() const { return which == Types::NegativeInfinity; }
|
||||
bool isPositiveInfinity() const { return which == Types::PositiveInfinity; }
|
||||
|
||||
|
||||
template <typename T>
|
||||
@ -459,7 +470,10 @@ public:
|
||||
|
||||
switch (which)
|
||||
{
|
||||
case Types::Null: return false;
|
||||
case Types::Null:
|
||||
case Types::NegativeInfinity:
|
||||
case Types::PositiveInfinity:
|
||||
return false;
|
||||
case Types::UInt64: return get<UInt64>() < rhs.get<UInt64>();
|
||||
case Types::UInt128: return get<UInt128>() < rhs.get<UInt128>();
|
||||
case Types::UInt256: return get<UInt256>() < rhs.get<UInt256>();
|
||||
@ -496,7 +510,10 @@ public:
|
||||
|
||||
switch (which)
|
||||
{
|
||||
case Types::Null: return true;
|
||||
case Types::Null:
|
||||
case Types::NegativeInfinity:
|
||||
case Types::PositiveInfinity:
|
||||
return true;
|
||||
case Types::UInt64: return get<UInt64>() <= rhs.get<UInt64>();
|
||||
case Types::UInt128: return get<UInt128>() <= rhs.get<UInt128>();
|
||||
case Types::UInt256: return get<UInt256>() <= rhs.get<UInt256>();
|
||||
@ -533,8 +550,11 @@ public:
|
||||
|
||||
switch (which)
|
||||
{
|
||||
case Types::Null: return true;
|
||||
case Types::UInt64: return get<UInt64>() == rhs.get<UInt64>();
|
||||
case Types::Null:
|
||||
case Types::NegativeInfinity:
|
||||
case Types::PositiveInfinity:
|
||||
return true;
|
||||
case Types::UInt64: return get<UInt64>() == rhs.get<UInt64>();
|
||||
case Types::Int64: return get<Int64>() == rhs.get<Int64>();
|
||||
case Types::Float64:
|
||||
{
|
||||
@ -573,6 +593,8 @@ public:
|
||||
switch (field.which)
|
||||
{
|
||||
case Types::Null: return f(field.template get<Null>());
|
||||
case Types::NegativeInfinity: return f(field.template get<NegativeInfinity>());
|
||||
case Types::PositiveInfinity: return f(field.template get<PositiveInfinity>());
|
||||
// gcc 8.2.1
|
||||
#if !defined(__clang__)
|
||||
#pragma GCC diagnostic push
|
||||
@ -731,6 +753,8 @@ using Row = std::vector<Field>;
|
||||
|
||||
|
||||
template <> struct Field::TypeToEnum<Null> { static const Types::Which value = Types::Null; };
|
||||
template <> struct Field::TypeToEnum<NegativeInfinity> { static const Types::Which value = Types::NegativeInfinity; };
|
||||
template <> struct Field::TypeToEnum<PositiveInfinity> { static const Types::Which value = Types::PositiveInfinity; };
|
||||
template <> struct Field::TypeToEnum<UInt64> { static const Types::Which value = Types::UInt64; };
|
||||
template <> struct Field::TypeToEnum<UInt128> { static const Types::Which value = Types::UInt128; };
|
||||
template <> struct Field::TypeToEnum<UInt256> { static const Types::Which value = Types::UInt256; };
|
||||
@ -751,6 +775,8 @@ template <> struct Field::TypeToEnum<DecimalField<DateTime64>>{ static const Typ
|
||||
template <> struct Field::TypeToEnum<AggregateFunctionStateData>{ static const Types::Which value = Types::AggregateFunctionState; };
|
||||
|
||||
template <> struct Field::EnumToType<Field::Types::Null> { using Type = Null; };
|
||||
template <> struct Field::EnumToType<Field::Types::NegativeInfinity> { using Type = NegativeInfinity; };
|
||||
template <> struct Field::EnumToType<Field::Types::PositiveInfinity> { using Type = PositiveInfinity; };
|
||||
template <> struct Field::EnumToType<Field::Types::UInt64> { using Type = UInt64; };
|
||||
template <> struct Field::EnumToType<Field::Types::UInt128> { using Type = UInt128; };
|
||||
template <> struct Field::EnumToType<Field::Types::UInt256> { using Type = UInt256; };
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <sparsehash/dense_hash_map>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -161,18 +162,24 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const
|
||||
|
||||
NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const
|
||||
{
|
||||
std::unordered_map<std::string_view, const NameAndTypePair *> self_columns;
|
||||
/// NOTE: It's better to make a map in `IStorage` than to create it here every time again.
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
google::dense_hash_map<StringRef, const DataTypePtr *, StringRefHash> types;
|
||||
#else
|
||||
google::sparsehash::dense_hash_map<StringRef, const DataTypePtr *, StringRefHash> types;
|
||||
#endif
|
||||
types.set_empty_key(StringRef());
|
||||
|
||||
for (const auto & column : *this)
|
||||
self_columns[column.name] = &column;
|
||||
types[column.name] = &column.type;
|
||||
|
||||
NamesAndTypesList res;
|
||||
for (const String & name : names)
|
||||
{
|
||||
auto it = self_columns.find(name);
|
||||
if (it == self_columns.end())
|
||||
auto it = types.find(name);
|
||||
if (it == types.end())
|
||||
throw Exception("No column " + name, ErrorCodes::THERE_IS_NO_COLUMN);
|
||||
res.emplace_back(*it->second);
|
||||
res.emplace_back(name, *it->second);
|
||||
}
|
||||
|
||||
return res;
|
||||
|
@ -90,6 +90,9 @@ void insertPostgreSQLValue(
|
||||
case ExternalResultDescription::ValueType::vtDate:
|
||||
assert_cast<ColumnUInt16 &>(column).insertValue(UInt16{LocalDate{std::string(value)}.getDayNum()});
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtDate32:
|
||||
assert_cast<ColumnInt32 &>(column).insertValue(Int32{LocalDate{std::string(value)}.getExtenedDayNum()});
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtDateTime:
|
||||
{
|
||||
ReadBufferFromString in(value);
|
||||
|
@ -7,7 +7,6 @@
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Core/ExternalResultDescription.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
|
@ -57,7 +57,7 @@ class IColumn;
|
||||
M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
|
||||
M(Milliseconds, hedged_connection_timeout_ms, DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS, "Connection timeout for establishing connection with replica for Hedged requests", 0) \
|
||||
M(Milliseconds, receive_data_timeout_ms, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_MS, "Connection timeout for receiving first packet of data or packet with positive progress from replica", 0) \
|
||||
M(Bool, use_hedged_requests, false, "Use hedged requests for distributed queries", 0) \
|
||||
M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \
|
||||
M(Bool, allow_changing_replica_until_first_data_packet, false, "Allow HedgedConnections to change replica until receiving first data packet", 0) \
|
||||
M(Milliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \
|
||||
M(Milliseconds, connection_pool_max_wait_ms, 0, "The wait time when the connection pool is full.", 0) \
|
||||
|
@ -14,6 +14,8 @@ namespace DB
|
||||
/// Data types for representing elementary values from a database in RAM.
|
||||
|
||||
struct Null {};
|
||||
struct NegativeInfinity {};
|
||||
struct PositiveInfinity {};
|
||||
|
||||
/// Ignore strange gcc warning https://gcc.gnu.org/bugzilla/show_bug.cgi?id=55776
|
||||
#if !defined(__clang__)
|
||||
@ -39,6 +41,7 @@ enum class TypeIndex
|
||||
Float32,
|
||||
Float64,
|
||||
Date,
|
||||
Date32,
|
||||
DateTime,
|
||||
DateTime64,
|
||||
String,
|
||||
@ -257,6 +260,7 @@ inline constexpr const char * getTypeName(TypeIndex idx)
|
||||
case TypeIndex::Float32: return "Float32";
|
||||
case TypeIndex::Float64: return "Float64";
|
||||
case TypeIndex::Date: return "Date";
|
||||
case TypeIndex::Date32: return "Date32";
|
||||
case TypeIndex::DateTime: return "DateTime";
|
||||
case TypeIndex::DateTime64: return "DateTime64";
|
||||
case TypeIndex::String: return "String";
|
||||
|
@ -73,6 +73,7 @@ bool callOnBasicType(TypeIndex number, F && f)
|
||||
switch (number)
|
||||
{
|
||||
case TypeIndex::Date: return f(TypePair<T, UInt16>());
|
||||
case TypeIndex::Date32: return f(TypePair<T, Int32>());
|
||||
case TypeIndex::DateTime: return f(TypePair<T, UInt32>());
|
||||
case TypeIndex::DateTime64: return f(TypePair<T, DateTime64>());
|
||||
default:
|
||||
@ -142,6 +143,7 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f)
|
||||
switch (type_num1)
|
||||
{
|
||||
case TypeIndex::Date: return callOnBasicType<UInt16, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
|
||||
case TypeIndex::Date32: return callOnBasicType<Int32, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
|
||||
case TypeIndex::DateTime: return callOnBasicType<UInt32, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
|
||||
case TypeIndex::DateTime64: return callOnBasicType<DateTime64, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
|
||||
default:
|
||||
@ -154,6 +156,7 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f)
|
||||
|
||||
|
||||
class DataTypeDate;
|
||||
class DataTypeDate32;
|
||||
class DataTypeString;
|
||||
class DataTypeFixedString;
|
||||
class DataTypeUUID;
|
||||
@ -192,6 +195,7 @@ bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... args)
|
||||
case TypeIndex::Decimal256: return f(TypePair<DataTypeDecimal<Decimal256>, T>(), std::forward<ExtraArgs>(args)...);
|
||||
|
||||
case TypeIndex::Date: return f(TypePair<DataTypeDate, T>(), std::forward<ExtraArgs>(args)...);
|
||||
case TypeIndex::Date32: return f(TypePair<DataTypeDate32, T>(), std::forward<ExtraArgs>(args)...);
|
||||
case TypeIndex::DateTime: return f(TypePair<DataTypeDateTime, T>(), std::forward<ExtraArgs>(args)...);
|
||||
case TypeIndex::DateTime64: return f(TypePair<DataTypeDateTime64, T>(), std::forward<ExtraArgs>(args)...);
|
||||
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
@ -28,12 +27,6 @@ std::ostream & operator<< <Field>(std::ostream & stream, const Field & what)
|
||||
return stream;
|
||||
}
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const IBlockInputStream & what)
|
||||
{
|
||||
stream << "IBlockInputStream(name = " << what.getName() << ")";
|
||||
return stream;
|
||||
}
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const NameAndTypePair & what)
|
||||
{
|
||||
stream << "NameAndTypePair(name = " << what.name << ", type = " << what.type << ")";
|
||||
|
@ -10,9 +10,6 @@ class Field;
|
||||
template <typename T, typename U = std::enable_if_t<std::is_same_v<T, Field>>>
|
||||
std::ostream & operator<<(std::ostream & stream, const T & what);
|
||||
|
||||
class IBlockInputStream;
|
||||
std::ostream & operator<<(std::ostream & stream, const IBlockInputStream & what);
|
||||
|
||||
struct NameAndTypePair;
|
||||
std::ostream & operator<<(std::ostream & stream, const NameAndTypePair & what);
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
|
@ -11,7 +11,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License. */
|
||||
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Processors/Sources/SourceWithProgress.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
|
||||
|
||||
|
@ -25,7 +25,6 @@ namespace ErrorCodes
|
||||
extern const int TOO_MANY_BYTES;
|
||||
extern const int TOO_MANY_ROWS_OR_BYTES;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int TOO_DEEP_PIPELINE;
|
||||
}
|
||||
|
||||
|
||||
@ -357,74 +356,4 @@ Block IBlockInputStream::getExtremes()
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
String IBlockInputStream::getTreeID() const
|
||||
{
|
||||
WriteBufferFromOwnString s;
|
||||
s << getName();
|
||||
|
||||
if (!children.empty())
|
||||
{
|
||||
s << "(";
|
||||
for (BlockInputStreams::const_iterator it = children.begin(); it != children.end(); ++it)
|
||||
{
|
||||
if (it != children.begin())
|
||||
s << ", ";
|
||||
s << (*it)->getTreeID();
|
||||
}
|
||||
s << ")";
|
||||
}
|
||||
|
||||
return s.str();
|
||||
}
|
||||
|
||||
|
||||
size_t IBlockInputStream::checkDepthImpl(size_t max_depth, size_t level) const
|
||||
{
|
||||
if (children.empty())
|
||||
return 0;
|
||||
|
||||
if (level > max_depth)
|
||||
throw Exception("Query pipeline is too deep. Maximum: " + toString(max_depth), ErrorCodes::TOO_DEEP_PIPELINE);
|
||||
|
||||
size_t res = 0;
|
||||
for (const auto & child : children)
|
||||
{
|
||||
size_t child_depth = child->checkDepth(level + 1);
|
||||
if (child_depth > res)
|
||||
res = child_depth;
|
||||
}
|
||||
|
||||
return res + 1;
|
||||
}
|
||||
|
||||
|
||||
void IBlockInputStream::dumpTree(WriteBuffer & ostr, size_t indent, size_t multiplier) const
|
||||
{
|
||||
ostr << String(indent, ' ') << getName();
|
||||
if (multiplier > 1)
|
||||
ostr << " × " << multiplier;
|
||||
//ostr << ": " << getHeader().dumpStructure();
|
||||
ostr << '\n';
|
||||
++indent;
|
||||
|
||||
/// If the subtree is repeated several times, then we output it once with the multiplier.
|
||||
using Multipliers = std::map<String, size_t>;
|
||||
Multipliers multipliers;
|
||||
|
||||
for (const auto & child : children)
|
||||
++multipliers[child->getTreeID()];
|
||||
|
||||
for (const auto & child : children)
|
||||
{
|
||||
String id = child->getTreeID();
|
||||
size_t & subtree_multiplier = multipliers[id];
|
||||
if (subtree_multiplier != 0) /// Already printed subtrees are marked with zero in the array of multipliers.
|
||||
{
|
||||
child->dumpTree(ostr, indent, subtree_multiplier);
|
||||
subtree_multiplier = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -23,15 +23,6 @@ namespace ErrorCodes
|
||||
class ProcessListElement;
|
||||
class EnabledQuota;
|
||||
class QueryStatus;
|
||||
struct SortColumnDescription;
|
||||
using SortDescription = std::vector<SortColumnDescription>;
|
||||
|
||||
/** Callback to track the progress of the query.
|
||||
* Used in IBlockInputStream and Context.
|
||||
* The function takes the number of rows in the last block, the number of bytes in the last block.
|
||||
* Note that the callback can be called from different threads.
|
||||
*/
|
||||
using ProgressCallback = std::function<void(const Progress & progress)>;
|
||||
|
||||
|
||||
/** The stream interface for reading data by blocks from the database.
|
||||
@ -93,15 +84,6 @@ public:
|
||||
*/
|
||||
virtual void readSuffix();
|
||||
|
||||
/// Must be called before `read()` and `readPrefix()`.
|
||||
void dumpTree(WriteBuffer & ostr, size_t indent = 0, size_t multiplier = 1) const;
|
||||
|
||||
/** Check the depth of the pipeline.
|
||||
* If max_depth is specified and the `depth` is greater - throw an exception.
|
||||
* Must be called before `read()` and `readPrefix()`.
|
||||
*/
|
||||
size_t checkDepth(size_t max_depth) const { return checkDepthImpl(max_depth, max_depth); }
|
||||
|
||||
/// Do not allow to change the table while the blocks stream and its children are alive.
|
||||
void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); }
|
||||
|
||||
@ -269,9 +251,6 @@ private:
|
||||
|
||||
size_t checkDepthImpl(size_t max_depth, size_t level) const;
|
||||
|
||||
/// Get text with names of this source and the entire subtree.
|
||||
String getTreeID() const;
|
||||
|
||||
template <typename F>
|
||||
void forEachChild(F && f)
|
||||
{
|
||||
|
@ -1,80 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Initialize another source on the first `read` call, and then use it.
|
||||
* This is needed, for example, to read from a table that will be populated
|
||||
* after creation of LazyBlockInputStream object, but before the first `read` call.
|
||||
*/
|
||||
class LazyBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
using Generator = std::function<BlockInputStreamPtr()>;
|
||||
|
||||
LazyBlockInputStream(const Block & header_, Generator generator_)
|
||||
: header(header_), generator(std::move(generator_))
|
||||
{
|
||||
}
|
||||
|
||||
LazyBlockInputStream(const char * name_, const Block & header_, Generator generator_)
|
||||
: name(name_), header(header_), generator(std::move(generator_))
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
Block getHeader() const override
|
||||
{
|
||||
return header;
|
||||
}
|
||||
|
||||
/// We call readPrefix lazily. Suppress default behaviour.
|
||||
void readPrefix() override {}
|
||||
|
||||
protected:
|
||||
Block readImpl() override
|
||||
{
|
||||
if (!input)
|
||||
{
|
||||
input = generator();
|
||||
|
||||
if (!input)
|
||||
return Block();
|
||||
|
||||
auto * p_input = dynamic_cast<IBlockInputStream *>(input.get());
|
||||
|
||||
if (p_input)
|
||||
{
|
||||
/// They could have been set before, but were not passed into the `input`.
|
||||
if (progress_callback)
|
||||
p_input->setProgressCallback(progress_callback);
|
||||
if (process_list_elem)
|
||||
p_input->setProcessListElement(process_list_elem);
|
||||
}
|
||||
|
||||
input->readPrefix();
|
||||
|
||||
{
|
||||
addChild(input);
|
||||
|
||||
if (isCancelled() && p_input)
|
||||
p_input->cancel(is_killed);
|
||||
}
|
||||
}
|
||||
|
||||
return input->read();
|
||||
}
|
||||
|
||||
private:
|
||||
const char * name = "Lazy";
|
||||
Block header;
|
||||
Generator generator;
|
||||
|
||||
BlockInputStreamPtr input;
|
||||
};
|
||||
|
||||
}
|
@ -1,24 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Empty stream of blocks of specified structure.
|
||||
class NullBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
NullBlockInputStream(const Block & header_) : header(header_) {}
|
||||
|
||||
Block getHeader() const override { return header; }
|
||||
String getName() const override { return "Null"; }
|
||||
|
||||
private:
|
||||
Block header;
|
||||
|
||||
Block readImpl() override { return {}; }
|
||||
};
|
||||
|
||||
}
|
@ -8,7 +8,6 @@
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <cstddef>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
23
src/DataTypes/DataTypeDate32.cpp
Normal file
23
src/DataTypes/DataTypeDate32.cpp
Normal file
@ -0,0 +1,23 @@
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationDate32.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
bool DataTypeDate32::equals(const IDataType & rhs) const
|
||||
{
|
||||
return typeid(rhs) == typeid(*this);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeDate32::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationDate32>();
|
||||
}
|
||||
|
||||
void registerDataTypeDate32(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType(
|
||||
"Date32", [] { return DataTypePtr(std::make_shared<DataTypeDate32>()); }, DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
29
src/DataTypes/DataTypeDate32.h
Normal file
29
src/DataTypes/DataTypeDate32.h
Normal file
@ -0,0 +1,29 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
#include <common/DateLUT.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class DataTypeDate32 final : public DataTypeNumberBase<Int32>
|
||||
{
|
||||
public:
|
||||
static constexpr auto family_name = "Date32";
|
||||
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Date32; }
|
||||
const char * getFamilyName() const override { return family_name; }
|
||||
|
||||
Field getDefault() const override
|
||||
{
|
||||
return -static_cast<Int64>(DateLUT::instance().getDayNumOffsetEpoch());
|
||||
}
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
protected:
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
}
|
@ -194,6 +194,7 @@ DataTypeFactory::DataTypeFactory()
|
||||
registerDataTypeNumbers(*this);
|
||||
registerDataTypeDecimal(*this);
|
||||
registerDataTypeDate(*this);
|
||||
registerDataTypeDate32(*this);
|
||||
registerDataTypeDateTime(*this);
|
||||
registerDataTypeString(*this);
|
||||
registerDataTypeFixedString(*this);
|
||||
|
@ -69,6 +69,7 @@ private:
|
||||
void registerDataTypeNumbers(DataTypeFactory & factory);
|
||||
void registerDataTypeDecimal(DataTypeFactory & factory);
|
||||
void registerDataTypeDate(DataTypeFactory & factory);
|
||||
void registerDataTypeDate32(DataTypeFactory & factory);
|
||||
void registerDataTypeDateTime(DataTypeFactory & factory);
|
||||
void registerDataTypeString(DataTypeFactory & factory);
|
||||
void registerDataTypeFixedString(DataTypeFactory & factory);
|
||||
|
@ -78,6 +78,8 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat
|
||||
return creator(static_cast<ColumnFixedString *>(nullptr));
|
||||
else if (which.isDate())
|
||||
return creator(static_cast<ColumnVector<UInt16> *>(nullptr));
|
||||
else if (which.isDate32())
|
||||
return creator(static_cast<ColumnVector<Int32> *>(nullptr));
|
||||
else if (which.isDateTime())
|
||||
return creator(static_cast<ColumnVector<UInt32> *>(nullptr));
|
||||
else if (which.isUUID())
|
||||
|
@ -19,6 +19,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int EMPTY_DATA_PASSED;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -27,6 +28,16 @@ DataTypePtr FieldToDataType::operator() (const Null &) const
|
||||
return std::make_shared<DataTypeNullable>(std::make_shared<DataTypeNothing>());
|
||||
}
|
||||
|
||||
DataTypePtr FieldToDataType::operator() (const NegativeInfinity &) const
|
||||
{
|
||||
throw Exception("It's invalid to have -inf literals in SQL", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
DataTypePtr FieldToDataType::operator() (const PositiveInfinity &) const
|
||||
{
|
||||
throw Exception("It's invalid to have +inf literals in SQL", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
DataTypePtr FieldToDataType::operator() (const UInt64 & x) const
|
||||
{
|
||||
if (x <= std::numeric_limits<UInt8>::max()) return std::make_shared<DataTypeUInt8>();
|
||||
|
@ -21,6 +21,8 @@ class FieldToDataType : public StaticVisitor<DataTypePtr>
|
||||
{
|
||||
public:
|
||||
DataTypePtr operator() (const Null & x) const;
|
||||
DataTypePtr operator() (const NegativeInfinity & x) const;
|
||||
DataTypePtr operator() (const PositiveInfinity & x) const;
|
||||
DataTypePtr operator() (const UInt64 & x) const;
|
||||
DataTypePtr operator() (const UInt128 & x) const;
|
||||
DataTypePtr operator() (const UInt256 & x) const;
|
||||
|
@ -322,8 +322,10 @@ struct WhichDataType
|
||||
constexpr bool isEnum() const { return isEnum8() || isEnum16(); }
|
||||
|
||||
constexpr bool isDate() const { return idx == TypeIndex::Date; }
|
||||
constexpr bool isDate32() const { return idx == TypeIndex::Date32; }
|
||||
constexpr bool isDateTime() const { return idx == TypeIndex::DateTime; }
|
||||
constexpr bool isDateTime64() const { return idx == TypeIndex::DateTime64; }
|
||||
constexpr bool isDateOrDate32() const { return isDate() || isDate32(); }
|
||||
|
||||
constexpr bool isString() const { return idx == TypeIndex::String; }
|
||||
constexpr bool isFixedString() const { return idx == TypeIndex::FixedString; }
|
||||
@ -347,6 +349,10 @@ struct WhichDataType
|
||||
template <typename T>
|
||||
inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); }
|
||||
template <typename T>
|
||||
inline bool isDate32(const T & data_type) { return WhichDataType(data_type).isDate32(); }
|
||||
template <typename T>
|
||||
inline bool isDateOrDate32(const T & data_type) { return WhichDataType(data_type).isDateOrDate32(); }
|
||||
template <typename T>
|
||||
inline bool isDateTime(const T & data_type) { return WhichDataType(data_type).isDateTime(); }
|
||||
template <typename T>
|
||||
inline bool isDateTime64(const T & data_type) { return WhichDataType(data_type).isDateTime64(); }
|
||||
@ -411,7 +417,7 @@ template <typename T>
|
||||
inline bool isColumnedAsNumber(const T & data_type)
|
||||
{
|
||||
WhichDataType which(data_type);
|
||||
return which.isInt() || which.isUInt() || which.isFloat() || which.isDate() || which.isDateTime() || which.isDateTime64() || which.isUUID();
|
||||
return which.isInt() || which.isUInt() || which.isFloat() || which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() || which.isUUID();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -478,6 +484,7 @@ template <typename T>
|
||||
class DataTypeNumber;
|
||||
|
||||
class DataTypeDate;
|
||||
class DataTypeDate32;
|
||||
class DataTypeDateTime;
|
||||
class DataTypeDateTime64;
|
||||
|
||||
@ -487,6 +494,7 @@ template <> inline constexpr bool IsDataTypeDecimal<DataTypeDateTime64> = true;
|
||||
template <typename T> constexpr bool IsDataTypeNumber<DataTypeNumber<T>> = true;
|
||||
|
||||
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDate> = true;
|
||||
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDate32> = true;
|
||||
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime> = true;
|
||||
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime64> = true;
|
||||
|
||||
|
78
src/DataTypes/Serializations/SerializationDate32.cpp
Normal file
78
src/DataTypes/Serializations/SerializationDate32.cpp
Normal file
@ -0,0 +1,78 @@
|
||||
#include <DataTypes/Serializations/SerializationDate32.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
void SerializationDate32::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
writeDateText(ExtendedDayNum(assert_cast<const ColumnInt32 &>(column).getData()[row_num]), ostr);
|
||||
}
|
||||
|
||||
void SerializationDate32::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextEscaped(column, istr, settings);
|
||||
}
|
||||
|
||||
void SerializationDate32::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ExtendedDayNum x;
|
||||
readDateText(x, istr);
|
||||
assert_cast<ColumnInt32 &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
void SerializationDate32::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationDate32::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('\'', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('\'', ostr);
|
||||
}
|
||||
|
||||
void SerializationDate32::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ExtendedDayNum x;
|
||||
assertChar('\'', istr);
|
||||
readDateText(x, istr);
|
||||
assertChar('\'', istr);
|
||||
assert_cast<ColumnInt32 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
|
||||
}
|
||||
|
||||
void SerializationDate32::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('"', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void SerializationDate32::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ExtendedDayNum x;
|
||||
assertChar('"', istr);
|
||||
readDateText(x, istr);
|
||||
assertChar('"', istr);
|
||||
assert_cast<ColumnInt32 &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
void SerializationDate32::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('"', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void SerializationDate32::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
LocalDate value;
|
||||
readCSV(value, istr);
|
||||
assert_cast<ColumnInt32 &>(column).getData().push_back(value.getExtenedDayNum());
|
||||
}
|
||||
}
|
21
src/DataTypes/Serializations/SerializationDate32.h
Normal file
21
src/DataTypes/Serializations/SerializationDate32.h
Normal file
@ -0,0 +1,21 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class SerializationDate32 final : public SerializationNumber<Int32>
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
};
|
||||
}
|
@ -1,8 +1,11 @@
|
||||
#include <DataTypes/Serializations/SerializationIP.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/formatIPv6.h>
|
||||
#include <Functions/FunctionsCoding.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -16,6 +16,7 @@ SRCS(
|
||||
DataTypeCustomIPv4AndIPv6.cpp
|
||||
DataTypeCustomSimpleAggregateFunction.cpp
|
||||
DataTypeDate.cpp
|
||||
DataTypeDate32.cpp
|
||||
DataTypeDateTime.cpp
|
||||
DataTypeDateTime64.cpp
|
||||
DataTypeDecimalBase.cpp
|
||||
@ -45,6 +46,7 @@ SRCS(
|
||||
Serializations/SerializationArray.cpp
|
||||
Serializations/SerializationCustomSimpleText.cpp
|
||||
Serializations/SerializationDate.cpp
|
||||
Serializations/SerializationDate32.cpp
|
||||
Serializations/SerializationDateTime.cpp
|
||||
Serializations/SerializationDateTime64.cpp
|
||||
Serializations/SerializationDecimal.cpp
|
||||
|
@ -6,13 +6,13 @@
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/Names.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include "DictionaryBlockInputStreamBase.h"
|
||||
#include "DictionaryStructure.h"
|
||||
#include "IDictionary.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -9,13 +9,14 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Dictionaries/IDictionary.h>
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -2,13 +2,13 @@
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
#include <Dictionaries/HierarchyDictionariesUtils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <common/range.h>
|
||||
@ -11,6 +10,7 @@
|
||||
#include "IDictionary.h"
|
||||
#include "RangeHashedDictionary.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/*
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <IO/FileEncryptionCommon.h>
|
||||
#include <IO/ReadBufferFromEncryptedFile.h>
|
||||
#include <IO/WriteBufferFromEncryptedFile.h>
|
||||
#include <boost/algorithm/hex.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,13 +13,87 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int INCORRECT_DISK_INDEX;
|
||||
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
using DiskEncryptedPtr = std::shared_ptr<DiskEncrypted>;
|
||||
using namespace FileEncryption;
|
||||
namespace
|
||||
{
|
||||
using DiskEncryptedPtr = std::shared_ptr<DiskEncrypted>;
|
||||
using namespace FileEncryption;
|
||||
|
||||
constexpr Algorithm DEFAULT_ENCRYPTION_ALGORITHM = Algorithm::AES_128_CTR;
|
||||
|
||||
String unhexKey(const String & hex)
|
||||
{
|
||||
try
|
||||
{
|
||||
return boost::algorithm::unhex(hex);
|
||||
}
|
||||
catch (const std::exception &)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read key_hex, check for valid characters [0-9a-fA-F] and length");
|
||||
}
|
||||
}
|
||||
|
||||
struct DiskEncryptedSettings
|
||||
{
|
||||
Algorithm encryption_algorithm;
|
||||
String key;
|
||||
DiskPtr wrapped_disk;
|
||||
String path_on_wrapped_disk;
|
||||
|
||||
DiskEncryptedSettings(
|
||||
const String & disk_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const DisksMap & map)
|
||||
{
|
||||
try
|
||||
{
|
||||
encryption_algorithm = DEFAULT_ENCRYPTION_ALGORITHM;
|
||||
if (config.has(config_prefix + ".algorithm"))
|
||||
parseFromString(encryption_algorithm, config.getString(config_prefix + ".algorithm"));
|
||||
|
||||
key = config.getString(config_prefix + ".key", "");
|
||||
String key_hex = config.getString(config_prefix + ".key_hex", "");
|
||||
if (!key.empty() && !key_hex.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Both 'key' and 'key_hex' are specified. There should be only one");
|
||||
|
||||
if (!key_hex.empty())
|
||||
{
|
||||
assert(key.empty());
|
||||
key = unhexKey(key_hex);
|
||||
}
|
||||
|
||||
FileEncryption::checkKeySize(encryption_algorithm, key.size());
|
||||
|
||||
String wrapped_disk_name = config.getString(config_prefix + ".disk", "");
|
||||
if (wrapped_disk_name.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Name of the wrapped disk must not be empty. An encrypted disk is a wrapper over another disk");
|
||||
|
||||
auto wrapped_disk_it = map.find(wrapped_disk_name);
|
||||
if (wrapped_disk_it == map.end())
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"The wrapped disk must have been announced earlier. No disk with name {}",
|
||||
wrapped_disk_name);
|
||||
wrapped_disk = wrapped_disk_it->second;
|
||||
|
||||
path_on_wrapped_disk = config.getString(config_prefix + ".path", "");
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("Disk " + disk_name);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
bool inline isSameDiskType(const IDisk & one, const IDisk & another)
|
||||
{
|
||||
return typeid(one) == typeid(another);
|
||||
}
|
||||
}
|
||||
|
||||
class DiskEncryptedReservation : public IReservation
|
||||
{
|
||||
@ -54,26 +129,52 @@ ReservationPtr DiskEncrypted::reserve(UInt64 bytes)
|
||||
return std::make_unique<DiskEncryptedReservation>(std::static_pointer_cast<DiskEncrypted>(shared_from_this()), std::move(reservation));
|
||||
}
|
||||
|
||||
DiskEncrypted::DiskEncrypted(const String & name_, DiskPtr disk_, const String & key_, const String & path_)
|
||||
: DiskDecorator(disk_)
|
||||
, name(name_), key(key_), disk_path(path_)
|
||||
, disk_absolute_path(delegate->getPath() + disk_path)
|
||||
DiskEncrypted::DiskEncrypted(
|
||||
const String & name_,
|
||||
DiskPtr wrapped_disk_,
|
||||
const String & path_on_wrapped_disk_,
|
||||
FileEncryption::Algorithm encryption_algorithm_,
|
||||
const String & key_)
|
||||
: DiskDecorator(wrapped_disk_), name(name_), disk_path(path_on_wrapped_disk_), encryption_algorithm(encryption_algorithm_), key(key_)
|
||||
{
|
||||
initialize();
|
||||
}
|
||||
|
||||
void DiskEncrypted::initialize()
|
||||
{
|
||||
disk_absolute_path = delegate->getPath() + disk_path;
|
||||
|
||||
// use wrapped_disk as an EncryptedDisk store
|
||||
if (disk_path.empty())
|
||||
return;
|
||||
|
||||
if (disk_path.back() != '/')
|
||||
throw Exception("Disk path must ends with '/', but '" + disk_path + "' doesn't.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Disk path must ends with '/', but '" + disk_path + "' doesn't.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
delegate->createDirectories(disk_path);
|
||||
}
|
||||
|
||||
void DiskEncrypted::copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path)
|
||||
{
|
||||
/// Check if we can copy the file without deciphering.
|
||||
if (isSameDiskType(*this, *to_disk))
|
||||
{
|
||||
/// Disk type is the same, check if the key is the same too.
|
||||
if (auto * to_encrypted_disk = typeid_cast<DiskEncrypted *>(to_disk.get()))
|
||||
{
|
||||
if ((encryption_algorithm == to_encrypted_disk->encryption_algorithm) && (key == to_encrypted_disk->key))
|
||||
{
|
||||
/// Key is the same so we can simply copy the encrypted file.
|
||||
delegate->copy(wrappedPath(from_path), to_encrypted_disk->delegate, to_encrypted_disk->wrappedPath(to_path));
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Copy the file through buffers with deciphering.
|
||||
copyThroughBuffers(from_path, to_disk, to_path);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
@ -85,37 +186,28 @@ std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
auto buffer = delegate->readFile(wrapped_path, buf_size, estimated_size, aio_threshold, mmap_threshold, mmap_cache);
|
||||
|
||||
String iv;
|
||||
size_t offset = 0;
|
||||
|
||||
if (exists(path) && getFileSize(path))
|
||||
{
|
||||
iv = readIV(kIVSize, *buffer);
|
||||
offset = kIVSize;
|
||||
}
|
||||
else
|
||||
iv = randomString(kIVSize);
|
||||
|
||||
return std::make_unique<ReadBufferFromEncryptedFile>(buf_size, std::move(buffer), iv, key, offset);
|
||||
InitVector iv;
|
||||
iv.read(*buffer);
|
||||
return std::make_unique<ReadBufferFromEncryptedFile>(buf_size, std::move(buffer), encryption_algorithm, key, iv);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskEncrypted::writeFile(const String & path, size_t buf_size, WriteMode mode)
|
||||
{
|
||||
String iv;
|
||||
size_t start_offset = 0;
|
||||
InitVector iv;
|
||||
UInt64 old_file_size = 0;
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
|
||||
if (mode == WriteMode::Append && exists(path) && getFileSize(path))
|
||||
{
|
||||
auto read_buffer = delegate->readFile(wrapped_path, kIVSize);
|
||||
iv = readIV(kIVSize, *read_buffer);
|
||||
start_offset = getFileSize(path);
|
||||
auto read_buffer = delegate->readFile(wrapped_path, InitVector::kSize);
|
||||
iv.read(*read_buffer);
|
||||
old_file_size = getFileSize(path);
|
||||
}
|
||||
else
|
||||
iv = randomString(kIVSize);
|
||||
iv = InitVector::random();
|
||||
|
||||
auto buffer = delegate->writeFile(wrapped_path, buf_size, mode);
|
||||
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), iv, key, start_offset);
|
||||
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), encryption_algorithm, key, iv, old_file_size);
|
||||
}
|
||||
|
||||
|
||||
@ -123,13 +215,13 @@ size_t DiskEncrypted::getFileSize(const String & path) const
|
||||
{
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
size_t size = delegate->getFileSize(wrapped_path);
|
||||
return size > kIVSize ? (size - kIVSize) : 0;
|
||||
return size > InitVector::kSize ? (size - InitVector::kSize) : 0;
|
||||
}
|
||||
|
||||
void DiskEncrypted::truncateFile(const String & path, size_t size)
|
||||
{
|
||||
auto wrapped_path = wrappedPath(path);
|
||||
delegate->truncateFile(wrapped_path, size ? (size + kIVSize) : 0);
|
||||
delegate->truncateFile(wrapped_path, size ? (size + InitVector::kSize) : 0);
|
||||
}
|
||||
|
||||
SyncGuardPtr DiskEncrypted::getDirectorySyncGuard(const String & path) const
|
||||
@ -144,22 +236,11 @@ void DiskEncrypted::applyNewSettings(
|
||||
const String & config_prefix,
|
||||
const DisksMap & map)
|
||||
{
|
||||
String wrapped_disk_name = config.getString(config_prefix + ".disk", "");
|
||||
if (wrapped_disk_name.empty())
|
||||
throw Exception("The wrapped disk name can not be empty. An encrypted disk is a wrapper over another disk. "
|
||||
"Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
key = config.getString(config_prefix + ".key", "");
|
||||
if (key.empty())
|
||||
throw Exception("Encrypted disk key can not be empty. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
auto wrapped_disk = map.find(wrapped_disk_name);
|
||||
if (wrapped_disk == map.end())
|
||||
throw Exception("The wrapped disk must have been announced earlier. No disk with name " + wrapped_disk_name + ". Disk " + name,
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
delegate = wrapped_disk->second;
|
||||
|
||||
disk_path = config.getString(config_prefix + ".path", "");
|
||||
DiskEncryptedSettings settings{name, config, config_prefix, map};
|
||||
delegate = settings.wrapped_disk;
|
||||
disk_path = settings.path_on_wrapped_disk;
|
||||
encryption_algorithm = settings.encryption_algorithm;
|
||||
key = settings.key;
|
||||
initialize();
|
||||
}
|
||||
|
||||
@ -169,28 +250,11 @@ void registerDiskEncrypted(DiskFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextPtr /*context*/,
|
||||
const DisksMap & map) -> DiskPtr {
|
||||
|
||||
String wrapped_disk_name = config.getString(config_prefix + ".disk", "");
|
||||
if (wrapped_disk_name.empty())
|
||||
throw Exception("The wrapped disk name can not be empty. An encrypted disk is a wrapper over another disk. "
|
||||
"Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
String key = config.getString(config_prefix + ".key", "");
|
||||
if (key.empty())
|
||||
throw Exception("Encrypted disk key can not be empty. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
if (key.size() != cipherKeyLength(defaultCipher()))
|
||||
throw Exception("Expected key with size " + std::to_string(cipherKeyLength(defaultCipher())) + ", got key with size " + std::to_string(key.size()),
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
auto wrapped_disk = map.find(wrapped_disk_name);
|
||||
if (wrapped_disk == map.end())
|
||||
throw Exception("The wrapped disk must have been announced earlier. No disk with name " + wrapped_disk_name + ". Disk " + name,
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
|
||||
|
||||
String relative_path = config.getString(config_prefix + ".path", "");
|
||||
|
||||
return std::make_shared<DiskEncrypted>(name, wrapped_disk->second, key, relative_path);
|
||||
const DisksMap & map) -> DiskPtr
|
||||
{
|
||||
DiskEncryptedSettings settings{name, config, config_prefix, map};
|
||||
return std::make_shared<DiskEncrypted>(
|
||||
name, settings.wrapped_disk, settings.path_on_wrapped_disk, settings.encryption_algorithm, settings.key);
|
||||
};
|
||||
factory.registerDiskType("encrypted", creator);
|
||||
}
|
||||
|
@ -13,11 +13,20 @@ namespace DB
|
||||
{
|
||||
class ReadBufferFromFileBase;
|
||||
class WriteBufferFromFileBase;
|
||||
namespace FileEncryption { enum class Algorithm; }
|
||||
|
||||
/// Encrypted disk ciphers all written files on the fly and writes the encrypted files to an underlying (normal) disk.
|
||||
/// And when we read files from an encrypted disk it deciphers them automatically,
|
||||
/// so we can work with a encrypted disk like it's a normal disk.
|
||||
class DiskEncrypted : public DiskDecorator
|
||||
{
|
||||
public:
|
||||
DiskEncrypted(const String & name_, DiskPtr disk_, const String & key_, const String & path_);
|
||||
DiskEncrypted(
|
||||
const String & name_,
|
||||
DiskPtr wrapped_disk_,
|
||||
const String & path_on_wrapped_disk_,
|
||||
FileEncryption::Algorithm encryption_algorithm_,
|
||||
const String & key_);
|
||||
|
||||
const String & getName() const override { return name; }
|
||||
const String & getPath() const override { return disk_absolute_path; }
|
||||
@ -102,10 +111,7 @@ public:
|
||||
delegate->listFiles(wrapped_path, file_names);
|
||||
}
|
||||
|
||||
void copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path) override
|
||||
{
|
||||
IDisk::copy(from_path, to_disk, to_path);
|
||||
}
|
||||
void copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path) override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
@ -219,9 +225,10 @@ private:
|
||||
}
|
||||
|
||||
String name;
|
||||
String key;
|
||||
String disk_path;
|
||||
String disk_absolute_path;
|
||||
FileEncryption::Algorithm encryption_algorithm;
|
||||
String key;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -309,7 +309,7 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr<IDisk> & to
|
||||
fs::copy(from, to, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way.
|
||||
}
|
||||
else
|
||||
IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers.
|
||||
copyThroughBuffers(from_path, to_disk, to_path); /// Base implementation.
|
||||
}
|
||||
|
||||
SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const
|
||||
|
@ -58,7 +58,7 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p
|
||||
}
|
||||
}
|
||||
|
||||
void IDisk::copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path)
|
||||
void IDisk::copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path)
|
||||
{
|
||||
auto & exec = to_disk->getExecutor();
|
||||
ResultsCollector results;
|
||||
@ -71,6 +71,11 @@ void IDisk::copy(const String & from_path, const std::shared_ptr<IDisk> & to_dis
|
||||
result.get();
|
||||
}
|
||||
|
||||
void IDisk::copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path)
|
||||
{
|
||||
copyThroughBuffers(from_path, to_disk, to_path);
|
||||
}
|
||||
|
||||
void IDisk::truncateFile(const String &, size_t)
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate operation is not implemented for disk of type {}", getType());
|
||||
|
@ -246,6 +246,11 @@ protected:
|
||||
/// Returns executor to perform asynchronous operations.
|
||||
virtual Executor & getExecutor() { return *executor; }
|
||||
|
||||
/// Base implementation of the function copy().
|
||||
/// It just opens two files, reads data by portions from the first file, and writes it to the second one.
|
||||
/// A derived class may override copy() to provide a faster implementation.
|
||||
void copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path);
|
||||
|
||||
private:
|
||||
std::unique_ptr<Executor> executor;
|
||||
};
|
||||
|
@ -1,5 +1 @@
|
||||
configure_file(config_formats.h.in ${ConfigIncludePath}/config_formats.h)
|
||||
|
||||
if (ENABLE_EXAMPLES)
|
||||
add_subdirectory(examples)
|
||||
endif()
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Processors/Formats/IRowInputFormat.h>
|
||||
#include <Processors/Formats/IRowOutputFormat.h>
|
||||
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||
#include <Processors/Formats/OutputStreamToOutputFormat.h>
|
||||
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
|
||||
#include <Processors/Formats/Impl/MySQLOutputFormat.h>
|
||||
|
@ -1,4 +0,0 @@
|
||||
set(SRCS )
|
||||
|
||||
add_executable (tab_separated_streams tab_separated_streams.cpp ${SRCS})
|
||||
target_link_libraries (tab_separated_streams PRIVATE clickhouse_aggregate_functions dbms)
|
@ -1,57 +0,0 @@
|
||||
#include <string>
|
||||
|
||||
#include <iostream>
|
||||
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
|
||||
|
||||
#include <DataStreams/copyData.h>
|
||||
#include <Processors/Formats/OutputStreamToOutputFormat.h>
|
||||
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
|
||||
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||
|
||||
|
||||
using namespace DB;
|
||||
|
||||
int main(int, char **)
|
||||
try
|
||||
{
|
||||
Block sample;
|
||||
{
|
||||
ColumnWithTypeAndName col;
|
||||
col.type = std::make_shared<DataTypeUInt64>();
|
||||
sample.insert(std::move(col));
|
||||
}
|
||||
{
|
||||
ColumnWithTypeAndName col;
|
||||
col.type = std::make_shared<DataTypeString>();
|
||||
sample.insert(std::move(col));
|
||||
}
|
||||
|
||||
ReadBufferFromFile in_buf("test_in");
|
||||
WriteBufferFromFile out_buf("test_out");
|
||||
|
||||
FormatSettings format_settings;
|
||||
|
||||
RowInputFormatParams in_params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0};
|
||||
RowOutputFormatParams out_params{[](const Columns & /* columns */, size_t /* row */){}};
|
||||
|
||||
InputFormatPtr input_format = std::make_shared<TabSeparatedRowInputFormat>(sample, in_buf, in_params, false, false, format_settings);
|
||||
BlockInputStreamPtr block_input = std::make_shared<InputStreamFromInputFormat>(std::move(input_format));
|
||||
|
||||
BlockOutputStreamPtr block_output = std::make_shared<OutputStreamToOutputFormat>(
|
||||
std::make_shared<TabSeparatedRowOutputFormat>(out_buf, sample, false, false, out_params, format_settings));
|
||||
|
||||
copyData(*block_input, *block_output);
|
||||
return 0;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
std::cerr << getCurrentExceptionMessage(true) << '\n';
|
||||
return 1;
|
||||
}
|
@ -42,6 +42,11 @@ struct ToYearWeekImpl
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
|
||||
return yw.first * 100 + yw.second;
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum (d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
|
||||
return yw.first * 100 + yw.second;
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
|
||||
@ -65,6 +70,10 @@ struct ToStartOfWeekImpl
|
||||
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
|
||||
// return time_zone.toFirstDayNumOfWeek(t, week_mode);
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
|
||||
@ -88,6 +97,11 @@ struct ToWeekImpl
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
|
@ -46,6 +46,7 @@ struct ZeroTransform
|
||||
{
|
||||
static inline UInt16 execute(Int64, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(UInt32, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(Int32, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(UInt16, const DateLUTImpl &) { return 0; }
|
||||
};
|
||||
|
||||
@ -61,6 +62,10 @@ struct ToDateImpl
|
||||
{
|
||||
return UInt16(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
|
||||
{
|
||||
return d;
|
||||
@ -69,6 +74,30 @@ struct ToDateImpl
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct ToDate32Impl
|
||||
{
|
||||
static constexpr auto name = "toDate32";
|
||||
|
||||
static inline Int32 execute(Int64 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return Int32(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline Int32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return Int32(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline Int32 execute(Int32 d, const DateLUTImpl &)
|
||||
{
|
||||
return d;
|
||||
}
|
||||
static inline Int32 execute(UInt16 d, const DateLUTImpl &)
|
||||
{
|
||||
return d;
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct ToStartOfDayImpl
|
||||
{
|
||||
static constexpr auto name = "toStartOfDay";
|
||||
@ -82,6 +111,10 @@ struct ToStartOfDayImpl
|
||||
{
|
||||
return time_zone.toDate(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDate(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDate(ExtendedDayNum(d));
|
||||
@ -104,6 +137,10 @@ struct ToMondayImpl
|
||||
//return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t));
|
||||
return time_zone.toFirstDayNumOfWeek(t);
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d));
|
||||
@ -124,6 +161,10 @@ struct ToStartOfMonthImpl
|
||||
{
|
||||
return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfMonth(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfMonth(ExtendedDayNum(d));
|
||||
@ -144,6 +185,10 @@ struct ToStartOfQuarterImpl
|
||||
{
|
||||
return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(d));
|
||||
@ -164,6 +209,10 @@ struct ToStartOfYearImpl
|
||||
{
|
||||
return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfYear(ExtendedDayNum(d));
|
||||
@ -186,7 +235,10 @@ struct ToTimeImpl
|
||||
{
|
||||
return time_zone.toTime(t) + 86400;
|
||||
}
|
||||
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -207,6 +259,10 @@ struct ToStartOfMinuteImpl
|
||||
{
|
||||
return time_zone.toStartOfMinute(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -242,6 +298,10 @@ struct ToStartOfSecondImpl
|
||||
{
|
||||
throw Exception("Illegal type DateTime of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -262,6 +322,10 @@ struct ToStartOfFiveMinuteImpl
|
||||
{
|
||||
return time_zone.toStartOfFiveMinute(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -282,6 +346,10 @@ struct ToStartOfTenMinutesImpl
|
||||
{
|
||||
return time_zone.toStartOfTenMinutes(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -302,6 +370,10 @@ struct ToStartOfFifteenMinutesImpl
|
||||
{
|
||||
return time_zone.toStartOfFifteenMinutes(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -326,6 +398,11 @@ struct TimeSlotImpl
|
||||
return t / 1800 * 1800;
|
||||
}
|
||||
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -348,6 +425,11 @@ struct ToStartOfHourImpl
|
||||
return time_zone.toStartOfHour(t);
|
||||
}
|
||||
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -368,6 +450,10 @@ struct ToYearImpl
|
||||
{
|
||||
return time_zone.toYear(t);
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(ExtendedDayNum(d));
|
||||
@ -388,6 +474,10 @@ struct ToQuarterImpl
|
||||
{
|
||||
return time_zone.toQuarter(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toQuarter(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toQuarter(ExtendedDayNum(d));
|
||||
@ -408,6 +498,10 @@ struct ToMonthImpl
|
||||
{
|
||||
return time_zone.toMonth(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toMonth(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toMonth(ExtendedDayNum(d));
|
||||
@ -428,6 +522,10 @@ struct ToDayOfMonthImpl
|
||||
{
|
||||
return time_zone.toDayOfMonth(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfMonth(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfMonth(ExtendedDayNum(d));
|
||||
@ -448,6 +546,10 @@ struct ToDayOfWeekImpl
|
||||
{
|
||||
return time_zone.toDayOfWeek(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfWeek(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfWeek(ExtendedDayNum(d));
|
||||
@ -468,6 +570,10 @@ struct ToDayOfYearImpl
|
||||
{
|
||||
return time_zone.toDayOfYear(t);
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfYear(ExtendedDayNum(d));
|
||||
@ -488,7 +594,10 @@ struct ToHourImpl
|
||||
{
|
||||
return time_zone.toHour(t);
|
||||
}
|
||||
|
||||
static inline UInt8 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt8 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -511,6 +620,11 @@ struct TimezoneOffsetImpl
|
||||
return time_zone.timezoneOffset(t);
|
||||
}
|
||||
|
||||
static inline time_t execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
|
||||
static inline time_t execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -531,6 +645,10 @@ struct ToMinuteImpl
|
||||
{
|
||||
return time_zone.toMinute(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt8 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -551,6 +669,10 @@ struct ToSecondImpl
|
||||
{
|
||||
return time_zone.toSecond(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt8 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -571,6 +693,10 @@ struct ToISOYearImpl
|
||||
{
|
||||
return time_zone.toISOYear(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOYear(ExtendedDayNum(d));
|
||||
@ -591,6 +717,10 @@ struct ToStartOfISOYearImpl
|
||||
{
|
||||
return time_zone.toFirstDayNumOfISOYear(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(d));
|
||||
@ -611,6 +741,10 @@ struct ToISOWeekImpl
|
||||
{
|
||||
return time_zone.toISOWeek(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOWeek(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOWeek(ExtendedDayNum(d));
|
||||
@ -631,6 +765,10 @@ struct ToRelativeYearNumImpl
|
||||
{
|
||||
return time_zone.toYear(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(ExtendedDayNum(d));
|
||||
@ -651,6 +789,10 @@ struct ToRelativeQuarterNumImpl
|
||||
{
|
||||
return time_zone.toRelativeQuarterNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeQuarterNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeQuarterNum(ExtendedDayNum(d));
|
||||
@ -671,6 +813,10 @@ struct ToRelativeMonthNumImpl
|
||||
{
|
||||
return time_zone.toRelativeMonthNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMonthNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMonthNum(ExtendedDayNum(d));
|
||||
@ -691,6 +837,10 @@ struct ToRelativeWeekNumImpl
|
||||
{
|
||||
return time_zone.toRelativeWeekNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeWeekNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeWeekNum(ExtendedDayNum(d));
|
||||
@ -711,6 +861,10 @@ struct ToRelativeDayNumImpl
|
||||
{
|
||||
return time_zone.toDayNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl &)
|
||||
{
|
||||
return static_cast<ExtendedDayNum>(d);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
|
||||
{
|
||||
return static_cast<DayNum>(d);
|
||||
@ -732,6 +886,10 @@ struct ToRelativeHourNumImpl
|
||||
{
|
||||
return time_zone.toRelativeHourNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeHourNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeHourNum(ExtendedDayNum(d));
|
||||
@ -752,6 +910,10 @@ struct ToRelativeMinuteNumImpl
|
||||
{
|
||||
return time_zone.toRelativeMinuteNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMinuteNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMinuteNum(ExtendedDayNum(d));
|
||||
@ -772,6 +934,10 @@ struct ToRelativeSecondNumImpl
|
||||
{
|
||||
return t;
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d));
|
||||
@ -792,6 +958,10 @@ struct ToYYYYMMImpl
|
||||
{
|
||||
return time_zone.toNumYYYYMM(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMM(static_cast<ExtendedDayNum>(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMM(static_cast<DayNum>(d));
|
||||
@ -812,6 +982,10 @@ struct ToYYYYMMDDImpl
|
||||
{
|
||||
return time_zone.toNumYYYYMMDD(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDD(static_cast<ExtendedDayNum>(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDD(static_cast<DayNum>(d));
|
||||
@ -832,6 +1006,10 @@ struct ToYYYYMMDDhhmmssImpl
|
||||
{
|
||||
return time_zone.toNumYYYYMMDDhhmmss(t);
|
||||
}
|
||||
static inline UInt64 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<ExtendedDayNum>(d)));
|
||||
}
|
||||
static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<DayNum>(d)));
|
||||
|
120
src/Functions/FunctionChar.cpp
Normal file
120
src/Functions/FunctionChar.cpp
Normal file
@ -0,0 +1,120 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
class FunctionChar : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "char";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionChar>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.empty())
|
||||
throw Exception("Number of arguments for function " + getName() + " can't be " + toString(arguments.size())
|
||||
+ ", should be at least 1", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
for (const auto & arg : arguments)
|
||||
{
|
||||
WhichDataType which(arg);
|
||||
if (!(which.isInt() || which.isUInt() || which.isFloat()))
|
||||
throw Exception("Illegal type " + arg->getName() + " of argument of function " + getName()
|
||||
+ ", must be Int, UInt or Float number",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
auto col_str = ColumnString::create();
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
const auto size_per_row = arguments.size() + 1;
|
||||
out_vec.resize(size_per_row * input_rows_count);
|
||||
out_offsets.resize(input_rows_count);
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
out_offsets[row] = size_per_row + out_offsets[row - 1];
|
||||
out_vec[row * size_per_row + size_per_row - 1] = '\0';
|
||||
}
|
||||
|
||||
Columns columns_holder(arguments.size());
|
||||
for (size_t idx = 0; idx < arguments.size(); ++idx)
|
||||
{
|
||||
//partial const column
|
||||
columns_holder[idx] = arguments[idx].column->convertToFullColumnIfConst();
|
||||
const IColumn * column = columns_holder[idx].get();
|
||||
|
||||
if (!(executeNumber<UInt8>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<UInt16>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<UInt32>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<UInt64>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<Int8>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<Int16>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<Int32>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<Int64>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<Float32>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<Float64>(*column, out_vec, idx, input_rows_count, size_per_row)))
|
||||
{
|
||||
throw Exception{"Illegal column " + arguments[idx].column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
}
|
||||
|
||||
return col_str;
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
bool executeNumber(const IColumn & src_data, ColumnString::Chars & out_vec, const size_t & column_idx, const size_t & rows, const size_t & size_per_row) const
|
||||
{
|
||||
const ColumnVector<T> * src_data_concrete = checkAndGetColumn<ColumnVector<T>>(&src_data);
|
||||
|
||||
if (!src_data_concrete)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
for (size_t row = 0; row < rows; ++row)
|
||||
{
|
||||
out_vec[row * size_per_row + column_idx] = static_cast<char>(src_data_concrete->getInt(row));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionChar(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionChar>(FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Functions/CustomWeekTransforms.h>
|
||||
@ -35,7 +36,7 @@ public:
|
||||
{
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -43,7 +44,7 @@ public:
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -59,7 +60,7 @@ public:
|
||||
}
|
||||
else if (arguments.size() == 3)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -105,6 +106,9 @@ public:
|
||||
if (which.isDate())
|
||||
return CustomWeekTransformImpl<DataTypeDate, ToDataType>::execute(
|
||||
arguments, result_type, input_rows_count, Transform{});
|
||||
else if (which.isDate32())
|
||||
return CustomWeekTransformImpl<DataTypeDate32, ToDataType>::execute(
|
||||
arguments, result_type, input_rows_count, Transform{});
|
||||
else if (which.isDateTime())
|
||||
return CustomWeekTransformImpl<DataTypeDateTime, ToDataType>::execute(
|
||||
arguments, result_type, input_rows_count, Transform{});
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <common/DateLUTImpl.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
|
||||
@ -50,7 +51,11 @@ struct AddSecondsImpl
|
||||
{
|
||||
return t + delta;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// use default datetime64 scale
|
||||
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta) * 1000;
|
||||
}
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d)) + delta;
|
||||
@ -71,7 +76,11 @@ struct AddMinutesImpl
|
||||
{
|
||||
return t + delta * 60;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// use default datetime64 scale
|
||||
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 60) * 1000;
|
||||
}
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 60;
|
||||
@ -91,7 +100,11 @@ struct AddHoursImpl
|
||||
{
|
||||
return t + delta * 3600;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// use default datetime64 scale
|
||||
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 3600) * 1000;
|
||||
}
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 3600;
|
||||
@ -117,6 +130,11 @@ struct AddDaysImpl
|
||||
{
|
||||
return d + delta;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &)
|
||||
{
|
||||
return d + delta;
|
||||
}
|
||||
};
|
||||
|
||||
struct AddWeeksImpl
|
||||
@ -124,17 +142,22 @@ struct AddWeeksImpl
|
||||
static constexpr auto name = "addWeeks";
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
|
||||
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int32 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return {time_zone.addWeeks(t.whole, delta), t.fractional};
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addWeeks(t, delta);
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
|
||||
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl &)
|
||||
{
|
||||
return d + delta * 7;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int32 delta, const DateLUTImpl &)
|
||||
{
|
||||
return d + delta * 7;
|
||||
}
|
||||
@ -159,6 +182,11 @@ struct AddMonthsImpl
|
||||
{
|
||||
return time_zone.addMonths(ExtendedDayNum(d), delta);
|
||||
}
|
||||
|
||||
static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addMonths(ExtendedDayNum(d), delta);
|
||||
}
|
||||
};
|
||||
|
||||
struct AddQuartersImpl
|
||||
@ -166,17 +194,22 @@ struct AddQuartersImpl
|
||||
static constexpr auto name = "addQuarters";
|
||||
|
||||
static inline DecimalUtils::DecimalComponents<DateTime64>
|
||||
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int32 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return {time_zone.addQuarters(t.whole, delta), t.fractional};
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
static inline UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addQuarters(t, delta);
|
||||
}
|
||||
|
||||
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
static inline UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addQuarters(ExtendedDayNum(d), delta);
|
||||
}
|
||||
|
||||
static inline Int32 execute(Int32 d, Int32 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addQuarters(ExtendedDayNum(d), delta);
|
||||
}
|
||||
@ -201,6 +234,11 @@ struct AddYearsImpl
|
||||
{
|
||||
return time_zone.addYears(ExtendedDayNum(d), delta);
|
||||
}
|
||||
|
||||
static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addYears(ExtendedDayNum(d), delta);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Transform>
|
||||
@ -342,7 +380,7 @@ template <typename FieldType> struct ResultDataTypeMap {};
|
||||
template <> struct ResultDataTypeMap<UInt16> { using ResultDataType = DataTypeDate; };
|
||||
template <> struct ResultDataTypeMap<Int16> { using ResultDataType = DataTypeDate; };
|
||||
template <> struct ResultDataTypeMap<UInt32> { using ResultDataType = DataTypeDateTime; };
|
||||
template <> struct ResultDataTypeMap<Int32> { using ResultDataType = DataTypeDateTime; };
|
||||
template <> struct ResultDataTypeMap<Int32> { using ResultDataType = DataTypeDate32; };
|
||||
template <> struct ResultDataTypeMap<DateTime64> { using ResultDataType = DataTypeDateTime64; };
|
||||
template <> struct ResultDataTypeMap<Int64> { using ResultDataType = DataTypeDateTime64; };
|
||||
}
|
||||
@ -375,7 +413,7 @@ public:
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() +
|
||||
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
@ -398,6 +436,8 @@ public:
|
||||
{
|
||||
case TypeIndex::Date:
|
||||
return resolveReturnType<DataTypeDate>(arguments);
|
||||
case TypeIndex::Date32:
|
||||
return resolveReturnType<DataTypeDate32>(arguments);
|
||||
case TypeIndex::DateTime:
|
||||
return resolveReturnType<DataTypeDateTime>(arguments);
|
||||
case TypeIndex::DateTime64:
|
||||
@ -437,16 +477,23 @@ public:
|
||||
|
||||
if constexpr (std::is_same_v<ResultDataType, DataTypeDate>)
|
||||
return std::make_shared<DataTypeDate>();
|
||||
else if constexpr (std::is_same_v<ResultDataType, DataTypeDate32>)
|
||||
return std::make_shared<DataTypeDate32>();
|
||||
else if constexpr (std::is_same_v<ResultDataType, DataTypeDateTime>)
|
||||
{
|
||||
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
|
||||
}
|
||||
else if constexpr (std::is_same_v<ResultDataType, DataTypeDateTime64>)
|
||||
{
|
||||
// TODO (vnemkov): what if there is an overload of Transform::execute() that returns DateTime64 from DateTime or Date ?
|
||||
// Shall we use the default scale or one from optional argument ?
|
||||
const auto & datetime64_type = assert_cast<const DataTypeDateTime64 &>(*arguments[0].type);
|
||||
return std::make_shared<DataTypeDateTime64>(datetime64_type.getScale(), extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
|
||||
if (typeid_cast<const DataTypeDateTime64 *>(arguments[0].type.get()))
|
||||
{
|
||||
const auto & datetime64_type = assert_cast<const DataTypeDateTime64 &>(*arguments[0].type);
|
||||
return std::make_shared<DataTypeDateTime64>(datetime64_type.getScale(), extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
|
||||
}
|
||||
else
|
||||
{
|
||||
return std::make_shared<DataTypeDateTime64>(DataTypeDateTime64::default_scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -470,6 +517,11 @@ public:
|
||||
return DateTimeAddIntervalImpl<DataTypeDate, TransformResultDataType<DataTypeDate>, Transform>::execute(
|
||||
Transform{}, arguments, result_type);
|
||||
}
|
||||
else if (which.isDate32())
|
||||
{
|
||||
return DateTimeAddIntervalImpl<DataTypeDate32, TransformResultDataType<DataTypeDate32>, Transform>::execute(
|
||||
Transform{}, arguments, result_type);
|
||||
}
|
||||
else if (which.isDateTime())
|
||||
{
|
||||
return DateTimeAddIntervalImpl<DataTypeDateTime, TransformResultDataType<DataTypeDateTime>, Transform>::execute(
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
@ -38,7 +39,7 @@ public:
|
||||
{
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -46,7 +47,7 @@ public:
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -57,7 +58,7 @@ public:
|
||||
"must be of type Date or DateTime. The 2nd argument (optional) must be "
|
||||
"a constant string with timezone name",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if (isDate(arguments[0].type) && std::is_same_v<ToDataType, DataTypeDate>)
|
||||
if ((isDate(arguments[0].type) || isDate32(arguments[0].type)) && (std::is_same_v<ToDataType, DataTypeDate> || std::is_same_v<ToDataType, DataTypeDate32>))
|
||||
throw Exception(
|
||||
"The timezone argument of function " + getName() + " is allowed only when the 1st argument has the type DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -103,6 +104,8 @@ public:
|
||||
|
||||
if (which.isDate())
|
||||
return DateTimeTransformImpl<DataTypeDate, ToDataType, Transform>::execute(arguments, result_type, input_rows_count);
|
||||
else if (which.isDate32())
|
||||
return DateTimeTransformImpl<DataTypeDate32, ToDataType, Transform>::execute(arguments, result_type, input_rows_count);
|
||||
else if (which.isDateTime())
|
||||
return DateTimeTransformImpl<DataTypeDateTime, ToDataType, Transform>::execute(arguments, result_type, input_rows_count);
|
||||
else if (which.isDateTime64())
|
||||
@ -146,6 +149,12 @@ public:
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic : is_not_monotonic;
|
||||
}
|
||||
else if (checkAndGetDataType<DataTypeDate32>(&type))
|
||||
{
|
||||
return Transform::FactorTransform::execute(Int32(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(Int32(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic : is_not_monotonic;
|
||||
}
|
||||
else
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), date_lut)
|
||||
@ -156,4 +165,3 @@ public:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
562
src/Functions/FunctionsBinaryRepr.cpp
Normal file
562
src/Functions/FunctionsBinaryRepr.cpp
Normal file
@ -0,0 +1,562 @@
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/BitHelpers.h>
|
||||
#include <Common/hex.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
/*
|
||||
* hex(x) - Returns hexadecimal representation; capital letters; there are no prefixes 0x or suffixes h.
|
||||
* For numbers, returns a variable-length string - hex in the "human" (big endian) format, with the leading zeros being cut,
|
||||
* but only by whole bytes. For dates and datetimes - the same as for numbers.
|
||||
* For example, hex(257) = '0101'.
|
||||
*
|
||||
* unhex(string) - Returns a string, hex of which is equal to `string` with regard of case and discarding one leading zero.
|
||||
* If such a string does not exist, could return arbitrary implementation specific value.
|
||||
*
|
||||
* bin(x) - Returns binary representation.
|
||||
*
|
||||
* unbin(x) - Returns a string, opposite to `bin`.
|
||||
*
|
||||
*/
|
||||
|
||||
struct HexImpl
|
||||
{
|
||||
static constexpr auto name = "hex";
|
||||
static constexpr size_t word_size = 2;
|
||||
|
||||
template <typename T>
|
||||
static void executeOneUInt(T x, char *& out)
|
||||
{
|
||||
bool was_nonzero = false;
|
||||
for (int offset = (sizeof(T) - 1) * 8; offset >= 0; offset -= 8)
|
||||
{
|
||||
UInt8 byte = x >> offset;
|
||||
|
||||
/// Skip leading zeros
|
||||
if (byte == 0 && !was_nonzero && offset)
|
||||
continue;
|
||||
|
||||
was_nonzero = true;
|
||||
writeHexByteUppercase(byte, out);
|
||||
out += word_size;
|
||||
}
|
||||
*out = '\0';
|
||||
++out;
|
||||
}
|
||||
|
||||
static void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out)
|
||||
{
|
||||
while (pos < end)
|
||||
{
|
||||
writeHexByteUppercase(*pos, out);
|
||||
++pos;
|
||||
out += word_size;
|
||||
}
|
||||
*out = '\0';
|
||||
++out;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, const size_t type_size_in_bytes)
|
||||
{
|
||||
const size_t hex_length = type_size_in_bytes * word_size + 1; /// Including trailing zero byte.
|
||||
auto col_str = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
size_t size = in_vec.size();
|
||||
out_offsets.resize(size);
|
||||
out_vec.resize(size * hex_length);
|
||||
|
||||
size_t pos = 0;
|
||||
char * out = reinterpret_cast<char *>(&out_vec[0]);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 * in_pos = reinterpret_cast<const UInt8 *>(&in_vec[i]);
|
||||
executeOneString(in_pos, in_pos + type_size_in_bytes, out);
|
||||
|
||||
pos += hex_length;
|
||||
out_offsets[i] = pos;
|
||||
}
|
||||
col_res = std::move(col_str);
|
||||
}
|
||||
};
|
||||
|
||||
struct UnhexImpl
|
||||
{
|
||||
static constexpr auto name = "unhex";
|
||||
static constexpr size_t word_size = 2;
|
||||
|
||||
static void decode(const char * pos, const char * end, char *& out)
|
||||
{
|
||||
if ((end - pos) & 1)
|
||||
{
|
||||
*out = unhex(*pos);
|
||||
++out;
|
||||
++pos;
|
||||
}
|
||||
while (pos < end)
|
||||
{
|
||||
*out = unhex2(pos);
|
||||
pos += word_size;
|
||||
++out;
|
||||
}
|
||||
*out = '\0';
|
||||
++out;
|
||||
}
|
||||
};
|
||||
|
||||
struct BinImpl
|
||||
{
|
||||
static constexpr auto name = "bin";
|
||||
static constexpr size_t word_size = 8;
|
||||
|
||||
template <typename T>
|
||||
static void executeOneUInt(T x, char *& out)
|
||||
{
|
||||
bool was_nonzero = false;
|
||||
for (int offset = (sizeof(T) - 1) * 8; offset >= 0; offset -= 8)
|
||||
{
|
||||
UInt8 byte = x >> offset;
|
||||
|
||||
/// Skip leading zeros
|
||||
if (byte == 0 && !was_nonzero && offset)
|
||||
continue;
|
||||
|
||||
was_nonzero = true;
|
||||
writeBinByte(byte, out);
|
||||
out += word_size;
|
||||
}
|
||||
*out = '\0';
|
||||
++out;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, const size_t type_size_in_bytes)
|
||||
{
|
||||
const size_t hex_length = type_size_in_bytes * word_size + 1; /// Including trailing zero byte.
|
||||
auto col_str = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
size_t size = in_vec.size();
|
||||
out_offsets.resize(size);
|
||||
out_vec.resize(size * hex_length);
|
||||
|
||||
size_t pos = 0;
|
||||
char * out = reinterpret_cast<char *>(out_vec.data());
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 * in_pos = reinterpret_cast<const UInt8 *>(&in_vec[i]);
|
||||
executeOneString(in_pos, in_pos + type_size_in_bytes, out);
|
||||
|
||||
pos += hex_length;
|
||||
out_offsets[i] = pos;
|
||||
}
|
||||
col_res = std::move(col_str);
|
||||
}
|
||||
|
||||
static void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out)
|
||||
{
|
||||
while (pos < end)
|
||||
{
|
||||
writeBinByte(*pos, out);
|
||||
++pos;
|
||||
out += word_size;
|
||||
}
|
||||
*out = '\0';
|
||||
++out;
|
||||
}
|
||||
};
|
||||
|
||||
struct UnbinImpl
|
||||
{
|
||||
static constexpr auto name = "unbin";
|
||||
static constexpr size_t word_size = 8;
|
||||
|
||||
static void decode(const char * pos, const char * end, char *& out)
|
||||
{
|
||||
if (pos == end)
|
||||
{
|
||||
*out = '\0';
|
||||
++out;
|
||||
return;
|
||||
}
|
||||
|
||||
UInt8 left = 0;
|
||||
|
||||
/// end - pos is the length of input.
|
||||
/// (length & 7) to make remain bits length mod 8 is zero to split.
|
||||
/// e.g. the length is 9 and the input is "101000001",
|
||||
/// first left_cnt is 1, left is 0, right shift, pos is 1, left = 1
|
||||
/// then, left_cnt is 0, remain input is '01000001'.
|
||||
for (UInt8 left_cnt = (end - pos) & 7; left_cnt > 0; --left_cnt)
|
||||
{
|
||||
left = left << 1;
|
||||
if (*pos != '0')
|
||||
left += 1;
|
||||
++pos;
|
||||
}
|
||||
|
||||
if (left != 0 || end - pos == 0)
|
||||
{
|
||||
*out = left;
|
||||
++out;
|
||||
}
|
||||
|
||||
assert((end - pos) % 8 == 0);
|
||||
|
||||
while (end - pos != 0)
|
||||
{
|
||||
UInt8 c = 0;
|
||||
for (UInt8 i = 0; i < 8; ++i)
|
||||
{
|
||||
c = c << 1;
|
||||
if (*pos != '0')
|
||||
c += 1;
|
||||
++pos;
|
||||
}
|
||||
*out = c;
|
||||
++out;
|
||||
}
|
||||
|
||||
*out = '\0';
|
||||
++out;
|
||||
}
|
||||
};
|
||||
|
||||
/// Encode number or string to string with binary or hexadecimal representation
|
||||
template <typename Impl>
|
||||
class EncodeToBinaryRepr : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
static constexpr size_t word_size = Impl::word_size;
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<EncodeToBinaryRepr>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
WhichDataType which(arguments[0]);
|
||||
|
||||
if (!which.isStringOrFixedString() &&
|
||||
!which.isDate() &&
|
||||
!which.isDateTime() &&
|
||||
!which.isDateTime64() &&
|
||||
!which.isUInt() &&
|
||||
!which.isFloat() &&
|
||||
!which.isDecimal() &&
|
||||
!which.isAggregateFunction())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const IColumn * column = arguments[0].column.get();
|
||||
ColumnPtr res_column;
|
||||
|
||||
WhichDataType which(column->getDataType());
|
||||
if (which.isAggregateFunction())
|
||||
{
|
||||
const ColumnPtr to_string = castColumn(arguments[0], std::make_shared<DataTypeString>());
|
||||
const auto * str_column = checkAndGetColumn<ColumnString>(to_string.get());
|
||||
tryExecuteString(str_column, res_column);
|
||||
return res_column;
|
||||
}
|
||||
|
||||
if (tryExecuteUInt<UInt8>(column, res_column) ||
|
||||
tryExecuteUInt<UInt16>(column, res_column) ||
|
||||
tryExecuteUInt<UInt32>(column, res_column) ||
|
||||
tryExecuteUInt<UInt64>(column, res_column) ||
|
||||
tryExecuteString(column, res_column) ||
|
||||
tryExecuteFixedString(column, res_column) ||
|
||||
tryExecuteFloat<Float32>(column, res_column) ||
|
||||
tryExecuteFloat<Float64>(column, res_column) ||
|
||||
tryExecuteDecimal<Decimal32>(column, res_column) ||
|
||||
tryExecuteDecimal<Decimal64>(column, res_column) ||
|
||||
tryExecuteDecimal<Decimal128>(column, res_column))
|
||||
return res_column;
|
||||
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool tryExecuteUInt(const IColumn * col, ColumnPtr & col_res) const
|
||||
{
|
||||
const ColumnVector<T> * col_vec = checkAndGetColumn<ColumnVector<T>>(col);
|
||||
|
||||
static constexpr size_t MAX_LENGTH = sizeof(T) * word_size + 1; /// Including trailing zero byte.
|
||||
|
||||
if (col_vec)
|
||||
{
|
||||
auto col_str = ColumnString::create();
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
const typename ColumnVector<T>::Container & in_vec = col_vec->getData();
|
||||
|
||||
size_t size = in_vec.size();
|
||||
out_offsets.resize(size);
|
||||
out_vec.resize(size * (word_size+1) + MAX_LENGTH); /// word_size+1 is length of one byte in hex/bin plus zero byte.
|
||||
|
||||
size_t pos = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
/// Manual exponential growth, so as not to rely on the linear amortized work time of `resize` (no one guarantees it).
|
||||
if (pos + MAX_LENGTH > out_vec.size())
|
||||
out_vec.resize(out_vec.size() * word_size + MAX_LENGTH);
|
||||
|
||||
char * begin = reinterpret_cast<char *>(&out_vec[pos]);
|
||||
char * end = begin;
|
||||
Impl::executeOneUInt(in_vec[i], end);
|
||||
|
||||
pos += end - begin;
|
||||
out_offsets[i] = pos;
|
||||
}
|
||||
out_vec.resize(pos);
|
||||
|
||||
col_res = std::move(col_str);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
bool tryExecuteString(const IColumn *col, ColumnPtr &col_res) const
|
||||
{
|
||||
const ColumnString * col_str_in = checkAndGetColumn<ColumnString>(col);
|
||||
|
||||
if (col_str_in)
|
||||
{
|
||||
auto col_str = ColumnString::create();
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
const ColumnString::Chars & in_vec = col_str_in->getChars();
|
||||
const ColumnString::Offsets & in_offsets = col_str_in->getOffsets();
|
||||
|
||||
size_t size = in_offsets.size();
|
||||
|
||||
out_offsets.resize(size);
|
||||
/// reserve `word_size` bytes for each non trailing zero byte from input + `size` bytes for trailing zeros
|
||||
out_vec.resize((in_vec.size() - size) * word_size + size);
|
||||
|
||||
char * begin = reinterpret_cast<char *>(out_vec.data());
|
||||
char * pos = begin;
|
||||
size_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t new_offset = in_offsets[i];
|
||||
|
||||
Impl::executeOneString(&in_vec[prev_offset], &in_vec[new_offset - 1], pos);
|
||||
|
||||
out_offsets[i] = pos - begin;
|
||||
|
||||
prev_offset = new_offset;
|
||||
}
|
||||
if (!out_offsets.empty() && out_offsets.back() != out_vec.size())
|
||||
throw Exception("Column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
col_res = std::move(col_str);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool tryExecuteDecimal(const IColumn * col, ColumnPtr & col_res) const
|
||||
{
|
||||
const ColumnDecimal<T> * col_dec = checkAndGetColumn<ColumnDecimal<T>>(col);
|
||||
if (col_dec)
|
||||
{
|
||||
const typename ColumnDecimal<T>::Container & in_vec = col_dec->getData();
|
||||
Impl::executeFloatAndDecimal(in_vec, col_res, sizeof(T));
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
static bool tryExecuteFixedString(const IColumn * col, ColumnPtr & col_res)
|
||||
{
|
||||
const ColumnFixedString * col_fstr_in = checkAndGetColumn<ColumnFixedString>(col);
|
||||
|
||||
if (col_fstr_in)
|
||||
{
|
||||
auto col_str = ColumnString::create();
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_str->getOffsets();
|
||||
|
||||
const ColumnString::Chars & in_vec = col_fstr_in->getChars();
|
||||
|
||||
size_t size = col_fstr_in->size();
|
||||
|
||||
out_offsets.resize(size);
|
||||
out_vec.resize(in_vec.size() * word_size + size);
|
||||
|
||||
char * begin = reinterpret_cast<char *>(out_vec.data());
|
||||
char * pos = begin;
|
||||
|
||||
size_t n = col_fstr_in->getN();
|
||||
|
||||
size_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t new_offset = prev_offset + n;
|
||||
|
||||
Impl::executeOneString(&in_vec[prev_offset], &in_vec[new_offset], pos);
|
||||
|
||||
out_offsets[i] = pos - begin;
|
||||
prev_offset = new_offset;
|
||||
}
|
||||
|
||||
if (!out_offsets.empty() && out_offsets.back() != out_vec.size())
|
||||
throw Exception("Column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
col_res = std::move(col_str);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool tryExecuteFloat(const IColumn * col, ColumnPtr & col_res) const
|
||||
{
|
||||
const ColumnVector<T> * col_vec = checkAndGetColumn<ColumnVector<T>>(col);
|
||||
if (col_vec)
|
||||
{
|
||||
const typename ColumnVector<T>::Container & in_vec = col_vec->getData();
|
||||
Impl::executeFloatAndDecimal(in_vec, col_res, sizeof(T));
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
/// Decode number or string from string with binary or hexadecimal representation
|
||||
template <typename Impl>
|
||||
class DecodeFromBinaryRepr : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Impl::name;
|
||||
static constexpr size_t word_size = Impl::word_size;
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<DecodeFromBinaryRepr>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr & column = arguments[0].column;
|
||||
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & out_vec = col_res->getChars();
|
||||
ColumnString::Offsets & out_offsets = col_res->getOffsets();
|
||||
|
||||
const ColumnString::Chars & in_vec = col->getChars();
|
||||
const ColumnString::Offsets & in_offsets = col->getOffsets();
|
||||
|
||||
size_t size = in_offsets.size();
|
||||
out_offsets.resize(size);
|
||||
out_vec.resize(in_vec.size() / word_size + size);
|
||||
|
||||
char * begin = reinterpret_cast<char *>(out_vec.data());
|
||||
char * pos = begin;
|
||||
size_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t new_offset = in_offsets[i];
|
||||
|
||||
Impl::decode(reinterpret_cast<const char *>(&in_vec[prev_offset]), reinterpret_cast<const char *>(&in_vec[new_offset - 1]), pos);
|
||||
|
||||
out_offsets[i] = pos - begin;
|
||||
|
||||
prev_offset = new_offset;
|
||||
}
|
||||
|
||||
out_vec.resize(pos - begin);
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionsBinaryRepr(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<EncodeToBinaryRepr<HexImpl>>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<DecodeFromBinaryRepr<UnhexImpl>>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<EncodeToBinaryRepr<BinImpl>>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<DecodeFromBinaryRepr<UnbinImpl>>(FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
337
src/Functions/FunctionsBitToArray.cpp
Normal file
337
src/Functions/FunctionsBitToArray.cpp
Normal file
@ -0,0 +1,337 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
|
||||
/** Functions for an unusual conversion to a string or array:
|
||||
*
|
||||
* bitmaskToList - takes an integer - a bitmask, returns a string of degrees of 2 separated by a comma.
|
||||
* for example, bitmaskToList(50) = '2,16,32'
|
||||
*
|
||||
* bitmaskToArray(x) - Returns an array of powers of two in the binary form of x. For example, bitmaskToArray(50) = [2, 16, 32].
|
||||
*
|
||||
*/
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class FunctionBitmaskToList : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "bitmaskToList";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmaskToList>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const DataTypePtr & type = arguments[0];
|
||||
|
||||
if (!isInteger(type))
|
||||
throw Exception("Cannot format " + type->getName() + " as bitmask string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
ColumnPtr res;
|
||||
if (!((res = executeType<UInt8>(arguments))
|
||||
|| (res = executeType<UInt16>(arguments))
|
||||
|| (res = executeType<UInt32>(arguments))
|
||||
|| (res = executeType<UInt64>(arguments))
|
||||
|| (res = executeType<Int8>(arguments))
|
||||
|| (res = executeType<Int16>(arguments))
|
||||
|| (res = executeType<Int32>(arguments))
|
||||
|| (res = executeType<Int64>(arguments))))
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
inline static void writeBitmask(T x, WriteBuffer & out)
|
||||
{
|
||||
using UnsignedT = make_unsigned_t<T>;
|
||||
UnsignedT u_x = x;
|
||||
|
||||
bool first = true;
|
||||
while (u_x)
|
||||
{
|
||||
UnsignedT y = u_x & (u_x - 1);
|
||||
UnsignedT bit = u_x ^ y;
|
||||
u_x = y;
|
||||
if (!first)
|
||||
writeChar(',', out);
|
||||
first = false;
|
||||
writeIntText(T(bit), out);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
ColumnPtr executeType(const ColumnsWithTypeAndName & columns) const
|
||||
{
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[0].column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
|
||||
ColumnString::Chars & data_to = col_to->getChars();
|
||||
ColumnString::Offsets & offsets_to = col_to->getOffsets();
|
||||
size_t size = vec_from.size();
|
||||
data_to.resize(size * 2);
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> buf_to(data_to);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
writeBitmask<T>(vec_from[i], buf_to);
|
||||
writeChar(0, buf_to);
|
||||
offsets_to[i] = buf_to.count();
|
||||
}
|
||||
|
||||
buf_to.finalize();
|
||||
return col_to;
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionBitmaskToArray : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "bitmaskToArray";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmaskToArray>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isInteger(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeArray>(arguments[0]);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
template <typename T>
|
||||
bool tryExecute(const IColumn * column, ColumnPtr & out_column) const
|
||||
{
|
||||
using UnsignedT = make_unsigned_t<T>;
|
||||
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(column))
|
||||
{
|
||||
auto col_values = ColumnVector<T>::create();
|
||||
auto col_offsets = ColumnArray::ColumnOffsets::create();
|
||||
|
||||
typename ColumnVector<T>::Container & res_values = col_values->getData();
|
||||
ColumnArray::Offsets & res_offsets = col_offsets->getData();
|
||||
|
||||
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
|
||||
size_t size = vec_from.size();
|
||||
res_offsets.resize(size);
|
||||
res_values.reserve(size * 2);
|
||||
|
||||
for (size_t row = 0; row < size; ++row)
|
||||
{
|
||||
UnsignedT x = vec_from[row];
|
||||
while (x)
|
||||
{
|
||||
UnsignedT y = x & (x - 1);
|
||||
UnsignedT bit = x ^ y;
|
||||
x = y;
|
||||
res_values.push_back(bit);
|
||||
}
|
||||
res_offsets[row] = res_values.size();
|
||||
}
|
||||
|
||||
out_column = ColumnArray::create(std::move(col_values), std::move(col_offsets));
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const IColumn * in_column = arguments[0].column.get();
|
||||
ColumnPtr out_column;
|
||||
|
||||
if (tryExecute<UInt8>(in_column, out_column) ||
|
||||
tryExecute<UInt16>(in_column, out_column) ||
|
||||
tryExecute<UInt32>(in_column, out_column) ||
|
||||
tryExecute<UInt64>(in_column, out_column) ||
|
||||
tryExecute<Int8>(in_column, out_column) ||
|
||||
tryExecute<Int16>(in_column, out_column) ||
|
||||
tryExecute<Int32>(in_column, out_column) ||
|
||||
tryExecute<Int64>(in_column, out_column))
|
||||
return out_column;
|
||||
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionBitPositionsToArray : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "bitPositionsToArray";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitPositionsToArray>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isInteger(arguments[0]))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of argument of function {}",
|
||||
getName(),
|
||||
arguments[0]->getName());
|
||||
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
template <typename T>
|
||||
ColumnPtr executeType(const IColumn * column) const
|
||||
{
|
||||
const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(column);
|
||||
if (!col_from)
|
||||
return nullptr;
|
||||
|
||||
auto result_array_values = ColumnVector<UInt64>::create();
|
||||
auto result_array_offsets = ColumnArray::ColumnOffsets::create();
|
||||
|
||||
auto & result_array_values_data = result_array_values->getData();
|
||||
auto & result_array_offsets_data = result_array_offsets->getData();
|
||||
|
||||
auto & vec_from = col_from->getData();
|
||||
size_t size = vec_from.size();
|
||||
result_array_offsets_data.resize(size);
|
||||
result_array_values_data.reserve(size * 2);
|
||||
|
||||
using UnsignedType = make_unsigned_t<T>;
|
||||
|
||||
for (size_t row = 0; row < size; ++row)
|
||||
{
|
||||
UnsignedType x = static_cast<UnsignedType>(vec_from[row]);
|
||||
|
||||
if constexpr (is_big_int_v<UnsignedType>)
|
||||
{
|
||||
size_t position = 0;
|
||||
|
||||
while (x)
|
||||
{
|
||||
if (x & 1)
|
||||
result_array_values_data.push_back(position);
|
||||
|
||||
x >>= 1;
|
||||
++position;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
while (x)
|
||||
{
|
||||
result_array_values_data.push_back(getTrailingZeroBitsUnsafe(x));
|
||||
x &= (x - 1);
|
||||
}
|
||||
}
|
||||
|
||||
result_array_offsets_data[row] = result_array_values_data.size();
|
||||
}
|
||||
|
||||
auto result_column = ColumnArray::create(std::move(result_array_values), std::move(result_array_offsets));
|
||||
|
||||
return result_column;
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const IColumn * in_column = arguments[0].column.get();
|
||||
ColumnPtr result_column;
|
||||
|
||||
if (!((result_column = executeType<UInt8>(in_column))
|
||||
|| (result_column = executeType<UInt16>(in_column))
|
||||
|| (result_column = executeType<UInt32>(in_column))
|
||||
|| (result_column = executeType<UInt32>(in_column))
|
||||
|| (result_column = executeType<UInt64>(in_column))
|
||||
|| (result_column = executeType<UInt128>(in_column))
|
||||
|| (result_column = executeType<UInt256>(in_column))
|
||||
|| (result_column = executeType<Int8>(in_column))
|
||||
|| (result_column = executeType<Int16>(in_column))
|
||||
|| (result_column = executeType<Int32>(in_column))
|
||||
|| (result_column = executeType<Int64>(in_column))
|
||||
|| (result_column = executeType<Int128>(in_column))
|
||||
|| (result_column = executeType<Int256>(in_column))))
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of first argument of function {}",
|
||||
arguments[0].column->getName(),
|
||||
getName());
|
||||
}
|
||||
|
||||
return result_column;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
void registerFunctionsBitToArray(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionBitPositionsToArray>();
|
||||
factory.registerFunction<FunctionBitmaskToArray>();
|
||||
factory.registerFunction<FunctionBitmaskToList>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,54 +0,0 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsCoding.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameFunctionIPv4NumToString { static constexpr auto name = "IPv4NumToString"; };
|
||||
struct NameFunctionIPv4NumToStringClassC { static constexpr auto name = "IPv4NumToStringClassC"; };
|
||||
|
||||
|
||||
void registerFunctionsCoding(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionToStringCutToZero>();
|
||||
factory.registerFunction<FunctionCutIPv6>();
|
||||
factory.registerFunction<FunctionIPv4ToIPv6>();
|
||||
factory.registerFunction<FunctionMACNumToString>();
|
||||
factory.registerFunction<FunctionMACStringTo<ParseMACImpl>>();
|
||||
factory.registerFunction<FunctionMACStringTo<ParseOUIImpl>>();
|
||||
factory.registerFunction<FunctionUUIDNumToString>();
|
||||
factory.registerFunction<FunctionUUIDStringToNum>();
|
||||
factory.registerFunction<FunctionHex>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionUnhex>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionBin>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionUnbin>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionChar>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionBitmaskToArray>();
|
||||
factory.registerFunction<FunctionBitPositionsToArray>();
|
||||
factory.registerFunction<FunctionToIPv4>();
|
||||
factory.registerFunction<FunctionToIPv6>();
|
||||
factory.registerFunction<FunctionIPv6CIDRToRange>();
|
||||
factory.registerFunction<FunctionIPv4CIDRToRange>();
|
||||
factory.registerFunction<FunctionIsIPv4String>();
|
||||
factory.registerFunction<FunctionIsIPv6String>();
|
||||
|
||||
factory.registerFunction<FunctionIPv4NumToString<0, NameFunctionIPv4NumToString>>();
|
||||
factory.registerFunction<FunctionIPv4NumToString<1, NameFunctionIPv4NumToStringClassC>>();
|
||||
/// MysQL compatibility alias.
|
||||
factory.registerFunction<FunctionIPv4NumToString<0, NameFunctionIPv4NumToString>>("INET_NTOA", FunctionFactory::CaseInsensitive);
|
||||
|
||||
factory.registerFunction<FunctionIPv4StringToNum>();
|
||||
/// MysQL compatibility alias.
|
||||
factory.registerFunction<FunctionIPv4StringToNum>("INET_ATON", FunctionFactory::CaseInsensitive);
|
||||
|
||||
factory.registerFunction<FunctionIPv6NumToString>();
|
||||
/// MysQL compatibility alias.
|
||||
factory.registerFunction<FunctionIPv6NumToString>("INET6_NTOA", FunctionFactory::CaseInsensitive);
|
||||
|
||||
factory.registerFunction<FunctionIPv6StringToNum>();
|
||||
/// MysQL compatibility alias.
|
||||
factory.registerFunction<FunctionIPv6StringToNum>("INET6_ATON", FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
File diff suppressed because it is too large
Load Diff
1077
src/Functions/FunctionsCodingIP.cpp
Normal file
1077
src/Functions/FunctionsCodingIP.cpp
Normal file
File diff suppressed because it is too large
Load Diff
236
src/Functions/FunctionsCodingUUID.cpp
Normal file
236
src/Functions/FunctionsCodingUUID.cpp
Normal file
@ -0,0 +1,236 @@
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/BitHelpers.h>
|
||||
#include <Common/hex.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
constexpr size_t uuid_bytes_length = 16;
|
||||
constexpr size_t uuid_text_length = 36;
|
||||
|
||||
class FunctionUUIDNumToString : public IFunction
|
||||
{
|
||||
|
||||
public:
|
||||
static constexpr auto name = "UUIDNumToString";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionUUIDNumToString>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const auto * ptr = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != uuid_bytes_length)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument of function " + getName() +
|
||||
", expected FixedString(" + toString(uuid_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & col_type_name = arguments[0];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto * col_in = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
if (col_in->getN() != uuid_bytes_length)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(uuid_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto size = col_in->size();
|
||||
const auto & vec_in = col_in->getChars();
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & vec_res = col_res->getChars();
|
||||
ColumnString::Offsets & offsets_res = col_res->getOffsets();
|
||||
vec_res.resize(size * (uuid_text_length + 1));
|
||||
offsets_res.resize(size);
|
||||
|
||||
size_t src_offset = 0;
|
||||
size_t dst_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
formatUUID(&vec_in[src_offset], &vec_res[dst_offset]);
|
||||
src_offset += uuid_bytes_length;
|
||||
dst_offset += uuid_text_length;
|
||||
vec_res[dst_offset] = 0;
|
||||
++dst_offset;
|
||||
offsets_res[i] = dst_offset;
|
||||
}
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionUUIDStringToNum : public IFunction
|
||||
{
|
||||
private:
|
||||
static void parseHex(const UInt8 * __restrict src, UInt8 * __restrict dst, const size_t num_bytes)
|
||||
{
|
||||
size_t src_pos = 0;
|
||||
size_t dst_pos = 0;
|
||||
for (; dst_pos < num_bytes; ++dst_pos)
|
||||
{
|
||||
dst[dst_pos] = unhex2(reinterpret_cast<const char *>(&src[src_pos]));
|
||||
src_pos += 2;
|
||||
}
|
||||
}
|
||||
|
||||
static void parseUUID(const UInt8 * src36, UInt8 * dst16)
|
||||
{
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
parseHex(&src36[0], &dst16[0], 4);
|
||||
parseHex(&src36[9], &dst16[4], 2);
|
||||
parseHex(&src36[14], &dst16[6], 2);
|
||||
parseHex(&src36[19], &dst16[8], 2);
|
||||
parseHex(&src36[24], &dst16[10], 6);
|
||||
}
|
||||
|
||||
public:
|
||||
static constexpr auto name = "UUIDStringToNum";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionUUIDStringToNum>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
/// String or FixedString(36)
|
||||
if (!isString(arguments[0]))
|
||||
{
|
||||
const auto * ptr = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != uuid_text_length)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument of function " + getName() +
|
||||
", expected FixedString(" + toString(uuid_text_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeFixedString>(uuid_bytes_length);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & col_type_name = arguments[0];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto * col_in = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
const auto & vec_in = col_in->getChars();
|
||||
const auto & offsets_in = col_in->getOffsets();
|
||||
const size_t size = offsets_in.size();
|
||||
|
||||
auto col_res = ColumnFixedString::create(uuid_bytes_length);
|
||||
|
||||
ColumnString::Chars & vec_res = col_res->getChars();
|
||||
vec_res.resize(size * uuid_bytes_length);
|
||||
|
||||
size_t src_offset = 0;
|
||||
size_t dst_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
/// If string has incorrect length - then return zero UUID.
|
||||
/// If string has correct length but contains something not like UUID - implementation specific behaviour.
|
||||
|
||||
size_t string_size = offsets_in[i] - src_offset;
|
||||
if (string_size == uuid_text_length + 1)
|
||||
parseUUID(&vec_in[src_offset], &vec_res[dst_offset]);
|
||||
else
|
||||
memset(&vec_res[dst_offset], 0, uuid_bytes_length);
|
||||
|
||||
dst_offset += uuid_bytes_length;
|
||||
src_offset += string_size;
|
||||
}
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else if (const auto * col_in_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
if (col_in_fixed->getN() != uuid_text_length)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in_fixed->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(uuid_text_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto size = col_in_fixed->size();
|
||||
const auto & vec_in = col_in_fixed->getChars();
|
||||
|
||||
auto col_res = ColumnFixedString::create(uuid_bytes_length);
|
||||
|
||||
ColumnString::Chars & vec_res = col_res->getChars();
|
||||
vec_res.resize(size * uuid_bytes_length);
|
||||
|
||||
size_t src_offset = 0;
|
||||
size_t dst_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
parseUUID(&vec_in[src_offset], &vec_res[dst_offset]);
|
||||
src_offset += uuid_text_length;
|
||||
dst_offset += uuid_bytes_length;
|
||||
}
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionsCodingUUID(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionUUIDNumToString>();
|
||||
factory.registerFunction<FunctionUUIDStringToNum>();
|
||||
}
|
||||
|
||||
}
|
@ -1081,7 +1081,7 @@ public:
|
||||
const DataTypeTuple * right_tuple = checkAndGetDataType<DataTypeTuple>(arguments[1].get());
|
||||
|
||||
bool both_represented_by_number = arguments[0]->isValueRepresentedByNumber() && arguments[1]->isValueRepresentedByNumber();
|
||||
bool has_date = left.isDate() || right.isDate();
|
||||
bool has_date = left.isDateOrDate32() || right.isDateOrDate32();
|
||||
|
||||
if (!((both_represented_by_number && !has_date) /// Do not allow to compare date and number.
|
||||
|| (left.isStringOrFixedString() || right.isStringOrFixedString()) /// Everything can be compared with string by conversion.
|
||||
|
@ -32,7 +32,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToDate>();
|
||||
/// MysQL compatibility alias.
|
||||
factory.registerFunction<FunctionToDate>("DATE", FunctionFactory::CaseInsensitive);
|
||||
|
||||
factory.registerFunction<FunctionToDate32>();
|
||||
factory.registerFunction<FunctionToDateTime>();
|
||||
factory.registerFunction<FunctionToDateTime32>();
|
||||
factory.registerFunction<FunctionToDateTime64>();
|
||||
@ -62,6 +62,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToFloat32OrZero>();
|
||||
factory.registerFunction<FunctionToFloat64OrZero>();
|
||||
factory.registerFunction<FunctionToDateOrZero>();
|
||||
factory.registerFunction<FunctionToDate32OrZero>();
|
||||
factory.registerFunction<FunctionToDateTimeOrZero>();
|
||||
factory.registerFunction<FunctionToDateTime64OrZero>();
|
||||
|
||||
@ -87,6 +88,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToFloat32OrNull>();
|
||||
factory.registerFunction<FunctionToFloat64OrNull>();
|
||||
factory.registerFunction<FunctionToDateOrNull>();
|
||||
factory.registerFunction<FunctionToDate32OrNull>();
|
||||
factory.registerFunction<FunctionToDateTimeOrNull>();
|
||||
factory.registerFunction<FunctionToDateTime64OrNull>();
|
||||
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
@ -131,7 +132,7 @@ struct ConvertImpl
|
||||
|
||||
if (std::is_same_v<Name, NameToUnixTimestamp>)
|
||||
{
|
||||
if (isDate(named_from.type))
|
||||
if (isDateOrDate32(named_from.type))
|
||||
throw Exception("Illegal type " + named_from.type->getName() + " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -284,6 +285,10 @@ struct ConvertImpl
|
||||
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeDateTime, DataTypeDate, ToDateImpl> {};
|
||||
|
||||
/** Conversion of DateTime to Date32: throw off time component.
|
||||
*/
|
||||
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate32, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeDateTime, DataTypeDate32, ToDate32Impl> {};
|
||||
|
||||
/** Conversion of Date to DateTime: adding 00:00:00 time component.
|
||||
*/
|
||||
@ -296,6 +301,11 @@ struct ToDateTimeImpl
|
||||
return time_zone.fromDayNum(DayNum(d));
|
||||
}
|
||||
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d));
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt32 dt, const DateLUTImpl & /*time_zone*/)
|
||||
{
|
||||
return dt;
|
||||
@ -311,6 +321,9 @@ struct ToDateTimeImpl
|
||||
template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeDate, DataTypeDateTime, ToDateTimeImpl> {};
|
||||
|
||||
template <typename Name> struct ConvertImpl<DataTypeDate32, DataTypeDateTime, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeDate32, DataTypeDateTime, ToDateTimeImpl> {};
|
||||
|
||||
/// Implementation of toDate function.
|
||||
|
||||
template <typename FromType, typename ToType>
|
||||
@ -321,7 +334,7 @@ struct ToDateTransform32Or64
|
||||
static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// since converting to Date, no need in values outside of default LUT range.
|
||||
return (from < 0xFFFF)
|
||||
return (from < DATE_LUT_MAX_DAY_NUM)
|
||||
? from
|
||||
: time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF)));
|
||||
}
|
||||
@ -338,7 +351,7 @@ struct ToDateTransform32Or64Signed
|
||||
/// The function should be monotonic (better for query optimizations), so we saturate instead of overflow.
|
||||
if (from < 0)
|
||||
return 0;
|
||||
return (from < 0xFFFF)
|
||||
return (from < DATE_LUT_MAX_DAY_NUM)
|
||||
? from
|
||||
: time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF)));
|
||||
}
|
||||
@ -357,6 +370,48 @@ struct ToDateTransform8Or16Signed
|
||||
}
|
||||
};
|
||||
|
||||
/// Implementation of toDate32 function.
|
||||
|
||||
template <typename FromType, typename ToType>
|
||||
struct ToDate32Transform32Or64
|
||||
{
|
||||
static constexpr auto name = "toDate32";
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return (from < DATE_LUT_MAX_EXTEND_DAY_NUM)
|
||||
? from
|
||||
: time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF)));
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType>
|
||||
struct ToDate32Transform32Or64Signed
|
||||
{
|
||||
static constexpr auto name = "toDate32";
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone)
|
||||
{
|
||||
static const Int32 daynum_min_offset = -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch());
|
||||
if (from < daynum_min_offset)
|
||||
return daynum_min_offset;
|
||||
return (from < DATE_LUT_MAX_EXTEND_DAY_NUM)
|
||||
? from
|
||||
: time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF)));
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType>
|
||||
struct ToDate32Transform8Or16Signed
|
||||
{
|
||||
static constexpr auto name = "toDate32";
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &)
|
||||
{
|
||||
return from;
|
||||
}
|
||||
};
|
||||
|
||||
/** Special case of converting Int8, Int16, (U)Int32 or (U)Int64 (and also, for convenience,
|
||||
* Float32, Float64) to Date. If the number is negative, saturate it to unix epoch time. If the
|
||||
* number is less than 65536, then it is treated as DayNum, and if it's greater or equals to 65536,
|
||||
@ -383,6 +438,23 @@ template <typename Name> struct ConvertImpl<DataTypeFloat32, DataTypeDate, Name,
|
||||
template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeFloat64, DataTypeDate, ToDateTransform32Or64Signed<Float64, UInt16>> {};
|
||||
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate32, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeUInt32, DataTypeDate32, ToDate32Transform32Or64<UInt32, Int32>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate32, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeUInt64, DataTypeDate32, ToDate32Transform32Or64<UInt64, Int32>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt8, DataTypeDate32, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeInt8, DataTypeDate32, ToDate32Transform8Or16Signed<Int8, Int32>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt16, DataTypeDate32, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeInt16, DataTypeDate32, ToDate32Transform8Or16Signed<Int16, Int32>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate32, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeInt32, DataTypeDate32, ToDate32Transform32Or64Signed<Int32, Int32>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate32, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeInt64, DataTypeDate32, ToDate32Transform32Or64Signed<Int64, Int32>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeFloat32, DataTypeDate32, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeFloat32, DataTypeDate32, ToDate32Transform32Or64Signed<Float32, Int32>> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate32, Name, ConvertDefaultBehaviorTag>
|
||||
: DateTimeTransformImpl<DataTypeFloat64, DataTypeDate32, ToDate32Transform32Or64Signed<Float64, Int32>> {};
|
||||
|
||||
|
||||
template <typename FromType, typename ToType>
|
||||
struct ToDateTimeTransform64
|
||||
@ -598,6 +670,17 @@ struct FormatImpl<DataTypeDate>
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FormatImpl<DataTypeDate32>
|
||||
{
|
||||
template <typename ReturnType = void>
|
||||
static ReturnType execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl *)
|
||||
{
|
||||
writeDateText(ExtendedDayNum(x), wb);
|
||||
return ReturnType(true);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct FormatImpl<DataTypeDateTime>
|
||||
{
|
||||
@ -709,6 +792,8 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
|
||||
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDate>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDate32>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
|
||||
@ -811,6 +896,14 @@ inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb
|
||||
x = tmp;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void parseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
ExtendedDayNum tmp(0);
|
||||
readDateText(tmp, rb);
|
||||
x = tmp;
|
||||
}
|
||||
|
||||
// NOTE: no need of extra overload of DateTime64, since readDateTimeText64 has different signature and that case is explicitly handled in the calling code.
|
||||
template <>
|
||||
inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
|
||||
@ -851,6 +944,18 @@ inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer &
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline bool tryParseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
ExtendedDayNum tmp(0);
|
||||
if (!tryReadDateText(tmp, rb))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
x = tmp;
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
|
||||
{
|
||||
@ -1075,7 +1180,9 @@ struct ConvertThroughParsing
|
||||
SerializationDecimal<typename ToDataType::FieldType>::readText(
|
||||
vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
|
||||
else
|
||||
{
|
||||
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
|
||||
}
|
||||
}
|
||||
|
||||
if (!isAllRead(read_buffer))
|
||||
@ -1125,7 +1232,16 @@ struct ConvertThroughParsing
|
||||
parsed = false;
|
||||
|
||||
if (!parsed)
|
||||
vec_to[i] = static_cast<typename ToDataType::FieldType>(0);
|
||||
{
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeDate32>)
|
||||
{
|
||||
vec_to[i] = -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch());
|
||||
}
|
||||
else
|
||||
{
|
||||
vec_to[i] = static_cast<typename ToDataType::FieldType>(0);
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
|
||||
(*vec_null_map_to)[i] = !parsed;
|
||||
@ -1275,6 +1391,7 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name, ConvertDefaultBeha
|
||||
|
||||
/// Declared early because used below.
|
||||
struct NameToDate { static constexpr auto name = "toDate"; };
|
||||
struct NameToDate32 { static constexpr auto name = "toDate32"; };
|
||||
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
|
||||
struct NameToDateTime32 { static constexpr auto name = "toDateTime32"; };
|
||||
struct NameToDateTime64 { static constexpr auto name = "toDateTime64"; };
|
||||
@ -1398,6 +1515,8 @@ public:
|
||||
|| std::is_same_v<Name, NameToUnixTimestamp>
|
||||
// toDate(value[, timezone : String])
|
||||
|| std::is_same_v<ToDataType, DataTypeDate> // TODO: shall we allow timestamp argument for toDate? DateTime knows nothing about timezones and this argument is ignored below.
|
||||
// toDate(value[, timezone : String])
|
||||
|| std::is_same_v<ToDataType, DataTypeDate32>
|
||||
// toDateTime(value[, timezone: String])
|
||||
|| std::is_same_v<ToDataType, DataTypeDateTime>
|
||||
// toDateTime64(value, scale : Integer[, timezone: String])
|
||||
@ -1585,7 +1704,9 @@ private:
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag>::execute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
else
|
||||
{
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag>::execute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
@ -1957,7 +2078,7 @@ struct ToDateMonotonicity
|
||||
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
|
||||
{
|
||||
auto which = WhichDataType(type);
|
||||
if (which.isDate() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16())
|
||||
if (which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16())
|
||||
return {true, true, true};
|
||||
else if (
|
||||
(which.isUInt() && ((left.isNull() || left.get<UInt64>() < 0xFFFF) && (right.isNull() || right.get<UInt64>() >= 0xFFFF)))
|
||||
@ -1999,8 +2120,8 @@ struct ToStringMonotonicity
|
||||
if (const auto * low_cardinality_type = checkAndGetDataType<DataTypeLowCardinality>(type_ptr))
|
||||
type_ptr = low_cardinality_type->getDictionaryType().get();
|
||||
|
||||
/// `toString` function is monotonous if the argument is Date or DateTime or String, or non-negative numbers with the same number of symbols.
|
||||
if (checkDataTypes<DataTypeDate, DataTypeDateTime, DataTypeString>(type_ptr))
|
||||
/// `toString` function is monotonous if the argument is Date or Date32 or DateTime or String, or non-negative numbers with the same number of symbols.
|
||||
if (checkDataTypes<DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeString>(type_ptr))
|
||||
return positive;
|
||||
|
||||
if (left.isNull() || right.isNull())
|
||||
@ -2058,6 +2179,7 @@ using FunctionToInt256 = FunctionConvert<DataTypeInt256, NameToInt256, ToNumberM
|
||||
using FunctionToFloat32 = FunctionConvert<DataTypeFloat32, NameToFloat32, ToNumberMonotonicity<Float32>>;
|
||||
using FunctionToFloat64 = FunctionConvert<DataTypeFloat64, NameToFloat64, ToNumberMonotonicity<Float64>>;
|
||||
using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToDateMonotonicity>;
|
||||
using FunctionToDate32 = FunctionConvert<DataTypeDate32, NameToDate32, ToDateMonotonicity>;
|
||||
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToDateTimeMonotonicity>;
|
||||
using FunctionToDateTime32 = FunctionConvert<DataTypeDateTime, NameToDateTime32, ToDateTimeMonotonicity>;
|
||||
using FunctionToDateTime64 = FunctionConvert<DataTypeDateTime64, NameToDateTime64, UnknownMonotonicity>;
|
||||
@ -2087,6 +2209,7 @@ template <> struct FunctionTo<DataTypeInt256> { using Type = FunctionToInt256; }
|
||||
template <> struct FunctionTo<DataTypeFloat32> { using Type = FunctionToFloat32; };
|
||||
template <> struct FunctionTo<DataTypeFloat64> { using Type = FunctionToFloat64; };
|
||||
template <> struct FunctionTo<DataTypeDate> { using Type = FunctionToDate; };
|
||||
template <> struct FunctionTo<DataTypeDate32> { using Type = FunctionToDate32; };
|
||||
template <> struct FunctionTo<DataTypeDateTime> { using Type = FunctionToDateTime; };
|
||||
template <> struct FunctionTo<DataTypeDateTime64> { using Type = FunctionToDateTime64; };
|
||||
template <> struct FunctionTo<DataTypeUUID> { using Type = FunctionToUUID; };
|
||||
@ -2117,6 +2240,7 @@ struct NameToInt256OrZero { static constexpr auto name = "toInt256OrZero"; };
|
||||
struct NameToFloat32OrZero { static constexpr auto name = "toFloat32OrZero"; };
|
||||
struct NameToFloat64OrZero { static constexpr auto name = "toFloat64OrZero"; };
|
||||
struct NameToDateOrZero { static constexpr auto name = "toDateOrZero"; };
|
||||
struct NameToDate32OrZero { static constexpr auto name = "toDate32OrZero"; };
|
||||
struct NameToDateTimeOrZero { static constexpr auto name = "toDateTimeOrZero"; };
|
||||
struct NameToDateTime64OrZero { static constexpr auto name = "toDateTime64OrZero"; };
|
||||
struct NameToDecimal32OrZero { static constexpr auto name = "toDecimal32OrZero"; };
|
||||
@ -2140,6 +2264,7 @@ using FunctionToInt256OrZero = FunctionConvertFromString<DataTypeInt256, NameToI
|
||||
using FunctionToFloat32OrZero = FunctionConvertFromString<DataTypeFloat32, NameToFloat32OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToFloat64OrZero = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDateOrZero = FunctionConvertFromString<DataTypeDate, NameToDateOrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDate32OrZero = FunctionConvertFromString<DataTypeDate32, NameToDate32OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDateTimeOrZero = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDateTime64OrZero = FunctionConvertFromString<DataTypeDateTime64, NameToDateTime64OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDecimal32OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal32>, NameToDecimal32OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
@ -2163,6 +2288,7 @@ struct NameToInt256OrNull { static constexpr auto name = "toInt256OrNull"; };
|
||||
struct NameToFloat32OrNull { static constexpr auto name = "toFloat32OrNull"; };
|
||||
struct NameToFloat64OrNull { static constexpr auto name = "toFloat64OrNull"; };
|
||||
struct NameToDateOrNull { static constexpr auto name = "toDateOrNull"; };
|
||||
struct NameToDate32OrNull { static constexpr auto name = "toDate32OrNull"; };
|
||||
struct NameToDateTimeOrNull { static constexpr auto name = "toDateTimeOrNull"; };
|
||||
struct NameToDateTime64OrNull { static constexpr auto name = "toDateTime64OrNull"; };
|
||||
struct NameToDecimal32OrNull { static constexpr auto name = "toDecimal32OrNull"; };
|
||||
@ -2186,6 +2312,7 @@ using FunctionToInt256OrNull = FunctionConvertFromString<DataTypeInt256, NameToI
|
||||
using FunctionToFloat32OrNull = FunctionConvertFromString<DataTypeFloat32, NameToFloat32OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToFloat64OrNull = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDateOrNull = FunctionConvertFromString<DataTypeDate, NameToDateOrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDate32OrNull = FunctionConvertFromString<DataTypeDate32, NameToDate32OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDateTimeOrNull = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDateTime64OrNull = FunctionConvertFromString<DataTypeDateTime64, NameToDateTime64OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDecimal32OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal32>, NameToDecimal32OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
@ -2475,7 +2602,7 @@ private:
|
||||
UInt32 scale = to_type->getScale();
|
||||
|
||||
WhichDataType which(type_index);
|
||||
bool ok = which.isNativeInt() || which.isNativeUInt() || which.isDecimal() || which.isFloat() || which.isDate() || which.isDateTime() || which.isDateTime64()
|
||||
bool ok = which.isNativeInt() || which.isNativeUInt() || which.isDecimal() || which.isFloat() || which.isDateOrDate32() || which.isDateTime() || which.isDateTime64()
|
||||
|| which.isStringOrFixedString();
|
||||
if (!ok)
|
||||
{
|
||||
@ -3137,6 +3264,7 @@ private:
|
||||
std::is_same_v<ToDataType, DataTypeFloat32> ||
|
||||
std::is_same_v<ToDataType, DataTypeFloat64> ||
|
||||
std::is_same_v<ToDataType, DataTypeDate> ||
|
||||
std::is_same_v<ToDataType, DataTypeDate32> ||
|
||||
std::is_same_v<ToDataType, DataTypeDateTime> ||
|
||||
std::is_same_v<ToDataType, DataTypeUUID>)
|
||||
{
|
||||
@ -3236,6 +3364,8 @@ public:
|
||||
return monotonicityForType(type);
|
||||
if (const auto * type = checkAndGetDataType<DataTypeDate>(to_type))
|
||||
return monotonicityForType(type);
|
||||
if (const auto * type = checkAndGetDataType<DataTypeDate32>(to_type))
|
||||
return monotonicityForType(type);
|
||||
if (const auto * type = checkAndGetDataType<DataTypeDateTime>(to_type))
|
||||
return monotonicityForType(type);
|
||||
if (const auto * type = checkAndGetDataType<DataTypeString>(to_type))
|
||||
|
@ -683,6 +683,8 @@ public:
|
||||
return executeType<Int64>(arguments);
|
||||
else if (which.isDate())
|
||||
return executeType<UInt16>(arguments);
|
||||
else if (which.isDate32())
|
||||
return executeType<Int32>(arguments);
|
||||
else if (which.isDateTime())
|
||||
return executeType<UInt32>(arguments);
|
||||
else if (which.isDecimal32())
|
||||
@ -986,6 +988,7 @@ private:
|
||||
else if (which.isEnum8()) executeIntType<Int8, first>(icolumn, vec_to);
|
||||
else if (which.isEnum16()) executeIntType<Int16, first>(icolumn, vec_to);
|
||||
else if (which.isDate()) executeIntType<UInt16, first>(icolumn, vec_to);
|
||||
else if (which.isDate32()) executeIntType<Int32, first>(icolumn, vec_to);
|
||||
else if (which.isDateTime()) executeIntType<UInt32, first>(icolumn, vec_to);
|
||||
/// TODO: executeIntType() for Decimal32/64 leads to incompatible result
|
||||
else if (which.isDecimal32()) executeBigIntType<Decimal32, first>(icolumn, vec_to);
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
@ -407,6 +408,9 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg
|
||||
using DateMap = ClearableHashMapWithStackMemory<DataTypeDate::FieldType,
|
||||
size_t, DefaultHash<DataTypeDate::FieldType>, INITIAL_SIZE_DEGREE>;
|
||||
|
||||
using Date32Map = ClearableHashMapWithStackMemory<DataTypeDate32::FieldType,
|
||||
size_t, DefaultHash<DataTypeDate32::FieldType>, INITIAL_SIZE_DEGREE>;
|
||||
|
||||
using DateTimeMap = ClearableHashMapWithStackMemory<
|
||||
DataTypeDateTime::FieldType, size_t,
|
||||
DefaultHash<DataTypeDateTime::FieldType>, INITIAL_SIZE_DEGREE>;
|
||||
@ -421,6 +425,8 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg
|
||||
|
||||
if (which.isDate())
|
||||
result_column = execute<DateMap, ColumnVector<DataTypeDate::FieldType>, true>(arrays, std::move(column));
|
||||
else if (which.isDate32())
|
||||
result_column = execute<Date32Map, ColumnVector<DataTypeDate32::FieldType>, true>(arrays, std::move(column));
|
||||
else if (which.isDateTime())
|
||||
result_column = execute<DateTimeMap, ColumnVector<DataTypeDateTime::FieldType>, true>(arrays, std::move(column));
|
||||
else if (which.isString())
|
||||
|
@ -1,132 +0,0 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
|
||||
/** Function for an unusual conversion to a string:
|
||||
*
|
||||
* bitmaskToList - takes an integer - a bitmask, returns a string of degrees of 2 separated by a comma.
|
||||
* for example, bitmaskToList(50) = '2,16,32'
|
||||
*/
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class FunctionBitmaskToList : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "bitmaskToList";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionBitmaskToList>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const DataTypePtr & type = arguments[0];
|
||||
|
||||
if (!isInteger(type))
|
||||
throw Exception("Cannot format " + type->getName() + " as bitmask string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
ColumnPtr res;
|
||||
if (!((res = executeType<UInt8>(arguments))
|
||||
|| (res = executeType<UInt16>(arguments))
|
||||
|| (res = executeType<UInt32>(arguments))
|
||||
|| (res = executeType<UInt64>(arguments))
|
||||
|| (res = executeType<Int8>(arguments))
|
||||
|| (res = executeType<Int16>(arguments))
|
||||
|| (res = executeType<Int32>(arguments))
|
||||
|| (res = executeType<Int64>(arguments))))
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
inline static void writeBitmask(T x, WriteBuffer & out)
|
||||
{
|
||||
using UnsignedT = make_unsigned_t<T>;
|
||||
UnsignedT u_x = x;
|
||||
|
||||
bool first = true;
|
||||
while (u_x)
|
||||
{
|
||||
UnsignedT y = u_x & (u_x - 1);
|
||||
UnsignedT bit = u_x ^ y;
|
||||
u_x = y;
|
||||
if (!first)
|
||||
writeChar(',', out);
|
||||
first = false;
|
||||
writeIntText(T(bit), out);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
ColumnPtr executeType(const ColumnsWithTypeAndName & columns) const
|
||||
{
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[0].column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
|
||||
ColumnString::Chars & data_to = col_to->getChars();
|
||||
ColumnString::Offsets & offsets_to = col_to->getOffsets();
|
||||
size_t size = vec_from.size();
|
||||
data_to.resize(size * 2);
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars> buf_to(data_to);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
writeBitmask<T>(vec_from[i], buf_to);
|
||||
writeChar(0, buf_to);
|
||||
offsets_to[i] = buf_to.count();
|
||||
}
|
||||
|
||||
buf_to.finalize();
|
||||
return col_to;
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
void registerFunctionBitmaskToList(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionBitmaskToList>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,7 +12,10 @@ void registerFunctionsArray(FunctionFactory &);
|
||||
void registerFunctionsTuple(FunctionFactory &);
|
||||
void registerFunctionsMap(FunctionFactory &);
|
||||
void registerFunctionsBitmap(FunctionFactory &);
|
||||
void registerFunctionsBinaryRepr(FunctionFactory &);
|
||||
void registerFunctionsCoding(FunctionFactory &);
|
||||
void registerFunctionsCodingUUID(FunctionFactory &);
|
||||
void registerFunctionChar(FunctionFactory &);
|
||||
void registerFunctionsComparison(FunctionFactory &);
|
||||
void registerFunctionsConditional(FunctionFactory &);
|
||||
void registerFunctionsConversion(FunctionFactory &);
|
||||
@ -73,7 +76,10 @@ void registerFunctions()
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
registerFunctionsBitmap(factory);
|
||||
#endif
|
||||
registerFunctionsBinaryRepr(factory);
|
||||
registerFunctionsCoding(factory);
|
||||
registerFunctionsCodingUUID(factory);
|
||||
registerFunctionChar(factory);
|
||||
registerFunctionsComparison(factory);
|
||||
registerFunctionsConditional(factory);
|
||||
registerFunctionsConversion(factory);
|
||||
|
@ -3,14 +3,14 @@ namespace DB
|
||||
|
||||
class FunctionFactory;
|
||||
|
||||
void registerFunctionBitmaskToList(FunctionFactory &);
|
||||
void registerFunctionsBitToArray(FunctionFactory &);
|
||||
void registerFunctionFormatReadableSize(FunctionFactory &);
|
||||
void registerFunctionFormatReadableQuantity(FunctionFactory &);
|
||||
void registerFunctionFormatReadableTimeDelta(FunctionFactory &);
|
||||
|
||||
void registerFunctionsFormatting(FunctionFactory & factory)
|
||||
{
|
||||
registerFunctionBitmaskToList(factory);
|
||||
registerFunctionsBitToArray(factory);
|
||||
registerFunctionFormatReadableSize(factory);
|
||||
registerFunctionFormatReadableQuantity(factory);
|
||||
registerFunctionFormatReadableTimeDelta(factory);
|
||||
|
@ -37,6 +37,7 @@ void registerFunctionCountMatches(FunctionFactory &);
|
||||
void registerFunctionEncodeXMLComponent(FunctionFactory &);
|
||||
void registerFunctionDecodeXMLComponent(FunctionFactory &);
|
||||
void registerFunctionExtractTextFromHTML(FunctionFactory &);
|
||||
void registerFunctionToStringCutToZero(FunctionFactory &);
|
||||
|
||||
|
||||
#if USE_BASE64
|
||||
@ -77,6 +78,7 @@ void registerFunctionsString(FunctionFactory & factory)
|
||||
registerFunctionEncodeXMLComponent(factory);
|
||||
registerFunctionDecodeXMLComponent(factory);
|
||||
registerFunctionExtractTextFromHTML(factory);
|
||||
registerFunctionToStringCutToZero(factory);
|
||||
#if USE_BASE64
|
||||
registerFunctionBase64Encode(factory);
|
||||
registerFunctionBase64Decode(factory);
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user