From ea32434888c1e3115b08563e6497e9e88c53c1cd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marek=20Vavru=C5=A1a?= Date: Tue, 25 Sep 2018 22:12:32 -0700 Subject: [PATCH 01/52] Storages/Kafka: fixed deadlocks, moved to background pool This fixes two deadlocks in Kafka engine found previously: * When exception is thrown before starting reading, consumer was never returned to the storage. Now it is claimed only when actually starting reading. * Fixed lockup on deinitialization when consumer only unsubscribed, but didn't close, and the endine then timeouted when waiting for consumer destruction. This also moves the stream thread to background worker pool. The reason for that is that it will compete with other tasks for time, so it will form a backpressure on insertion when the system is busy. --- dbms/src/Storages/Kafka/StorageKafka.cpp | 179 +++++++++++++---------- dbms/src/Storages/Kafka/StorageKafka.h | 7 +- 2 files changed, 107 insertions(+), 79 deletions(-) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index ed18e03df7f..a9c9d9d85db 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -2,7 +2,6 @@ #include #if USE_RDKAFKA -#include #include #include #include @@ -49,6 +48,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TIMEOUT_EXCEEDED; } using namespace Poco::Util; @@ -132,7 +132,9 @@ public: : ReadBuffer(nullptr, 0), consumer(consumer_), current(nullptr), current_pending(false), log(log_), read_messages(0), row_delimiter(row_delimiter_) { - LOG_TRACE(log, "Row delimiter is: " << row_delimiter); + if (row_delimiter != '\0') { + LOG_TRACE(log, "Row delimiter is: " << row_delimiter); + } } ~ReadBufferFromKafkaConsumer() override { reset(); } @@ -156,23 +158,21 @@ class KafkaBlockInputStream : public IProfilingBlockInputStream { public: - KafkaBlockInputStream(StorageKafka & storage_, StorageKafka::ConsumerPtr consumer_, const Context & context_, const String & schema, size_t max_block_size) - : storage(storage_), consumer(consumer_) + KafkaBlockInputStream(StorageKafka & storage_, const Context & context_, const String & schema, size_t max_block_size_) + : storage(storage_), consumer(nullptr), context(context_), max_block_size(max_block_size_) { // Always skip unknown fields regardless of the context (JSON or TSKV) - Context context = context_; context.setSetting("input_format_skip_unknown_fields", UInt64(1)); if (schema.size() > 0) context.setSetting("format_schema", schema); - - // Create a formatted reader on Kafka messages - LOG_TRACE(storage.log, "Creating formatted reader"); - read_buf = std::make_unique(consumer->stream, storage.log, storage.row_delimiter); - reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size); } ~KafkaBlockInputStream() override { + if (!hasClaimed()) { + return; + } + // An error was thrown during the stream or it did not finish successfully // The read offsets weren't comitted, so consumer must rejoin the group from the original starting point if (!finalized) @@ -184,6 +184,7 @@ public: // Return consumer for another reader storage.pushConsumer(consumer); + consumer = nullptr; } String getName() const override @@ -193,16 +194,27 @@ public: Block readImpl() override { - if (isCancelledOrThrowIfKilled()) + if (isCancelledOrThrowIfKilled() || !hasClaimed()) return {}; return reader->read(); } - Block getHeader() const override { return reader->getHeader(); } + Block getHeader() const override { return storage.getSampleBlock(); } void readPrefixImpl() override { + if (!hasClaimed()) { + // Create a formatted reader on Kafka messages + LOG_TRACE(storage.log, "Creating formatted reader"); + consumer = storage.tryClaimConsumer(context.getSettingsRef().queue_max_wait_ms.totalMilliseconds()); + if (consumer == nullptr) + throw Exception("Failed to claim consumer: ", ErrorCodes::TIMEOUT_EXCEEDED); + + read_buf = std::make_unique(consumer->stream, storage.log, storage.row_delimiter); + reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size); + } + // Start reading data finalized = false; reader->readPrefix(); @@ -210,10 +222,11 @@ public: void readSuffixImpl() override { - reader->readSuffix(); - - // Store offsets read in this stream - read_buf->commit(); + if (hasClaimed()) { + reader->readSuffix(); + // Store offsets read in this stream + read_buf->commit(); + } // Mark as successfully finished finalized = true; @@ -222,10 +235,15 @@ public: private: StorageKafka & storage; StorageKafka::ConsumerPtr consumer; + Context context; + size_t max_block_size; Block sample_block; std::unique_ptr read_buf; - BlockInputStreamPtr reader; + BlockInputStreamPtr reader = nullptr; bool finalized = false; + + // Return true if consumer has been claimed by the stream + bool hasClaimed() { return consumer != nullptr; } }; static void loadFromConfig(struct rd_kafka_conf_s * conf, const AbstractConfiguration & config, const std::string & path) @@ -260,8 +278,10 @@ StorageKafka::StorageKafka( row_delimiter(row_delimiter_), schema_name(context.getMacros()->expand(schema_name_)), num_consumers(num_consumers_), log(&Logger::get("StorageKafka (" + table_name_ + ")")), - semaphore(0, num_consumers_), mutex(), consumers(), event_update() + semaphore(0, num_consumers_), mutex(), consumers() { + task = context.getSchedulePool().createTask(log->name(), [this]{ streamThread(); }); + task->deactivate(); } @@ -286,12 +306,8 @@ BlockInputStreams StorageKafka::read( // Claim as many consumers as requested, but don't block for (size_t i = 0; i < stream_count; ++i) { - auto consumer = tryClaimConsumer(0); - if (consumer == nullptr) - break; - // Use block size of 1, otherwise LIMIT won't work properly as it will buffer excess messages in the last block - streams.push_back(std::make_shared(*this, consumer, context, schema_name, 1)); + streams.emplace_back(std::make_shared(*this, context, schema_name, 1)); } LOG_DEBUG(log, "Starting reading " << streams.size() << " streams, " << max_block_size << " block size"); @@ -326,7 +342,7 @@ void StorageKafka::startup() } // Start the reader thread - stream_thread = std::thread(&StorageKafka::streamThread, this); + task->activateAndSchedule(); } @@ -334,27 +350,24 @@ void StorageKafka::shutdown() { // Interrupt streaming thread stream_cancelled = true; - event_update.set(); - // Unsubscribe from assignments - LOG_TRACE(log, "Unsubscribing from assignments"); + // Close all consumers for (size_t i = 0; i < num_created_consumers; ++i) { auto consumer = claimConsumer(); - consumer->unsubscribe(); + consumer->close(); } - // Wait for stream thread to finish - if (stream_thread.joinable()) - stream_thread.join(); - + LOG_TRACE(log, "Waiting for cleanup"); rd_kafka_wait_destroyed(CLEANUP_TIMEOUT_MS); + + task->deactivate(); } void StorageKafka::updateDependencies() { - event_update.set(); + task->activateAndSchedule(); } @@ -423,49 +436,45 @@ void StorageKafka::pushConsumer(StorageKafka::ConsumerPtr c) void StorageKafka::streamThread() { - setThreadName("KafkaStreamThr"); - CurrentThread::initializeQuery(); - - while (!stream_cancelled) + try { - try + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!stream_cancelled) { - // Keep streaming as long as there are attached views and streaming is not cancelled - while (!stream_cancelled) + // Check if all dependencies are attached + auto dependencies = context.getDependencies(database_name, table_name); + if (dependencies.size() == 0) + break; + + // Check the dependencies are ready? + bool ready = true; + for (const auto & db_tab : dependencies) { - // Check if all dependencies are attached - auto dependencies = context.getDependencies(database_name, table_name); - if (dependencies.size() == 0) - break; - // Check the dependencies are ready? - bool ready = true; - for (const auto & db_tab : dependencies) - { - if (!context.tryGetTable(db_tab.first, db_tab.second)) - ready = false; - } - if (!ready) - break; - - LOG_DEBUG(log, "Started streaming to " << dependencies.size() << " attached views"); - streamToViews(); - LOG_DEBUG(log, "Stopped streaming to views"); + if (!context.tryGetTable(db_tab.first, db_tab.second)) + ready = false; } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + if (!ready) + break; - // Wait for attached views - event_update.tryWait(READ_POLL_MS); + LOG_DEBUG(log, "Started streaming to " << dependencies.size() << " attached views"); + + // Reschedule if not limited + if (!streamToViews()) + break; + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } - LOG_DEBUG(log, "Stream thread finished"); + // Wait for attached views + if (!stream_cancelled) + task->scheduleAfter(READ_POLL_MS); } -void StorageKafka::streamToViews() +bool StorageKafka::streamToViews() { auto table = context.getTable(database_name, table_name); if (!table) @@ -486,9 +495,8 @@ void StorageKafka::streamToViews() streams.reserve(num_consumers); for (size_t i = 0; i < num_consumers; ++i) { - auto consumer = claimConsumer(); - auto stream = std::make_shared(*this, consumer, context, schema_name, block_size); - streams.push_back(stream); + auto stream = std::make_shared(*this, context, schema_name, block_size); + streams.emplace_back(stream); // Limit read batch to maximum block size to allow DDL IProfilingBlockInputStream::LocalLimits limits; @@ -498,12 +506,27 @@ void StorageKafka::streamToViews() p_stream->setLimits(limits); } - auto in = std::make_shared>(streams, nullptr, num_consumers); + // Join multiple streams if necessary + BlockInputStreamPtr in; + if (streams.size() > 1) { + in = std::make_shared>(streams, nullptr, num_consumers); + } else { + in = streams[0]; + } // Execute the query InterpreterInsertQuery interpreter{insert, context}; auto block_io = interpreter.execute(); copyData(*in, *block_io.out, &stream_cancelled); + + // Check whether the limits were applied during query execution + bool limits_applied = false; + if (IProfilingBlockInputStream * p_stream = dynamic_cast(in.get())) { + const BlockStreamProfileInfo & info = p_stream->getProfileInfo(); + limits_applied = info.hasAppliedLimit(); + } + + return limits_applied; } @@ -523,12 +546,7 @@ StorageKafka::Consumer::Consumer(struct rd_kafka_conf_s * conf) StorageKafka::Consumer::~Consumer() { - if (stream != nullptr) - { - rd_kafka_consumer_close(stream); - rd_kafka_destroy(stream); - stream = nullptr; - } + close(); } @@ -562,6 +580,15 @@ void StorageKafka::Consumer::unsubscribe() rd_kafka_unsubscribe(stream); } +void StorageKafka::Consumer::close() +{ + if (stream != nullptr) + { + rd_kafka_consumer_close(stream); + rd_kafka_destroy(stream); + stream = nullptr; + } +} void registerStorageKafka(StorageFactory & factory) { diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index 5b806d71d8f..f7e77b77c7a 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -6,6 +6,7 @@ #include #include +#include #include #include #include @@ -62,6 +63,7 @@ private: void subscribe(const Names & topics); void unsubscribe(); + void close(); struct rd_kafka_s * stream = nullptr; }; @@ -93,8 +95,7 @@ private: std::vector consumers; /// Available consumers // Stream thread - Poco::Event event_update; - std::thread stream_thread; + BackgroundSchedulePool::TaskHolder task; std::atomic stream_cancelled{false}; void consumerConfiguration(struct rd_kafka_conf_s * conf); @@ -103,7 +104,7 @@ private: void pushConsumer(ConsumerPtr c); void streamThread(); - void streamToViews(); + bool streamToViews(); protected: StorageKafka( From cbbcb6d9df85420eacf6f85330b06e6aff15e2ea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marek=20Vavru=C5=A1a?= Date: Tue, 25 Sep 2018 22:22:03 -0700 Subject: [PATCH 02/52] Formats/CapnProtoRowInputStream: support Nested and Tuple, fix alignment issues This updated contrib/capnproto to a newer version that fixes problems with unaligned access to message frames. It also adds support for parsing Struct types as Tuple (named or unnamed), and Nested array types. The `struct X { a @0 :UInt64; b @1 :Text }` in Cap'nProto is equivalent to `x Tuple(a UInt64, b String)` in ClickHouse. Arrays of Struct types such as `y List(X)` are equivalent to `y Nested(a UInt64, b String)`. --- contrib/capnproto | 2 +- dbms/src/Formats/CapnProtoRowInputStream.cpp | 68 ++++++++++++++++---- dbms/src/Formats/CapnProtoRowInputStream.h | 3 +- 3 files changed, 59 insertions(+), 14 deletions(-) diff --git a/contrib/capnproto b/contrib/capnproto index 7173ab638fd..a00ccd91b37 160000 --- a/contrib/capnproto +++ b/contrib/capnproto @@ -1 +1 @@ -Subproject commit 7173ab638fdf144032411dc69fb1082cd473e08f +Subproject commit a00ccd91b3746ef2ab51d40fe3265829949d1ace diff --git a/dbms/src/Formats/CapnProtoRowInputStream.cpp b/dbms/src/Formats/CapnProtoRowInputStream.cpp index 72fbd376399..598f33f49cb 100644 --- a/dbms/src/Formats/CapnProtoRowInputStream.cpp +++ b/dbms/src/Formats/CapnProtoRowInputStream.cpp @@ -68,13 +68,24 @@ Field convertNodeToField(capnp::DynamicValue::Reader value) auto listValue = value.as(); Array res(listValue.size()); for (auto i : kj::indices(listValue)) - res[i] = convertNodeToField(listValue[i]); + res[i] = convertNodeToField(listValue[i]); + return res; } case capnp::DynamicValue::ENUM: return UInt64(value.as().getRaw()); case capnp::DynamicValue::STRUCT: - throw Exception("STRUCT type not supported, read individual fields instead"); + { + auto structValue = value.as(); + const auto & fields = structValue.getSchema().getFields(); + + Field field = Tuple(TupleBackend(fields.size())); + TupleBackend & tuple = get(field).toUnderType(); + for (auto i : kj::indices(fields)) + tuple[i] = convertNodeToField(structValue.get(fields[i])); + + return field; + } case capnp::DynamicValue::CAPABILITY: throw Exception("CAPABILITY type not supported"); case capnp::DynamicValue::ANY_POINTER: @@ -88,9 +99,8 @@ capnp::StructSchema::Field getFieldOrThrow(capnp::StructSchema node, const std:: KJ_IF_MAYBE(child, node.findFieldByName(field)) return *child; else - throw Exception("Field " + field + " doesn't exist in schema."); + throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr()); } - void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields, capnp::StructSchema reader) { String last; @@ -110,13 +120,28 @@ void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields // Descend to a nested structure for (; level < field.tokens.size() - 1; ++level) { - last = field.tokens[level]; - parent = getFieldOrThrow(reader, last); - reader = parent.getType().asStruct(); - actions.push_back({Action::PUSH, parent}); + auto node = getFieldOrThrow(reader, field.tokens[level]); + if (node.getType().isStruct()) { + // Descend to field structure + last = field.tokens[level]; + parent = node; + reader = parent.getType().asStruct(); + actions.push_back({Action::PUSH, parent}); + } else if (node.getType().isList()) { + break; // Collect list + } else + throw Exception("Field " + field.tokens[level] + "is neither Struct nor List"); } + // Read field from the structure - actions.push_back({Action::READ, getFieldOrThrow(reader, field.tokens[level]), field.pos}); + auto node = getFieldOrThrow(reader, field.tokens[level]); + if (node.getType().isList() && actions.size() > 0 && actions.back().field == node) { + // The field list here flattens Nested elements into multiple arrays + // In order to map Nested types in Cap'nProto back, they need to be collected + actions.back().columns.push_back(field.pos); + } else { + actions.push_back({Action::READ, node, {field.pos}}); + } } } @@ -176,7 +201,7 @@ bool CapnProtoRowInputStream::read(MutableColumns & columns) array = heap_array.asPtr(); } - capnp::FlatArrayMessageReader msg(array); + capnp::UnalignedFlatArrayMessageReader msg(array); std::vector stack; stack.push_back(msg.getRoot(root)); @@ -186,9 +211,28 @@ bool CapnProtoRowInputStream::read(MutableColumns & columns) { case Action::READ: { - auto & col = columns[action.column]; Field value = convertNodeToField(stack.back().get(action.field)); - col->insert(value); + if (action.columns.size() > 1) { + // Nested columns must be flattened into several arrays + // e.g. Array(Tuple(x ..., y ...)) -> Array(x ...), Array(y ...) + const Array & collected = DB::get(value); + size_t size = collected.size(); + // The flattened array contains an array of a part of the nested tuple + Array flattened(size); + for (size_t column_index = 0; column_index < action.columns.size(); ++column_index) { + // Populate array with a single tuple elements + for (size_t off = 0; off < size; ++off) { + const TupleBackend & tuple = DB::get(collected[off]).toUnderType(); + flattened[off] = tuple[column_index]; + } + auto & col = columns[action.columns[column_index]]; + col->insert(flattened); + } + } else { + auto & col = columns[action.columns[0]]; + col->insert(value); + } + break; } case Action::POP: diff --git a/dbms/src/Formats/CapnProtoRowInputStream.h b/dbms/src/Formats/CapnProtoRowInputStream.h index 9152649bcbb..a7fcce49143 100644 --- a/dbms/src/Formats/CapnProtoRowInputStream.h +++ b/dbms/src/Formats/CapnProtoRowInputStream.h @@ -41,12 +41,13 @@ private: void createActions(const NestedFieldList & sortedFields, capnp::StructSchema reader); /* Action for state machine for traversing nested structures. */ + using BlockPositionList = std::vector; struct Action { enum Type { POP, PUSH, READ }; Type type; capnp::StructSchema::Field field = {}; - size_t column = 0; + BlockPositionList columns = {}; }; // Wrapper for classes that could throw in destructor From e90484db77be1c1d766c097a6b51a9828babae87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Sep 2018 22:43:10 +0300 Subject: [PATCH 03/52] Fix conversion between Date and DateTime when inserting in VALUES format and "input_format_values_interpret_expressions" is true #3226 --- dbms/src/DataTypes/IDataType.h | 5 ++ dbms/src/Interpreters/convertFieldToType.cpp | 68 +++++++++++-------- ...nsert_values_datetime_conversion.reference | 1 + ...0724_insert_values_datetime_conversion.sql | 3 + 4 files changed, 47 insertions(+), 30 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00724_insert_values_datetime_conversion.reference create mode 100644 dbms/tests/queries/0_stateless/00724_insert_values_datetime_conversion.sql diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 2bc3355f033..acf7cd27d56 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -411,6 +411,11 @@ struct WhichDataType { TypeIndex idx; + /// For late initialization. + WhichDataType() + : idx(TypeIndex::Nothing) + {} + WhichDataType(const IDataType & data_type) : idx(data_type.getTypeId()) {} diff --git a/dbms/src/Interpreters/convertFieldToType.cpp b/dbms/src/Interpreters/convertFieldToType.cpp index 8373f0d0694..517caa0edf6 100644 --- a/dbms/src/Interpreters/convertFieldToType.cpp +++ b/dbms/src/Interpreters/convertFieldToType.cpp @@ -21,6 +21,8 @@ #include #include +#include + namespace DB { @@ -138,63 +140,69 @@ UInt64 stringToDateTime(const String & s) return UInt64(date_time); } -Field convertFieldToTypeImpl(const Field & src, const IDataType & type) +Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const IDataType * from_type_hint) { - if (type.isValueRepresentedByNumber()) + WhichDataType which_type(type); + WhichDataType which_from_type; + if (from_type_hint) + which_from_type = WhichDataType(*from_type_hint); + + /// Conversion between Date and DateTime and vice versa. + if (which_type.isDate() && which_from_type.isDateTime()) { - if (typeid_cast(&type)) return convertNumericType(src, type); - if (typeid_cast(&type)) return convertNumericType(src, type); - if (typeid_cast(&type)) return convertNumericType(src, type); - if (typeid_cast(&type)) return convertNumericType(src, type); - if (typeid_cast(&type)) return convertNumericType(src, type); - if (typeid_cast(&type)) return convertNumericType(src, type); - if (typeid_cast(&type)) return convertNumericType(src, type); - if (typeid_cast(&type)) return convertNumericType(src, type); - if (typeid_cast(&type)) return convertNumericType(src, type); - if (typeid_cast(&type)) return convertNumericType(src, type); + return UInt64(static_cast(*from_type_hint).getTimeZone().toDayNum(src.get())); + } + else if (which_type.isDateTime() && which_from_type.isDate()) + { + return UInt64(static_cast(type).getTimeZone().fromDayNum(DayNum(src.get()))); + } + else if (type.isValueRepresentedByNumber()) + { + if (which_type.isUInt8()) return convertNumericType(src, type); + if (which_type.isUInt16()) return convertNumericType(src, type); + if (which_type.isUInt32()) return convertNumericType(src, type); + if (which_type.isUInt64()) return convertNumericType(src, type); + if (which_type.isInt8()) return convertNumericType(src, type); + if (which_type.isInt16()) return convertNumericType(src, type); + if (which_type.isInt32()) return convertNumericType(src, type); + if (which_type.isInt64()) return convertNumericType(src, type); + if (which_type.isFloat32()) return convertNumericType(src, type); + if (which_type.isFloat64()) return convertNumericType(src, type); if (auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); if (auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); if (auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); - const bool is_date = typeid_cast(&type); - bool is_datetime = false; - bool is_enum = false; - bool is_uuid = false; - - if (!is_date) - if (!(is_datetime = typeid_cast(&type))) - if (!(is_uuid = typeid_cast(&type))) - if (!(is_enum = dynamic_cast(&type))) - throw Exception{"Logical error: unknown numeric type " + type.getName(), ErrorCodes::LOGICAL_ERROR}; + if (!which_type.isDateOrDateTime() && !which_type.isUUID() && !which_type.isEnum()) + throw Exception{"Logical error: unknown numeric type " + type.getName(), ErrorCodes::LOGICAL_ERROR}; /// Numeric values for Enums should not be used directly in IN section - if (src.getType() == Field::Types::UInt64 && !is_enum) + if (src.getType() == Field::Types::UInt64 && !which_type.isEnum()) return src; if (src.getType() == Field::Types::String) { - if (is_date) + if (which_type.isDate()) { /// Convert 'YYYY-MM-DD' Strings to Date return UInt64(stringToDate(src.get())); } - else if (is_datetime) + else if (which_type.isDateTime()) { /// Convert 'YYYY-MM-DD hh:mm:ss' Strings to DateTime return stringToDateTime(src.get()); } - else if (is_uuid) + else if (which_type.isUUID()) { return stringToUUID(src.get()); } - else if (is_enum) + else if (which_type.isEnum()) { /// Convert String to Enum's value return dynamic_cast(type).castToValue(src); } } } - else if (isStringOrFixedString(type)) + else if (which_type.isStringOrFixedString()) { if (src.getType() == Field::Types::String) return src; @@ -252,9 +260,9 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co if (auto * with_dict_type = typeid_cast(&to_type)) return convertFieldToType(from_value, *with_dict_type->getDictionaryType(), from_type_hint); else if (auto * nullable_type = typeid_cast(&to_type)) - return convertFieldToTypeImpl(from_value, *nullable_type->getNestedType()); + return convertFieldToTypeImpl(from_value, *nullable_type->getNestedType(), from_type_hint); else - return convertFieldToTypeImpl(from_value, to_type); + return convertFieldToTypeImpl(from_value, to_type, from_type_hint); } diff --git a/dbms/tests/queries/0_stateless/00724_insert_values_datetime_conversion.reference b/dbms/tests/queries/0_stateless/00724_insert_values_datetime_conversion.reference new file mode 100644 index 00000000000..63fe3e78b4c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00724_insert_values_datetime_conversion.reference @@ -0,0 +1 @@ +2000-01-01 2000-01-01 00:00:00 diff --git a/dbms/tests/queries/0_stateless/00724_insert_values_datetime_conversion.sql b/dbms/tests/queries/0_stateless/00724_insert_values_datetime_conversion.sql new file mode 100644 index 00000000000..a86ea7a2fad --- /dev/null +++ b/dbms/tests/queries/0_stateless/00724_insert_values_datetime_conversion.sql @@ -0,0 +1,3 @@ +CREATE TEMPORARY TABLE test (d Date, dt DateTime); +INSERT INTO test VALUES (toDateTime('2000-01-01 01:02:03'), toDate('2000-01-01')); +SELECT * FROM test; From a971a0bc07b83af564496879e1a219a4fba9d498 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marek=20Vavru=C5=A1a?= Date: Mon, 24 Sep 2018 22:58:23 -0700 Subject: [PATCH 04/52] PushingToViewsBlockOutputStream: process blocks concurrently The current model is to process blocks for attached views in sequence. This is not ideal when the processing time for each view varies, or is blocking (for example with replicated tables), as processing of next-in-line view is blocked by wait in it's predecessor. This commit changes the behavior to process 2 or more attached views concurrently. --- .../PushingToViewsBlockOutputStream.cpp | 82 ++++++++++++------- .../PushingToViewsBlockOutputStream.h | 2 + dbms/src/Interpreters/Settings.h | 1 + 3 files changed, 57 insertions(+), 28 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 8216d53d197..a4cd29c778c 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -2,9 +2,12 @@ #include #include #include +#include +#include +#include +#include #include - namespace DB { @@ -73,35 +76,26 @@ void PushingToViewsBlockOutputStream::write(const Block & block) if (replicated_output && replicated_output->lastBlockIsDuplicate()) return; - /// Insert data into materialized views only after successful insert into main table - for (auto & view : views) - { - try + // Insert data into materialized views only after successful insert into main table + bool allow_concurrent_view_processing = context.getSettingsRef().allow_concurrent_view_processing; + if (allow_concurrent_view_processing && views.size() > 1) { + // Push to views concurrently if enabled, and more than one view is attached + ThreadPool pool(std::min(getNumberOfPhysicalCPUCores(), views.size())); + for (size_t view_num = 0; view_num < views.size(); ++view_num) { - BlockInputStreamPtr from = std::make_shared(block); - InterpreterSelectQuery select(view.query, *views_context, from); - BlockInputStreamPtr in = std::make_shared(select.execute().in); - /// Squashing is needed here because the materialized view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - in = std::make_shared( - in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); - - in->readPrefix(); - - while (Block result_block = in->read()) - { - Nested::validateArraySizes(result_block); - view.out->write(result_block); - } - - in->readSuffix(); - } - catch (Exception & ex) - { - ex.addMessage("while pushing to view " + view.database + "." + view.table); - throw; + auto thread_group = CurrentThread::getGroup(); + pool.schedule([=] () { + setThreadName("PushingToViewsBlockOutputStream"); + CurrentThread::attachToIfDetached(thread_group); + process(block, view_num); + }); } + // Wait for concurrent view processing + pool.wait(); + } else { + // Process sequentially + for (size_t view_num = 0; view_num < views.size(); ++view_num) + process(block, view_num); } } @@ -152,4 +146,36 @@ void PushingToViewsBlockOutputStream::flush() view.out->flush(); } +void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_num) +{ + auto & view = views[view_num]; + + try + { + BlockInputStreamPtr from = std::make_shared(block); + InterpreterSelectQuery select(view.query, *views_context, from); + BlockInputStreamPtr in = std::make_shared(select.execute().in); + /// Squashing is needed here because the materialized view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + in = std::make_shared( + in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); + + in->readPrefix(); + + while (Block result_block = in->read()) + { + Nested::validateArraySizes(result_block); + view.out->write(result_block); + } + + in->readSuffix(); + } + catch (Exception & ex) + { + ex.addMessage("while pushing to view " + backQuoteIfNeed(view.database) + "." + backQuoteIfNeed(view.table)); + throw; + } +} + } diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h index 0de1d1e2ee4..3381a828ff0 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -47,6 +47,8 @@ private: std::vector views; std::unique_ptr views_context; + + void process(const Block & block, size_t view_num); }; diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 60bd04fd5f2..ae4e5efefd9 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -291,6 +291,7 @@ struct Settings M(SettingUInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).") \ M(SettingBool, calculate_text_stack_trace, 1, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.") \ M(SettingBool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.") \ + M(SettingBool, allow_concurrent_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.") \ #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ From b35ab511cb593ab2f09db9cc3f9dc3bde8a646c1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 27 Sep 2018 13:27:45 +0800 Subject: [PATCH 05/52] ISSUES-3225 fix alias conflict when predicate optimization --- .../Interpreters/PredicateExpressionsOptimizer.cpp | 12 ++++++++++++ .../src/Interpreters/PredicateExpressionsOptimizer.h | 2 ++ .../0_stateless/00597_push_down_predicate.reference | 1 + .../0_stateless/00597_push_down_predicate.sql | 1 + 4 files changed, 16 insertions(+) diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 812dc3a0b7d..97130efa7a8 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -222,6 +222,8 @@ void PredicateExpressionsOptimizer::cloneOuterPredicateForInnerPredicate( { inner_predicate = outer_predicate->clone(); + /// clears the alias name contained in the outer predicate + cleanExpressionAlias(inner_predicate); IdentifiersWithQualifiedNameSet new_expression_requires; getDependenciesAndQualifiedOfExpression(inner_predicate, new_expression_requires, tables); @@ -419,4 +421,14 @@ std::vector PredicateExpressionsOptimizer::getSelectTables return tables_expression; } +void PredicateExpressionsOptimizer::cleanExpressionAlias(ASTPtr & expression) +{ + const auto my_alias = expression->tryGetAlias(); + if (!my_alias.empty()) + expression->setAlias(""); + + for (auto & child : expression->children) + cleanExpressionAlias(child); +} + } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index e656c4e0a15..6687860d979 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -88,6 +88,8 @@ private: std::vector getSelectTablesExpression(ASTSelectQuery * select_query); ASTs evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk); + + void cleanExpressionAlias(ASTPtr & expression); }; } diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index 22142c4748f..484107c0947 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -6,6 +6,7 @@ 1 1 1 +1 1 1 3 3 3 3 diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql index ede8b008867..36378bda64a 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -17,6 +17,7 @@ SELECT 1 AS id WHERE id = 1; SELECT arrayJoin([1,2,3]) AS id WHERE id = 1; SELECT '-------Need push down-------'; +SELECT toString(value) AS value FROM (SELECT 1 AS value) WHERE value = '1'; SELECT * FROM (SELECT 1 AS id UNION ALL SELECT 2) WHERE id = 1; SELECT * FROM (SELECT arrayJoin([1, 2, 3]) AS id) WHERE id = 1; SELECT id FROM (SELECT arrayJoin([1, 2, 3]) AS id) WHERE id = 1; From 90240cb64be5b4298c8a50be7f48d7e043554e62 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Thu, 27 Sep 2018 13:01:10 +0300 Subject: [PATCH 06/52] support database and table macros --- dbms/src/Common/Macros.cpp | 28 ++++++++++++++----- dbms/src/Common/Macros.h | 3 +- .../Storages/StorageReplicatedMergeTree.cpp | 2 +- 3 files changed, 24 insertions(+), 9 deletions(-) diff --git a/dbms/src/Common/Macros.cpp b/dbms/src/Common/Macros.cpp index d0bb1235741..0748e9bc1d1 100644 --- a/dbms/src/Common/Macros.cpp +++ b/dbms/src/Common/Macros.cpp @@ -2,6 +2,9 @@ #include #include +#include +#include + namespace DB { @@ -11,7 +14,10 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -Macros::Macros() {} +Macros::Macros() +{ + LOG_DEBUG(&Logger::get("Macros"), "creating in empty contructor"); +} Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key) { @@ -21,9 +27,10 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & { macros[key] = config.getString(root_key + "." + key); } + } -String Macros::expand(const String & s, size_t level) const +String Macros::expand(const String & s, size_t level, const String & database_name, const String & table_name) const { if (s.find('{') == String::npos) return s; @@ -57,16 +64,23 @@ String Macros::expand(const String & s, size_t level) const String macro_name = s.substr(begin, end - begin); - auto it = macros.find(macro_name); - if (it == macros.end()) - throw Exception("No macro " + macro_name + " in config", ErrorCodes::SYNTAX_ERROR); + if (macro_name == "database") + res += database_name; + else if (macro_name == "table") + res += table_name; + else + { + auto it = macros.find(macro_name); + if (it == macros.end()) + throw Exception("No macro " + macro_name + " in config", ErrorCodes::SYNTAX_ERROR); - res += it->second; + res += it->second; + } pos = end + 1; } - return expand(res, level + 1); + return expand(res, level + 1, database_name, table_name); } Names Macros::expand(const Names & source_names, size_t level) const diff --git a/dbms/src/Common/Macros.h b/dbms/src/Common/Macros.h index ce905723433..7fdc86b9ed0 100644 --- a/dbms/src/Common/Macros.h +++ b/dbms/src/Common/Macros.h @@ -2,6 +2,7 @@ #include #include +#include #include @@ -29,7 +30,7 @@ public: /** Replace the substring of the form {macro_name} with the value for macro_name, obtained from the config file. * level - the level of recursion. */ - String expand(const String & s, size_t level = 0) const; + String expand(const String & s, size_t level = 0, const String & database_name = "", const String & table_name = "") const; /** Apply expand for the list. */ diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 735200bb293..c96575d9156 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -207,7 +207,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( : context(context_), database_name(database_name_), table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'), - zookeeper_path(context.getMacros()->expand(zookeeper_path_)), + zookeeper_path(context.getMacros()->expand(zookeeper_path_, 0, database_name, table_name)), replica_name(context.getMacros()->expand(replica_name_)), data(database_name, table_name, full_path, columns_, From f8a72e849b36eb496b266d095bd98e5a4aadbad0 Mon Sep 17 00:00:00 2001 From: Kirill Malev Date: Thu, 27 Sep 2018 14:45:19 +0300 Subject: [PATCH 07/52] space fix --- docs/en/operations/configuration_files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/configuration_files.md b/docs/en/operations/configuration_files.md index 621a4a30e6e..d55cf7c2001 100644 --- a/docs/en/operations/configuration_files.md +++ b/docs/en/operations/configuration_files.md @@ -14,7 +14,7 @@ If `replace` is specified, it replaces the entire element with the specified one If `remove` is specified, it deletes the element. -The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros]()server_settings/settings.md#server_settings-macros)). +The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros]() server_settings/settings.md#server_settings-macros)). Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. From 65c8a00e5115b2114b8994130bc7f7826647dcd0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 27 Sep 2018 17:45:35 +0300 Subject: [PATCH 08/52] Fix low cardinality functions result cache if function can't be executed on default arguments. --- dbms/src/Functions/IFunction.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index f1459492f58..e8f35094629 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -440,7 +440,8 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si if (auto * res_type_with_dict = typeid_cast(res.type.get())) { const auto * low_cardinality_column = findLowCardinalityArgument(block, args); - bool use_cache = low_cardinality_result_cache + bool can_be_executed_on_default_arguments = canBeExecutedOnDefaultArguments(); + bool use_cache = low_cardinality_result_cache && can_be_executed_on_default_arguments && low_cardinality_column && low_cardinality_column->isSharedDictionary(); PreparedFunctionLowCardinalityResultCache::DictionaryKey key; @@ -460,7 +461,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si block_without_dicts.safeGetByPosition(result).type = res_type_with_dict->getDictionaryType(); ColumnPtr indexes = replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes( - block_without_dicts, args, canBeExecutedOnDefaultArguments()); + block_without_dicts, args, can_be_executed_on_default_arguments); executeWithoutColumnsWithDictionary(block_without_dicts, args, result, block_without_dicts.rows()); From 5a211b1ab74a2f90ada84ae5a58839b271cae36f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 27 Sep 2018 18:16:47 +0300 Subject: [PATCH 09/52] More comments. --- dbms/src/Functions/IFunction.cpp | 3 +++ dbms/src/Interpreters/ExpressionActions.h | 3 +++ 2 files changed, 6 insertions(+) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index e8f35094629..f74eb26cd60 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -42,6 +42,9 @@ namespace ErrorCodes } +/// Cache for functions result if it was executed on low cardinality column. +/// It's LRUCache which stores function result executed on dictionary and index mapping. +/// It's expected that cache_size is a number of reading streams (so, will store single cached value per thread). class PreparedFunctionLowCardinalityResultCache { public: diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index eec0ab7e903..4dc81c7d938 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -89,8 +89,11 @@ public: ColumnPtr added_column; /// For APPLY_FUNCTION and LEFT ARRAY JOIN. + /// FunctionBuilder is used before action was added to ExpressionActions (when we don't know types of arguments). FunctionBuilderPtr function_builder; + /// Can be used after action was added to ExpressionActions if we want to get function signature or properties like monotonicity. FunctionBasePtr function_base; + /// Prepared function which is used in function execution. PreparedFunctionPtr function; Names argument_names; bool is_function_compiled = false; From 169dbfe200127bf20e486ed092fe12d16012c499 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Sep 2018 06:40:09 +0300 Subject: [PATCH 10/52] Fixed discrepancy between default values of "log_queries" and "log_query_threads" settings [#CLICKHOUSE-4030] --- dbms/src/Interpreters/ThreadStatusExt.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/ThreadStatusExt.cpp b/dbms/src/Interpreters/ThreadStatusExt.cpp index fe9002d92f0..558e2ea3f4f 100644 --- a/dbms/src/Interpreters/ThreadStatusExt.cpp +++ b/dbms/src/Interpreters/ThreadStatusExt.cpp @@ -105,10 +105,13 @@ void ThreadStatus::finalizePerformanceCounters() try { - bool log_to_query_thread_log = global_context && query_context && query_context->getSettingsRef().log_query_threads.value != 0; - if (log_to_query_thread_log) - if (auto thread_log = global_context->getQueryThreadLog()) - logToQueryThreadLog(*thread_log); + if (global_context && query_context) + { + auto & settings = query_context->getSettingsRef(); + if (settings.log_queries && settings.log_query_threads) + if (auto thread_log = global_context->getQueryThreadLog()) + logToQueryThreadLog(*thread_log); + } } catch (...) { From 138e70197e732ea05ad87f2bb6cb433b6aa0376c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Sep 2018 06:42:08 +0300 Subject: [PATCH 11/52] Fixed comments and docs #3241 --- dbms/src/Interpreters/ProfileEventsExt.h | 2 +- dbms/src/Interpreters/Settings.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/ProfileEventsExt.h b/dbms/src/Interpreters/ProfileEventsExt.h index 4883505ee64..2ae9941b67f 100644 --- a/dbms/src/Interpreters/ProfileEventsExt.h +++ b/dbms/src/Interpreters/ProfileEventsExt.h @@ -6,7 +6,7 @@ namespace ProfileEvents { -/// Dumps profile events to two column Array(String) and Array(UInt64) +/// Dumps profile events to two columns Array(String) and Array(UInt64) void dumpToArrayColumns(const Counters & counters, DB::IColumn * column_names, DB::IColumn * column_value, bool nonzero_only = true); } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 60bd04fd5f2..15d07ca7062 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -275,7 +275,7 @@ struct Settings M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \ M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \ - M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table.") \ + M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.") \ M(SettingString, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \ M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.") \ \ @@ -332,7 +332,7 @@ struct Settings /// Write changed settings to buffer. (For example, to be sent to remote server.) void serialize(WriteBuffer & buf) const; - /// Dumps profile events to two column Array(String) and Array(UInt64) + /// Dumps profile events to two columns of type Array(String) void dumpToArrayColumns(IColumn * column_names, IColumn * column_values, bool changed_only = true); }; From 35d3a65881a7a2d715580e590541770e245ae83e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Sep 2018 07:35:59 +0300 Subject: [PATCH 12/52] Fixed postprocess-traces script [#CLICKHOUSE-4017] --- utils/postprocess-traces/postprocess-traces.pl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/postprocess-traces/postprocess-traces.pl b/utils/postprocess-traces/postprocess-traces.pl index a63a6b3534f..476fb46418f 100755 --- a/utils/postprocess-traces/postprocess-traces.pl +++ b/utils/postprocess-traces/postprocess-traces.pl @@ -15,7 +15,7 @@ while (my $line = <>) if ($line =~ '^#') { $line =~ s/^#\d+\s+//; - $line =~ s/ \([^\)]+=[^\)]+\) / /g; + $line =~ s/ \([^\)]+=.+\) at / at /g; push @current_stack, $line; } From 97add69825be2f821a020bd141523a6b420fd9d0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 27 Sep 2018 18:55:22 +0300 Subject: [PATCH 13/52] Rename WithDictionary to LowCardinality. --- .../AggregateFunctionFactory.cpp | 14 +- dbms/src/Columns/ColumnConst.cpp | 2 +- ...ictionary.cpp => ColumnLowCardinality.cpp} | 152 ++++++------ ...ithDictionary.h => ColumnLowCardinality.h} | 32 +-- dbms/src/Columns/FilterDescription.cpp | 6 +- dbms/src/Columns/IColumn.h | 8 +- ...umnLowCardinalityToFullBlockInputStream.h} | 14 +- dbms/src/DataTypes/DataTypeFactory.cpp | 4 +- ...tionary.cpp => DataTypeLowCardinality.cpp} | 226 +++++++++--------- ...hDictionary.h => DataTypeLowCardinality.h} | 8 +- dbms/src/DataTypes/IDataType.h | 2 +- dbms/src/Functions/FunctionsConversion.h | 48 ++-- dbms/src/Functions/IFunction.cpp | 98 ++++---- dbms/src/Functions/IFunction.h | 26 +- dbms/src/Functions/lowCardinalityIndices.cpp | 12 +- dbms/src/Functions/lowCardinalityKeys.cpp | 14 +- dbms/src/Functions/toLowCardinality.cpp | 14 +- dbms/src/Functions/toTypeName.cpp | 2 +- dbms/src/Interpreters/AggregationCommon.h | 2 +- dbms/src/Interpreters/Aggregator.cpp | 16 +- dbms/src/Interpreters/Aggregator.h | 8 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 4 +- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 5 +- dbms/src/Interpreters/Set.cpp | 10 +- dbms/src/Interpreters/convertFieldToType.cpp | 6 +- .../Storages/MergeTree/MergeTreeReader.cpp | 2 +- 27 files changed, 367 insertions(+), 370 deletions(-) rename dbms/src/Columns/{ColumnWithDictionary.cpp => ColumnLowCardinality.cpp} (77%) rename dbms/src/Columns/{ColumnWithDictionary.h => ColumnLowCardinality.h} (91%) rename dbms/src/DataStreams/{ConvertColumnWithDictionaryToFullBlockInputStream.h => ConvertColumnLowCardinalityToFullBlockInputStream.h} (75%) rename dbms/src/DataTypes/{DataTypeWithDictionary.cpp => DataTypeLowCardinality.cpp} (77%) rename dbms/src/DataTypes/{DataTypeWithDictionary.h => DataTypeLowCardinality.h} (97%) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index d644433ef3e..7c1cb537c7c 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -13,7 +13,7 @@ #include #include -#include +#include namespace DB @@ -42,14 +42,14 @@ void AggregateFunctionFactory::registerFunction(const String & name, Creator cre ErrorCodes::LOGICAL_ERROR); } -static DataTypes convertTypesWithDictionaryToNested(const DataTypes & types) +static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) { DataTypes res_types; res_types.reserve(types.size()); for (const auto & type : types) { - if (auto * type_with_dict = typeid_cast(type.get())) - res_types.push_back(type_with_dict->getDictionaryType()); + if (auto * low_cardinality_type = typeid_cast(type.get())) + res_types.push_back(low_cardinality_type->getDictionaryType()); else res_types.push_back(type); } @@ -63,7 +63,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( const Array & parameters, int recursion_level) const { - auto type_without_dictionary = convertTypesWithDictionaryToNested(argument_types); + auto type_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types); /// If one of types is Nullable, we apply aggregate function combinator "Null". @@ -74,7 +74,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( if (!combinator) throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR); - DataTypes nested_types = combinator->transformArguments(type_without_dictionary); + DataTypes nested_types = combinator->transformArguments(type_without_low_cardinality); AggregateFunctionPtr nested_function; @@ -87,7 +87,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( return combinator->transformAggregateFunction(nested_function, argument_types, parameters); } - auto res = getImpl(name, type_without_dictionary, parameters, recursion_level); + auto res = getImpl(name, type_without_low_cardinality, parameters, recursion_level); if (!res) throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR); return res; diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index 95c21786484..3703d24f1cb 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -32,7 +32,7 @@ ColumnPtr ColumnConst::convertToFullColumn() const ColumnPtr ColumnConst::removeLowCardinality() const { - return ColumnConst::create(data->convertToFullColumnIfWithDictionary(), s); + return ColumnConst::create(data->convertToFullColumnIfLowCardinality(), s); } ColumnPtr ColumnConst::filter(const Filter & filt, ssize_t /*result_size_hint*/) const diff --git a/dbms/src/Columns/ColumnWithDictionary.cpp b/dbms/src/Columns/ColumnLowCardinality.cpp similarity index 77% rename from dbms/src/Columns/ColumnWithDictionary.cpp rename to dbms/src/Columns/ColumnLowCardinality.cpp index 6056596360d..45fd9a0da50 100644 --- a/dbms/src/Columns/ColumnWithDictionary.cpp +++ b/dbms/src/Columns/ColumnLowCardinality.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -109,34 +109,34 @@ namespace } -ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_, bool is_shared) +ColumnLowCardinality::ColumnLowCardinality(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_, bool is_shared) : dictionary(std::move(column_unique_), is_shared), idx(std::move(indexes_)) { idx.check(getDictionary().size()); } -void ColumnWithDictionary::insert(const Field & x) +void ColumnLowCardinality::insert(const Field & x) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsert(x)); idx.check(getDictionary().size()); } -void ColumnWithDictionary::insertDefault() +void ColumnLowCardinality::insertDefault() { idx.insertPosition(getDictionary().getDefaultValueIndex()); } -void ColumnWithDictionary::insertFrom(const IColumn & src, size_t n) +void ColumnLowCardinality::insertFrom(const IColumn & src, size_t n) { - auto * src_with_dict = typeid_cast(&src); + auto * low_cardinality_src = typeid_cast(&src); - if (!src_with_dict) - throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); + if (!low_cardinality_src) + throw Exception("Expected ColumnLowCardinality, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); - size_t position = src_with_dict->getIndexes().getUInt(n); + size_t position = low_cardinality_src->getIndexes().getUInt(n); - if (&src_with_dict->getDictionary() == &getDictionary()) + if (&low_cardinality_src->getDictionary() == &getDictionary()) { /// Dictionary is shared with src column. Insert only index. idx.insertPosition(position); @@ -144,31 +144,31 @@ void ColumnWithDictionary::insertFrom(const IColumn & src, size_t n) else { compactIfSharedDictionary(); - const auto & nested = *src_with_dict->getDictionary().getNestedColumn(); + const auto & nested = *low_cardinality_src->getDictionary().getNestedColumn(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(nested, position)); } idx.check(getDictionary().size()); } -void ColumnWithDictionary::insertFromFullColumn(const IColumn & src, size_t n) +void ColumnLowCardinality::insertFromFullColumn(const IColumn & src, size_t n) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(src, n)); idx.check(getDictionary().size()); } -void ColumnWithDictionary::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnLowCardinality::insertRangeFrom(const IColumn & src, size_t start, size_t length) { - auto * src_with_dict = typeid_cast(&src); + auto * low_cardinality_src = typeid_cast(&src); - if (!src_with_dict) - throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); + if (!low_cardinality_src) + throw Exception("Expected ColumnLowCardinality, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); - if (&src_with_dict->getDictionary() == &getDictionary()) + if (&low_cardinality_src->getDictionary() == &getDictionary()) { /// Dictionary is shared with src column. Insert only indexes. - idx.insertPositionsRange(src_with_dict->getIndexes(), start, length); + idx.insertPositionsRange(low_cardinality_src->getIndexes(), start, length); } else { @@ -176,10 +176,10 @@ void ColumnWithDictionary::insertRangeFrom(const IColumn & src, size_t start, si /// TODO: Support native insertion from other unique column. It will help to avoid null map creation. - auto sub_idx = (*src_with_dict->getIndexes().cut(start, length)).mutate(); + auto sub_idx = (*low_cardinality_src->getIndexes().cut(start, length)).mutate(); auto idx_map = mapUniqueIndex(*sub_idx); - auto src_nested = src_with_dict->getDictionary().getNestedColumn(); + auto src_nested = low_cardinality_src->getDictionary().getNestedColumn(); auto used_keys = src_nested->index(*idx_map, 0); auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(*used_keys, 0, used_keys->size()); @@ -188,7 +188,7 @@ void ColumnWithDictionary::insertRangeFrom(const IColumn & src, size_t start, si idx.check(getDictionary().size()); } -void ColumnWithDictionary::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length) +void ColumnLowCardinality::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length) { compactIfSharedDictionary(); auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(src, start, length); @@ -196,7 +196,7 @@ void ColumnWithDictionary::insertRangeFromFullColumn(const IColumn & src, size_t idx.check(getDictionary().size()); } -void ColumnWithDictionary::insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions) +void ColumnLowCardinality::insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions) { Index(positions.getPtr()).check(keys.size()); compactIfSharedDictionary(); @@ -205,26 +205,26 @@ void ColumnWithDictionary::insertRangeFromDictionaryEncodedColumn(const IColumn idx.check(getDictionary().size()); } -void ColumnWithDictionary::insertData(const char * pos, size_t length) +void ColumnLowCardinality::insertData(const char * pos, size_t length) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertData(pos, length)); idx.check(getDictionary().size()); } -void ColumnWithDictionary::insertDataWithTerminatingZero(const char * pos, size_t length) +void ColumnLowCardinality::insertDataWithTerminatingZero(const char * pos, size_t length) { compactIfSharedDictionary(); idx.insertPosition(dictionary.getColumnUnique().uniqueInsertDataWithTerminatingZero(pos, length)); idx.check(getDictionary().size()); } -StringRef ColumnWithDictionary::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +StringRef ColumnLowCardinality::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const { return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin); } -const char * ColumnWithDictionary::deserializeAndInsertFromArena(const char * pos) +const char * ColumnLowCardinality::deserializeAndInsertFromArena(const char * pos) { compactIfSharedDictionary(); @@ -235,26 +235,26 @@ const char * ColumnWithDictionary::deserializeAndInsertFromArena(const char * po return new_pos; } -void ColumnWithDictionary::gather(ColumnGathererStream & gatherer) +void ColumnLowCardinality::gather(ColumnGathererStream & gatherer) { gatherer.gather(*this); } -MutableColumnPtr ColumnWithDictionary::cloneResized(size_t size) const +MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const { auto unique_ptr = dictionary.getColumnUniquePtr(); - return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), getIndexes().cloneResized(size)); + return ColumnLowCardinality::create((*std::move(unique_ptr)).mutate(), getIndexes().cloneResized(size)); } -int ColumnWithDictionary::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { - const auto & column_with_dictionary = static_cast(rhs); + const auto & low_cardinality_column = static_cast(rhs); size_t n_index = getIndexes().getUInt(n); - size_t m_index = column_with_dictionary.getIndexes().getUInt(m); - return getDictionary().compareAt(n_index, m_index, column_with_dictionary.getDictionary(), nan_direction_hint); + size_t m_index = low_cardinality_column.getIndexes().getUInt(m); + return getDictionary().compareAt(n_index, m_index, low_cardinality_column.getDictionary(), nan_direction_hint); } -void ColumnWithDictionary::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { if (limit == 0) limit = size(); @@ -289,65 +289,65 @@ void ColumnWithDictionary::getPermutation(bool reverse, size_t limit, int nan_di } } -std::vector ColumnWithDictionary::scatter(ColumnIndex num_columns, const Selector & selector) const +std::vector ColumnLowCardinality::scatter(ColumnIndex num_columns, const Selector & selector) const { auto columns = getIndexes().scatter(num_columns, selector); for (auto & column : columns) { auto unique_ptr = dictionary.getColumnUniquePtr(); - column = ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), std::move(column)); + column = ColumnLowCardinality::create((*std::move(unique_ptr)).mutate(), std::move(column)); } return columns; } -void ColumnWithDictionary::setSharedDictionary(const ColumnPtr & column_unique) +void ColumnLowCardinality::setSharedDictionary(const ColumnPtr & column_unique) { if (!empty()) - throw Exception("Can't set ColumnUnique for ColumnWithDictionary because is't not empty.", + throw Exception("Can't set ColumnUnique for ColumnLowCardinality because is't not empty.", ErrorCodes::LOGICAL_ERROR); dictionary.setShared(column_unique); } -ColumnWithDictionary::MutablePtr ColumnWithDictionary::compact() +ColumnLowCardinality::MutablePtr ColumnLowCardinality::compact() { auto positions = idx.getPositions(); /// Create column with new indexes and old dictionary. - auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), (*std::move(positions)).mutate()); + auto column = ColumnLowCardinality::create(getDictionary().assumeMutable(), (*std::move(positions)).mutate()); /// Will create new dictionary. column->compactInplace(); return column; } -ColumnWithDictionary::MutablePtr ColumnWithDictionary::cutAndCompact(size_t start, size_t length) const +ColumnLowCardinality::MutablePtr ColumnLowCardinality::cutAndCompact(size_t start, size_t length) const { auto sub_positions = (*idx.getPositions()->cut(start, length)).mutate(); /// Create column with new indexes and old dictionary. - auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), std::move(sub_positions)); + auto column = ColumnLowCardinality::create(getDictionary().assumeMutable(), std::move(sub_positions)); /// Will create new dictionary. column->compactInplace(); return column; } -void ColumnWithDictionary::compactInplace() +void ColumnLowCardinality::compactInplace() { auto positions = idx.detachPositions(); dictionary.compact(positions); idx.attachPositions(std::move(positions)); } -void ColumnWithDictionary::compactIfSharedDictionary() +void ColumnLowCardinality::compactIfSharedDictionary() { if (dictionary.isShared()) compactInplace(); } -ColumnWithDictionary::DictionaryEncodedColumn -ColumnWithDictionary::getMinimalDictionaryEncodedColumn(size_t offset, size_t limit) const +ColumnLowCardinality::DictionaryEncodedColumn +ColumnLowCardinality::getMinimalDictionaryEncodedColumn(size_t offset, size_t limit) const { MutableColumnPtr sub_indexes = (*std::move(idx.getPositions()->cut(offset, limit))).mutate(); auto indexes_map = mapUniqueIndex(*sub_indexes); @@ -356,7 +356,7 @@ ColumnWithDictionary::getMinimalDictionaryEncodedColumn(size_t offset, size_t li return {std::move(sub_keys), std::move(sub_indexes)}; } -ColumnPtr ColumnWithDictionary::countKeys() const +ColumnPtr ColumnLowCardinality::countKeys() const { const auto & nested_column = getDictionary().getNestedColumn(); size_t dict_size = nested_column->size(); @@ -368,20 +368,20 @@ ColumnPtr ColumnWithDictionary::countKeys() const -ColumnWithDictionary::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {} +ColumnLowCardinality::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {} -ColumnWithDictionary::Index::Index(MutableColumnPtr && positions) : positions(std::move(positions)) +ColumnLowCardinality::Index::Index(MutableColumnPtr && positions) : positions(std::move(positions)) { updateSizeOfType(); } -ColumnWithDictionary::Index::Index(ColumnPtr positions) : positions(std::move(positions)) +ColumnLowCardinality::Index::Index(ColumnPtr positions) : positions(std::move(positions)) { updateSizeOfType(); } template -void ColumnWithDictionary::Index::callForType(Callback && callback, size_t size_of_type) +void ColumnLowCardinality::Index::callForType(Callback && callback, size_t size_of_type) { switch (size_of_type) { @@ -390,13 +390,13 @@ void ColumnWithDictionary::Index::callForType(Callback && callback, size_t size_ case sizeof(UInt32): { callback(UInt32()); break; } case sizeof(UInt64): { callback(UInt64()); break; } default: { - throw Exception("Unexpected size of index type for ColumnWithDictionary: " + toString(size_of_type), + throw Exception("Unexpected size of index type for ColumnLowCardinality: " + toString(size_of_type), ErrorCodes::LOGICAL_ERROR); } } } -size_t ColumnWithDictionary::Index::getSizeOfIndexType(const IColumn & column, size_t hint) +size_t ColumnLowCardinality::Index::getSizeOfIndexType(const IColumn & column, size_t hint) { auto checkFor = [&](auto type) { return typeid_cast *>(&column) != nullptr; }; auto tryGetSizeFor = [&](auto type) -> size_t { return checkFor(type) ? sizeof(decltype(type)) : 0; }; @@ -419,22 +419,22 @@ size_t ColumnWithDictionary::Index::getSizeOfIndexType(const IColumn & column, s if (auto size = tryGetSizeFor(UInt64())) return size; - throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected UInt, got " + column.getName(), + throw Exception("Unexpected indexes type for ColumnLowCardinality. Expected UInt, got " + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } -void ColumnWithDictionary::Index::attachPositions(ColumnPtr positions_) +void ColumnLowCardinality::Index::attachPositions(ColumnPtr positions_) { positions = std::move(positions_); updateSizeOfType(); } template -typename ColumnVector::Container & ColumnWithDictionary::Index::getPositionsData() +typename ColumnVector::Container & ColumnLowCardinality::Index::getPositionsData() { auto * positions_ptr = typeid_cast *>(positions->assumeMutable().get()); if (!positions_ptr) - throw Exception("Invalid indexes type for ColumnWithDictionary." + throw Exception("Invalid indexes type for ColumnLowCardinality." " Expected UInt" + toString(8 * sizeof(IndexType)) + ", got " + positions->getName(), ErrorCodes::LOGICAL_ERROR); @@ -442,11 +442,11 @@ typename ColumnVector::Container & ColumnWithDictionary::Index::getPo } template -const typename ColumnVector::Container & ColumnWithDictionary::Index::getPositionsData() const +const typename ColumnVector::Container & ColumnLowCardinality::Index::getPositionsData() const { const auto * positions_ptr = typeid_cast *>(positions.get()); if (!positions_ptr) - throw Exception("Invalid indexes type for ColumnWithDictionary." + throw Exception("Invalid indexes type for ColumnLowCardinality." " Expected UInt" + toString(8 * sizeof(IndexType)) + ", got " + positions->getName(), ErrorCodes::LOGICAL_ERROR); @@ -454,7 +454,7 @@ const typename ColumnVector::Container & ColumnWithDictionary::Index: } template -void ColumnWithDictionary::Index::convertPositions() +void ColumnLowCardinality::Index::convertPositions() { auto convert = [&](auto x) { @@ -485,14 +485,14 @@ void ColumnWithDictionary::Index::convertPositions() checkSizeOfType(); } -void ColumnWithDictionary::Index::expandType() +void ColumnLowCardinality::Index::expandType() { auto expand = [&](auto type) { using CurIndexType = decltype(type); constexpr auto next_size = NumberTraits::nextSize(sizeof(CurIndexType)); if (next_size == sizeof(CurIndexType)) - throw Exception("Can't expand indexes type for ColumnWithDictionary from type: " + throw Exception("Can't expand indexes type for ColumnLowCardinality from type: " + demangle(typeid(CurIndexType).name()), ErrorCodes::LOGICAL_ERROR); using NewIndexType = typename NumberTraits::Construct::Type; @@ -502,14 +502,14 @@ void ColumnWithDictionary::Index::expandType() callForType(std::move(expand), size_of_type); } -UInt64 ColumnWithDictionary::Index::getMaxPositionForCurrentType() const +UInt64 ColumnLowCardinality::Index::getMaxPositionForCurrentType() const { UInt64 value = 0; callForType([&](auto type) { value = std::numeric_limits::max(); }, size_of_type); return value; } -size_t ColumnWithDictionary::Index::getPositionAt(size_t row) const +size_t ColumnLowCardinality::Index::getPositionAt(size_t row) const { size_t pos; auto getPosition = [&](auto type) @@ -522,7 +522,7 @@ size_t ColumnWithDictionary::Index::getPositionAt(size_t row) const return pos; } -void ColumnWithDictionary::Index::insertPosition(UInt64 position) +void ColumnLowCardinality::Index::insertPosition(UInt64 position) { while (position > getMaxPositionForCurrentType()) expandType(); @@ -531,7 +531,7 @@ void ColumnWithDictionary::Index::insertPosition(UInt64 position) checkSizeOfType(); } -void ColumnWithDictionary::Index::insertPositionsRange(const IColumn & column, size_t offset, size_t limit) +void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, size_t offset, size_t limit) { auto insertForType = [&](auto type) { @@ -571,13 +571,13 @@ void ColumnWithDictionary::Index::insertPositionsRange(const IColumn & column, s !insertForType(UInt16()) && !insertForType(UInt32()) && !insertForType(UInt64())) - throw Exception("Invalid column for ColumnWithDictionary index. Expected UInt, got " + column.getName(), + throw Exception("Invalid column for ColumnLowCardinality index. Expected UInt, got " + column.getName(), ErrorCodes::ILLEGAL_COLUMN); checkSizeOfType(); } -void ColumnWithDictionary::Index::check(size_t /*max_dictionary_size*/) +void ColumnLowCardinality::Index::check(size_t /*max_dictionary_size*/) { /// TODO: remove /* @@ -601,14 +601,14 @@ void ColumnWithDictionary::Index::check(size_t /*max_dictionary_size*/) */ } -void ColumnWithDictionary::Index::checkSizeOfType() +void ColumnLowCardinality::Index::checkSizeOfType() { if (size_of_type != getSizeOfIndexType(*positions, size_of_type)) throw Exception("Invalid size of type. Expected " + toString(8 * size_of_type) + ", but positions are " + positions->getName(), ErrorCodes::LOGICAL_ERROR); } -void ColumnWithDictionary::Index::countKeys(ColumnUInt64::Container & counts) const +void ColumnLowCardinality::Index::countKeys(ColumnUInt64::Container & counts) const { auto counter = [&](auto x) { @@ -621,25 +621,25 @@ void ColumnWithDictionary::Index::countKeys(ColumnUInt64::Container & counts) co } -ColumnWithDictionary::Dictionary::Dictionary(MutableColumnPtr && column_unique_, bool is_shared) +ColumnLowCardinality::Dictionary::Dictionary(MutableColumnPtr && column_unique_, bool is_shared) : column_unique(std::move(column_unique_)), shared(is_shared) { checkColumn(*column_unique); } -ColumnWithDictionary::Dictionary::Dictionary(ColumnPtr column_unique_, bool is_shared) +ColumnLowCardinality::Dictionary::Dictionary(ColumnPtr column_unique_, bool is_shared) : column_unique(std::move(column_unique_)), shared(is_shared) { checkColumn(*column_unique); } -void ColumnWithDictionary::Dictionary::checkColumn(const IColumn & column) +void ColumnLowCardinality::Dictionary::checkColumn(const IColumn & column) { if (!dynamic_cast(&column)) - throw Exception("ColumnUnique expected as an argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); + throw Exception("ColumnUnique expected as an argument of ColumnLowCardinality.", ErrorCodes::ILLEGAL_COLUMN); } -void ColumnWithDictionary::Dictionary::setShared(const ColumnPtr & dictionary) +void ColumnLowCardinality::Dictionary::setShared(const ColumnPtr & dictionary) { checkColumn(*dictionary); @@ -647,7 +647,7 @@ void ColumnWithDictionary::Dictionary::setShared(const ColumnPtr & dictionary) shared = true; } -void ColumnWithDictionary::Dictionary::compact(ColumnPtr & positions) +void ColumnLowCardinality::Dictionary::compact(ColumnPtr & positions) { auto new_column_unique = column_unique->cloneEmpty(); diff --git a/dbms/src/Columns/ColumnWithDictionary.h b/dbms/src/Columns/ColumnLowCardinality.h similarity index 91% rename from dbms/src/Columns/ColumnWithDictionary.h rename to dbms/src/Columns/ColumnLowCardinality.h index 7ca3a01853c..a125fee35ab 100644 --- a/dbms/src/Columns/ColumnWithDictionary.h +++ b/dbms/src/Columns/ColumnLowCardinality.h @@ -13,21 +13,21 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -class ColumnWithDictionary final : public COWPtrHelper +class ColumnLowCardinality final : public COWPtrHelper { - friend class COWPtrHelper; + friend class COWPtrHelper; - ColumnWithDictionary(MutableColumnPtr && column_unique, MutableColumnPtr && indexes, bool is_shared = false); - ColumnWithDictionary(const ColumnWithDictionary & other) = default; + ColumnLowCardinality(MutableColumnPtr && column_unique, MutableColumnPtr && indexes, bool is_shared = false); + ColumnLowCardinality(const ColumnLowCardinality & other) = default; public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. */ - using Base = COWPtrHelper; + using Base = COWPtrHelper; static Ptr create(const ColumnPtr & column_unique_, const ColumnPtr & indexes_, bool is_shared = false) { - return ColumnWithDictionary::create(column_unique_->assumeMutable(), indexes_->assumeMutable(), is_shared); + return ColumnLowCardinality::create(column_unique_->assumeMutable(), indexes_->assumeMutable(), is_shared); } static MutablePtr create(MutableColumnPtr && column_unique, MutableColumnPtr && indexes, bool is_shared = false) @@ -35,11 +35,11 @@ public: return Base::create(std::move(column_unique), std::move(indexes), is_shared); } - std::string getName() const override { return "ColumnWithDictionary"; } - const char * getFamilyName() const override { return "ColumnWithDictionary"; } + std::string getName() const override { return "ColumnLowCardinality"; } + const char * getFamilyName() const override { return "ColumnLowCardinality"; } ColumnPtr convertToFullColumn() const { return getDictionary().getNestedColumn()->index(getIndexes(), 0); } - ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); } + ColumnPtr convertToFullColumnIfLowCardinality() const override { return convertToFullColumn(); } MutableColumnPtr cloneResized(size_t size) const override; size_t size() const override { return getIndexes().size(); } @@ -59,7 +59,7 @@ public: bool isNullAt(size_t n) const override { return getDictionary().isNullAt(getIndexes().getUInt(n)); } ColumnPtr cut(size_t start, size_t length) const override { - return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().cut(start, length)); + return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().cut(start, length)); } void insert(const Field & x) override; @@ -89,17 +89,17 @@ public: ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { - return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint)); + return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint)); } ColumnPtr permute(const Permutation & perm, size_t limit) const override { - return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().permute(perm, limit)); + return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().permute(perm, limit)); } ColumnPtr index(const IColumn & indexes_, size_t limit) const override { - return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().index(indexes_, limit)); + return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().index(indexes_, limit)); } int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; @@ -108,7 +108,7 @@ public: ColumnPtr replicate(const Offsets & offsets) const override { - return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets)); + return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets)); } std::vector scatter(ColumnIndex num_columns, const Selector & selector) const override; @@ -138,7 +138,7 @@ public: bool isFixedAndContiguous() const override { return getDictionary().isFixedAndContiguous(); } size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); } bool isNumeric() const override { return getDictionary().isNumeric(); } - bool withDictionary() const override { return true; } + bool lowCardinality() const override { return true; } const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); } const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); } @@ -166,7 +166,7 @@ public: ///void setIndexes(MutableColumnPtr && indexes_) { indexes = std::move(indexes_); } - /// Set shared ColumnUnique for empty column with dictionary. + /// Set shared ColumnUnique for empty low cardinality column. void setSharedDictionary(const ColumnPtr & column_unique); bool isSharedDictionary() const { return dictionary.isShared(); } diff --git a/dbms/src/Columns/FilterDescription.cpp b/dbms/src/Columns/FilterDescription.cpp index 92c2765feda..104599ba974 100644 --- a/dbms/src/Columns/FilterDescription.cpp +++ b/dbms/src/Columns/FilterDescription.cpp @@ -27,7 +27,7 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) if (column.isColumnConst()) { const ColumnConst & column_const = static_cast(column); - ColumnPtr column_nested = column_const.getDataColumnPtr()->convertToFullColumnIfWithDictionary(); + ColumnPtr column_nested = column_const.getDataColumnPtr()->convertToFullColumnIfLowCardinality(); if (!typeid_cast(column_nested.get())) { @@ -50,8 +50,8 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) FilterDescription::FilterDescription(const IColumn & column_) { - if (column_.withDictionary()) - data_holder = column_.convertToFullColumnIfWithDictionary(); + if (column_.lowCardinality()) + data_holder = column_.convertToFullColumnIfLowCardinality(); const auto & column = data_holder ? *data_holder : column_; diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index e1e088fbf0c..034bf9d5ca4 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -47,9 +47,9 @@ public: */ virtual Ptr convertToFullColumnIfConst() const { return {}; } - /// If column isn't ColumnWithDictionary, return itself. - /// If column is ColumnWithDictionary, transforms is to full column. - virtual Ptr convertToFullColumnIfWithDictionary() const { return getPtr(); } + /// If column isn't ColumnLowCardinality, return itself. + /// If column is ColumnLowCardinality, transforms is to full column. + virtual Ptr convertToFullColumnIfLowCardinality() const { return getPtr(); } /// Creates empty column with the same type. virtual MutablePtr cloneEmpty() const { return cloneResized(0); } @@ -333,7 +333,7 @@ public: /// Can be inside ColumnNullable. virtual bool canBeInsideNullable() const { return false; } - virtual bool withDictionary() const { return false; } + virtual bool lowCardinality() const { return false; } virtual ~IColumn() {} diff --git a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h b/dbms/src/DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h similarity index 75% rename from dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h rename to dbms/src/DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h index 1e3ce882dab..2ced0adbfcf 100644 --- a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h +++ b/dbms/src/DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h @@ -1,8 +1,8 @@ #pragma once #include -#include -#include +#include +#include #include namespace DB @@ -13,15 +13,15 @@ namespace DB * Unlike UnionBlockInputStream, it does this sequentially. * Blocks of different sources are not interleaved with each other. */ -class ConvertColumnWithDictionaryToFullBlockInputStream : public IProfilingBlockInputStream +class ConvertColumnLowCardinalityToFullBlockInputStream : public IProfilingBlockInputStream { public: - explicit ConvertColumnWithDictionaryToFullBlockInputStream(const BlockInputStreamPtr & input) + explicit ConvertColumnLowCardinalityToFullBlockInputStream(const BlockInputStreamPtr & input) { children.push_back(input); } - String getName() const override { return "ConvertColumnWithDictionaryToFull"; } + String getName() const override { return "ConvertColumnLowCardinalityToFull"; } Block getHeader() const override { return convert(children.at(0)->getHeader()); } @@ -36,9 +36,9 @@ private: if (auto * column_const = typeid_cast(column.column.get())) column.column = column_const->removeLowCardinality(); else - column.column = column.column->convertToFullColumnIfWithDictionary(); + column.column = column.column->convertToFullColumnIfLowCardinality(); - if (auto * low_cardinality_type = typeid_cast(column.type.get())) + if (auto * low_cardinality_type = typeid_cast(column.type.get())) column.type = low_cardinality_type->getDictionaryType(); } diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index ad123a7431b..8689efbd5f7 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -143,7 +143,7 @@ void registerDataTypeUUID(DataTypeFactory & factory); void registerDataTypeAggregateFunction(DataTypeFactory & factory); void registerDataTypeNested(DataTypeFactory & factory); void registerDataTypeInterval(DataTypeFactory & factory); -void registerDataTypeWithDictionary(DataTypeFactory & factory); +void registerDataTypeLowCardinality(DataTypeFactory & factory); DataTypeFactory::DataTypeFactory() @@ -163,7 +163,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeAggregateFunction(*this); registerDataTypeNested(*this); registerDataTypeInterval(*this); - registerDataTypeWithDictionary(*this); + registerDataTypeLowCardinality(*this); } } diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.cpp b/dbms/src/DataTypes/DataTypeLowCardinality.cpp similarity index 77% rename from dbms/src/DataTypes/DataTypeWithDictionary.cpp rename to dbms/src/DataTypes/DataTypeLowCardinality.cpp index 23a07ecc590..1e973bc9c9b 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinality.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -24,18 +24,18 @@ namespace ErrorCodes namespace { - const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column) + const ColumnLowCardinality & getColumnLowCardinality(const IColumn & column) { - return typeid_cast(column); + return typeid_cast(column); } - ColumnWithDictionary & getColumnWithDictionary(IColumn & column) + ColumnLowCardinality & getColumnLowCardinality(IColumn & column) { - return typeid_cast(column); + return typeid_cast(column); } } -DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_) +DataTypeLowCardinality::DataTypeLowCardinality(DataTypePtr dictionary_type_) : dictionary_type(std::move(dictionary_type_)) { auto inner_type = dictionary_type; @@ -45,11 +45,11 @@ DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_) if (!isStringOrFixedString(inner_type) && !isDateOrDateTime(inner_type) && !isNumber(inner_type)) - throw Exception("DataTypeWithDictionary is supported only for numbers, strings, Date or DateTime, but got " + throw Exception("DataTypeLowCardinality is supported only for numbers, strings, Date or DateTime, but got " + dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } -void DataTypeWithDictionary::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void DataTypeLowCardinality::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::DictionaryKeys); dictionary_type->enumerateStreams(callback, path); @@ -74,7 +74,7 @@ struct KeysSerializationVersion static void checkVersion(UInt64 version) { if (version != SharedDictionariesWithAdditionalKeys) - throw Exception("Invalid version for DataTypeWithDictionary key column.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Invalid version for DataTypeLowCardinality key column.", ErrorCodes::LOGICAL_ERROR); } KeysSerializationVersion(UInt64 version) : value(static_cast(version)) { checkVersion(version); } @@ -115,7 +115,7 @@ struct IndexesSerializationType if (value <= TUInt64) return; - throw Exception("Invalid type for DataTypeWithDictionary index column.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Invalid type for DataTypeLowCardinality index column.", ErrorCodes::LOGICAL_ERROR); } void serialize(WriteBuffer & buffer) const @@ -179,15 +179,15 @@ struct IndexesSerializationType IndexesSerializationType() = default; }; -struct SerializeStateWithDictionary : public IDataType::SerializeBinaryBulkState +struct SerializeStateLowCardinality : public IDataType::SerializeBinaryBulkState { KeysSerializationVersion key_version; MutableColumnUniquePtr shared_dictionary; - explicit SerializeStateWithDictionary(UInt64 key_version) : key_version(key_version) {} + explicit SerializeStateLowCardinality(UInt64 key_version) : key_version(key_version) {} }; -struct DeserializeStateWithDictionary : public IDataType::DeserializeBinaryBulkState +struct DeserializeStateLowCardinality : public IDataType::DeserializeBinaryBulkState { KeysSerializationVersion key_version; ColumnUniquePtr global_dictionary; @@ -197,46 +197,46 @@ struct DeserializeStateWithDictionary : public IDataType::DeserializeBinaryBulkS ColumnPtr null_map; UInt64 num_pending_rows = 0; - explicit DeserializeStateWithDictionary(UInt64 key_version) : key_version(key_version) {} + explicit DeserializeStateLowCardinality(UInt64 key_version) : key_version(key_version) {} }; -static SerializeStateWithDictionary * checkAndGetWithDictionarySerializeState( +static SerializeStateLowCardinality * checkAndGetLowCardinalitySerializeState( IDataType::SerializeBinaryBulkStatePtr & state) { if (!state) - throw Exception("Got empty state for DataTypeWithDictionary.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Got empty state for DataTypeLowCardinality.", ErrorCodes::LOGICAL_ERROR); - auto * with_dictionary_state = typeid_cast(state.get()); - if (!with_dictionary_state) + auto * low_cardinality_state = typeid_cast(state.get()); + if (!low_cardinality_state) { auto & state_ref = *state; - throw Exception("Invalid SerializeBinaryBulkState for DataTypeWithDictionary. Expected: " - + demangle(typeid(SerializeStateWithDictionary).name()) + ", got " + throw Exception("Invalid SerializeBinaryBulkState for DataTypeLowCardinality. Expected: " + + demangle(typeid(SerializeStateLowCardinality).name()) + ", got " + demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR); } - return with_dictionary_state; + return low_cardinality_state; } -static DeserializeStateWithDictionary * checkAndGetWithDictionaryDeserializeState( +static DeserializeStateLowCardinality * checkAndGetLowCardinalityDeserializeState( IDataType::DeserializeBinaryBulkStatePtr & state) { if (!state) - throw Exception("Got empty state for DataTypeWithDictionary.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Got empty state for DataTypeLowCardinality.", ErrorCodes::LOGICAL_ERROR); - auto * with_dictionary_state = typeid_cast(state.get()); - if (!with_dictionary_state) + auto * low_cardinality_state = typeid_cast(state.get()); + if (!low_cardinality_state) { auto & state_ref = *state; - throw Exception("Invalid DeserializeBinaryBulkState for DataTypeWithDictionary. Expected: " - + demangle(typeid(DeserializeStateWithDictionary).name()) + ", got " + throw Exception("Invalid DeserializeBinaryBulkState for DataTypeLowCardinality. Expected: " + + demangle(typeid(DeserializeStateLowCardinality).name()) + ", got " + demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR); } - return with_dictionary_state; + return low_cardinality_state; } -void DataTypeWithDictionary::serializeBinaryBulkStatePrefix( +void DataTypeLowCardinality::serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -245,7 +245,7 @@ void DataTypeWithDictionary::serializeBinaryBulkStatePrefix( settings.path.pop_back(); if (!stream) - throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStatePrefix", + throw Exception("Got empty stream in DataTypeLowCardinality::serializeBinaryBulkStatePrefix", ErrorCodes::LOGICAL_ERROR); /// Write version and create SerializeBinaryBulkState. @@ -253,36 +253,36 @@ void DataTypeWithDictionary::serializeBinaryBulkStatePrefix( writeIntBinary(key_version, *stream); - state = std::make_shared(key_version); + state = std::make_shared(key_version); } -void DataTypeWithDictionary::serializeBinaryBulkStateSuffix( +void DataTypeLowCardinality::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - auto * state_with_dictionary = checkAndGetWithDictionarySerializeState(state); - KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value); + auto * low_cardinality_state = checkAndGetLowCardinalitySerializeState(state); + KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); - if (state_with_dictionary->shared_dictionary && settings.low_cardinality_max_dictionary_size) + if (low_cardinality_state->shared_dictionary && settings.low_cardinality_max_dictionary_size) { - auto nested_column = state_with_dictionary->shared_dictionary->getNestedNotNullableColumn(); + auto nested_column = low_cardinality_state->shared_dictionary->getNestedNotNullableColumn(); settings.path.push_back(Substream::DictionaryKeys); auto * stream = settings.getter(settings.path); settings.path.pop_back(); if (!stream) - throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStateSuffix", + throw Exception("Got empty stream in DataTypeLowCardinality::serializeBinaryBulkStateSuffix", ErrorCodes::LOGICAL_ERROR); UInt64 num_keys = nested_column->size(); writeIntBinary(num_keys, *stream); removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *stream, 0, num_keys); - state_with_dictionary->shared_dictionary = nullptr; + low_cardinality_state->shared_dictionary = nullptr; } } -void DataTypeWithDictionary::deserializeBinaryBulkStatePrefix( +void DataTypeLowCardinality::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { @@ -296,7 +296,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkStatePrefix( UInt64 keys_version; readIntBinary(keys_version, *stream); - state = std::make_shared(keys_version); + state = std::make_shared(keys_version); } namespace @@ -475,7 +475,7 @@ namespace } } -void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( +void DataTypeLowCardinality::serializeBinaryBulkWithMultipleStreams( const IColumn & column, size_t offset, size_t limit, @@ -492,16 +492,16 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( return; if (!keys_stream) - throw Exception("Got empty stream for DataTypeWithDictionary keys.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Got empty stream for DataTypeLowCardinality keys.", ErrorCodes::LOGICAL_ERROR); if (!indexes_stream) - throw Exception("Got empty stream for DataTypeWithDictionary indexes.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Got empty stream for DataTypeLowCardinality indexes.", ErrorCodes::LOGICAL_ERROR); - const ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + const ColumnLowCardinality & low_cardinality_column = typeid_cast(column); - auto * state_with_dictionary = checkAndGetWithDictionarySerializeState(state); - auto & global_dictionary = state_with_dictionary->shared_dictionary; - KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value); + auto * low_cardinality_state = checkAndGetLowCardinalitySerializeState(state); + auto & global_dictionary = low_cardinality_state->shared_dictionary; + KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); bool need_update_dictionary = global_dictionary == nullptr; if (need_update_dictionary) @@ -510,7 +510,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( size_t max_limit = column.size() - offset; limit = limit ? std::min(limit, max_limit) : max_limit; - auto sub_column = column_with_dictionary.cutAndCompact(offset, limit); + auto sub_column = low_cardinality_column.cutAndCompact(offset, limit); ColumnPtr positions = sub_column->getIndexesPtr(); ColumnPtr keys = sub_column->getDictionary().getNestedColumn(); @@ -520,7 +520,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( auto indexes_with_overflow = global_dictionary->uniqueInsertRangeWithOverflow(*keys, 0, keys->size(), settings.low_cardinality_max_dictionary_size); size_t max_size = settings.low_cardinality_max_dictionary_size + indexes_with_overflow.overflowed_keys->size(); - ColumnWithDictionary::Index(indexes_with_overflow.indexes->getPtr()).check(max_size); + ColumnLowCardinality::Index(indexes_with_overflow.indexes->getPtr()).check(max_size); if (global_dictionary->size() > settings.low_cardinality_max_dictionary_size) throw Exception("Got dictionary with size " + toString(global_dictionary->size()) + @@ -553,7 +553,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( UInt64 num_keys = nested_column->size(); writeIntBinary(num_keys, *keys_stream); removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *keys_stream, 0, num_keys); - state_with_dictionary->shared_dictionary = nullptr; + low_cardinality_state->shared_dictionary = nullptr; } if (need_additional_keys) @@ -568,13 +568,13 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams( index_version.getDataType()->serializeBinaryBulk(*positions, *indexes_stream, 0, num_rows); } -void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( +void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { - ColumnWithDictionary & column_with_dictionary = typeid_cast(column); + ColumnLowCardinality & low_cardinality_column = typeid_cast(column); settings.path.push_back(Substream::DictionaryKeys); auto * keys_stream = settings.getter(settings.path); @@ -586,15 +586,15 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( return; if (!keys_stream) - throw Exception("Got empty stream for DataTypeWithDictionary keys.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Got empty stream for DataTypeLowCardinality keys.", ErrorCodes::LOGICAL_ERROR); if (!indexes_stream) - throw Exception("Got empty stream for DataTypeWithDictionary indexes.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Got empty stream for DataTypeLowCardinality indexes.", ErrorCodes::LOGICAL_ERROR); - auto * state_with_dictionary = checkAndGetWithDictionaryDeserializeState(state); - KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value); + auto * low_cardinality_state = checkAndGetLowCardinalityDeserializeState(state); + KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); - auto readDictionary = [this, state_with_dictionary, keys_stream]() + auto readDictionary = [this, low_cardinality_state, keys_stream]() { UInt64 num_keys; readIntBinary(num_keys, *keys_stream); @@ -604,62 +604,62 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( keys_type->deserializeBinaryBulk(*global_dict_keys, *keys_stream, num_keys, 0); auto column_unique = createColumnUnique(*dictionary_type, std::move(global_dict_keys)); - state_with_dictionary->global_dictionary = std::move(column_unique); + low_cardinality_state->global_dictionary = std::move(column_unique); }; - auto readAdditionalKeys = [this, state_with_dictionary, indexes_stream]() + auto readAdditionalKeys = [this, low_cardinality_state, indexes_stream]() { UInt64 num_keys; readIntBinary(num_keys, *indexes_stream); auto keys_type = removeNullable(dictionary_type); auto additional_keys = keys_type->createColumn(); keys_type->deserializeBinaryBulk(*additional_keys, *indexes_stream, num_keys, 0); - state_with_dictionary->additional_keys = std::move(additional_keys); + low_cardinality_state->additional_keys = std::move(additional_keys); - if (!state_with_dictionary->index_type.need_global_dictionary && dictionary_type->isNullable()) + if (!low_cardinality_state->index_type.need_global_dictionary && dictionary_type->isNullable()) { auto null_map = ColumnUInt8::create(num_keys, 0); if (num_keys) null_map->getElement(0) = 1; - state_with_dictionary->null_map = std::move(null_map); + low_cardinality_state->null_map = std::move(null_map); } }; - auto readIndexes = [this, state_with_dictionary, indexes_stream, &column_with_dictionary](UInt64 num_rows) + auto readIndexes = [this, low_cardinality_state, indexes_stream, &low_cardinality_column](UInt64 num_rows) { - auto indexes_type = state_with_dictionary->index_type.getDataType(); + auto indexes_type = low_cardinality_state->index_type.getDataType(); MutableColumnPtr indexes_column = indexes_type->createColumn(); indexes_type->deserializeBinaryBulk(*indexes_column, *indexes_stream, num_rows, 0); - auto & global_dictionary = state_with_dictionary->global_dictionary; - const auto & additional_keys = state_with_dictionary->additional_keys; + auto & global_dictionary = low_cardinality_state->global_dictionary; + const auto & additional_keys = low_cardinality_state->additional_keys; - bool has_additional_keys = state_with_dictionary->index_type.has_additional_keys; - bool column_is_empty = column_with_dictionary.empty(); + bool has_additional_keys = low_cardinality_state->index_type.has_additional_keys; + bool column_is_empty = low_cardinality_column.empty(); - if (!state_with_dictionary->index_type.need_global_dictionary) + if (!low_cardinality_state->index_type.need_global_dictionary) { ColumnPtr keys_column = additional_keys; - if (state_with_dictionary->null_map) - keys_column = ColumnNullable::create(additional_keys, state_with_dictionary->null_map); - column_with_dictionary.insertRangeFromDictionaryEncodedColumn(*keys_column, *indexes_column); + if (low_cardinality_state->null_map) + keys_column = ColumnNullable::create(additional_keys, low_cardinality_state->null_map); + low_cardinality_column.insertRangeFromDictionaryEncodedColumn(*keys_column, *indexes_column); } else if (!has_additional_keys) { if (column_is_empty) - column_with_dictionary.setSharedDictionary(global_dictionary); + low_cardinality_column.setSharedDictionary(global_dictionary); - auto local_column = ColumnWithDictionary::create(global_dictionary, std::move(indexes_column)); - column_with_dictionary.insertRangeFrom(*local_column, 0, num_rows); + auto local_column = ColumnLowCardinality::create(global_dictionary, std::move(indexes_column)); + low_cardinality_column.insertRangeFrom(*local_column, 0, num_rows); } else { auto maps = mapIndexWithAdditionalKeys(*indexes_column, global_dictionary->size()); - ColumnWithDictionary::Index(maps.additional_keys_map->getPtr()).check(additional_keys->size()); + ColumnLowCardinality::Index(maps.additional_keys_map->getPtr()).check(additional_keys->size()); - ColumnWithDictionary::Index(indexes_column->getPtr()).check( + ColumnLowCardinality::Index(indexes_column->getPtr()).check( maps.dictionary_map->size() + maps.additional_keys_map->size()); auto used_keys = (*std::move(global_dictionary->getNestedColumn()->index(*maps.dictionary_map, 0))).mutate(); @@ -677,23 +677,23 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( used_keys->insertRangeFrom(*used_add_keys, 0, used_add_keys->size()); } - column_with_dictionary.insertRangeFromDictionaryEncodedColumn(*used_keys, *indexes_column); + low_cardinality_column.insertRangeFromDictionaryEncodedColumn(*used_keys, *indexes_column); } }; if (!settings.continuous_reading) - state_with_dictionary->num_pending_rows = 0; + low_cardinality_state->num_pending_rows = 0; bool first_dictionary = true; while (limit) { - if (state_with_dictionary->num_pending_rows == 0) + if (low_cardinality_state->num_pending_rows == 0) { if (indexes_stream->eof()) break; - auto & index_type = state_with_dictionary->index_type; - auto & global_dictionary = state_with_dictionary->global_dictionary; + auto & index_type = low_cardinality_state->index_type; + auto & global_dictionary = low_cardinality_state->global_dictionary; index_type.deserialize(*indexes_stream); @@ -703,51 +703,51 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams( first_dictionary = false; } - if (state_with_dictionary->index_type.has_additional_keys) + if (low_cardinality_state->index_type.has_additional_keys) readAdditionalKeys(); else - state_with_dictionary->additional_keys = nullptr; + low_cardinality_state->additional_keys = nullptr; - readIntBinary(state_with_dictionary->num_pending_rows, *indexes_stream); + readIntBinary(low_cardinality_state->num_pending_rows, *indexes_stream); } - size_t num_rows_to_read = std::min(limit, state_with_dictionary->num_pending_rows); + size_t num_rows_to_read = std::min(limit, low_cardinality_state->num_pending_rows); readIndexes(num_rows_to_read); limit -= num_rows_to_read; - state_with_dictionary->num_pending_rows -= num_rows_to_read; + low_cardinality_state->num_pending_rows -= num_rows_to_read; } } -void DataTypeWithDictionary::serializeBinary(const Field & field, WriteBuffer & ostr) const +void DataTypeLowCardinality::serializeBinary(const Field & field, WriteBuffer & ostr) const { dictionary_type->serializeBinary(field, ostr); } -void DataTypeWithDictionary::deserializeBinary(Field & field, ReadBuffer & istr) const +void DataTypeLowCardinality::deserializeBinary(Field & field, ReadBuffer & istr) const { dictionary_type->deserializeBinary(field, istr); } template -void DataTypeWithDictionary::serializeImpl( +void DataTypeLowCardinality::serializeImpl( const IColumn & column, size_t row_num, WriteBuffer & ostr, - DataTypeWithDictionary::SerealizeFunctionPtr func, Args & ... args) const + DataTypeLowCardinality::SerealizeFunctionPtr func, Args & ... args) const { - auto & column_with_dictionary = getColumnWithDictionary(column); - size_t unique_row_number = column_with_dictionary.getIndexes().getUInt(row_num); - (dictionary_type.get()->*func)(*column_with_dictionary.getDictionary().getNestedColumn(), unique_row_number, ostr, std::forward(args)...); + auto & low_cardinality_column = getColumnLowCardinality(column); + size_t unique_row_number = low_cardinality_column.getIndexes().getUInt(row_num); + (dictionary_type.get()->*func)(*low_cardinality_column.getDictionary().getNestedColumn(), unique_row_number, ostr, std::forward(args)...); } template -void DataTypeWithDictionary::deserializeImpl( +void DataTypeLowCardinality::deserializeImpl( IColumn & column, ReadBuffer & istr, - DataTypeWithDictionary::DeserealizeFunctionPtr func, Args & ... args) const + DataTypeLowCardinality::DeserealizeFunctionPtr func, Args & ... args) const { - auto & column_with_dictionary = getColumnWithDictionary(column); - auto temp_column = column_with_dictionary.getDictionary().getNestedColumn()->cloneEmpty(); + auto & low_cardinality_column= getColumnLowCardinality(column); + auto temp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); (dictionary_type.get()->*func)(*temp_column, istr, std::forward(args)...); - column_with_dictionary.insertFromFullColumn(*temp_column, 0); + low_cardinality_column.insertFromFullColumn(*temp_column, 0); } namespace @@ -774,7 +774,7 @@ namespace } template -MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDataType & keys_type, +MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator) { auto * type = &keys_type; @@ -800,12 +800,12 @@ MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDat return column; } - throw Exception("Unexpected dictionary type for DataTypeWithDictionary: " + type->getName(), + throw Exception("Unexpected dictionary type for DataTypeLowCardinality: " + type->getName(), ErrorCodes::LOGICAL_ERROR); } -MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type) +MutableColumnUniquePtr DataTypeLowCardinality::createColumnUnique(const IDataType & keys_type) { auto creator = [&](auto x) { @@ -815,7 +815,7 @@ MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataTyp return createColumnUniqueImpl(keys_type, creator); } -MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys) +MutableColumnUniquePtr DataTypeLowCardinality::createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys) { auto creator = [&](auto x) { @@ -825,20 +825,20 @@ MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataTyp return createColumnUniqueImpl(keys_type, creator); } -MutableColumnPtr DataTypeWithDictionary::createColumn() const +MutableColumnPtr DataTypeLowCardinality::createColumn() const { MutableColumnPtr indexes = DataTypeUInt8().createColumn(); MutableColumnPtr dictionary = createColumnUnique(*dictionary_type); - return ColumnWithDictionary::create(std::move(dictionary), std::move(indexes)); + return ColumnLowCardinality::create(std::move(dictionary), std::move(indexes)); } -bool DataTypeWithDictionary::equals(const IDataType & rhs) const +bool DataTypeLowCardinality::equals(const IDataType & rhs) const { if (typeid(rhs) != typeid(*this)) return false; - auto & rhs_with_dictionary = static_cast(rhs); - return dictionary_type->equals(*rhs_with_dictionary.dictionary_type); + auto & low_cardinality_rhs= static_cast(rhs); + return dictionary_type->equals(*low_cardinality_rhs.dictionary_type); } @@ -848,10 +848,10 @@ static DataTypePtr create(const ASTPtr & arguments) throw Exception("LowCardinality data type family must have single argument - type of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return std::make_shared(DataTypeFactory::instance().get(arguments->children[0])); + return std::make_shared(DataTypeFactory::instance().get(arguments->children[0])); } -void registerDataTypeWithDictionary(DataTypeFactory & factory) +void registerDataTypeLowCardinality(DataTypeFactory & factory) { factory.registerDataType("LowCardinality", create); } @@ -859,8 +859,8 @@ void registerDataTypeWithDictionary(DataTypeFactory & factory) DataTypePtr removeLowCardinality(const DataTypePtr & type) { - if (auto * type_with_dictionary = typeid_cast(type.get())) - return type_with_dictionary->getDictionaryType(); + if (auto * low_cardinality_type = typeid_cast(type.get())) + return low_cardinality_type->getDictionaryType(); return type; } diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeLowCardinality.h similarity index 97% rename from dbms/src/DataTypes/DataTypeWithDictionary.h rename to dbms/src/DataTypes/DataTypeLowCardinality.h index af65d184e7e..a51e009534b 100644 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ b/dbms/src/DataTypes/DataTypeLowCardinality.h @@ -5,13 +5,13 @@ namespace DB { -class DataTypeWithDictionary : public IDataType +class DataTypeLowCardinality : public IDataType { private: DataTypePtr dictionary_type; public: - DataTypeWithDictionary(DataTypePtr dictionary_type_); + DataTypeLowCardinality(DataTypePtr dictionary_type_); const DataTypePtr & getDictionaryType() const { return dictionary_type; } @@ -136,7 +136,7 @@ public: bool isCategorial() const override { return false; } bool isNullable() const override { return false; } bool onlyNull() const override { return false; } - bool withDictionary() const override { return true; } + bool lowCardinality() const override { return true; } static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type); static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys); @@ -161,7 +161,7 @@ private: static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator); }; -/// Returns dictionary type if type is DataTypeWithDictionary, type otherwise. +/// Returns dictionary type if type is DataTypeLowCardinality, type otherwise. DataTypePtr removeLowCardinality(const DataTypePtr & type); } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index acf7cd27d56..e3f02c9b4f6 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -396,7 +396,7 @@ public: */ virtual bool canBeInsideNullable() const { return false; } - virtual bool withDictionary() const { return false; } + virtual bool lowCardinality() const { return false; } /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index a189d69e3d5..58e528a6ead 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -37,8 +37,8 @@ #include #include #include -#include -#include +#include +#include namespace DB @@ -1374,7 +1374,7 @@ protected: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } private: @@ -1750,10 +1750,10 @@ private: WrapperType prepareUnpackDictionaries(const DataTypePtr & from_type, const DataTypePtr & to_type) const { - const auto * from_with_dict = typeid_cast(from_type.get()); - const auto * to_with_dict = typeid_cast(to_type.get()); - const auto & from_nested = from_with_dict ? from_with_dict->getDictionaryType() : from_type; - const auto & to_nested = to_with_dict ? to_with_dict->getDictionaryType() : to_type; + const auto * from_low_cardinality = typeid_cast(from_type.get()); + const auto * to_low_cardinality = typeid_cast(to_type.get()); + const auto & from_nested = from_low_cardinality ? from_low_cardinality->getDictionaryType() : from_type; + const auto & to_nested = to_low_cardinality ? to_low_cardinality->getDictionaryType() : to_type; if (from_type->onlyNull()) { @@ -1768,10 +1768,10 @@ private: } auto wrapper = prepareRemoveNullable(from_nested, to_nested); - if (!from_with_dict && !to_with_dict) + if (!from_low_cardinality && !to_low_cardinality) return wrapper; - return [wrapper, from_with_dict, to_with_dict] + return [wrapper, from_low_cardinality, to_low_cardinality] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) { auto & arg = block.getByPosition(arguments[0]); @@ -1790,21 +1790,21 @@ private: auto tmp_rows_count = input_rows_count; - if (to_with_dict) - res.type = to_with_dict->getDictionaryType(); + if (to_low_cardinality) + res.type = to_low_cardinality->getDictionaryType(); - if (from_with_dict) + if (from_low_cardinality) { - auto * col_with_dict = typeid_cast(prev_arg_col.get()); - arg.column = col_with_dict->getDictionary().getNestedColumn(); - arg.type = from_with_dict->getDictionaryType(); + auto * col_low_cardinality = typeid_cast(prev_arg_col.get()); + arg.column = col_low_cardinality->getDictionary().getNestedColumn(); + arg.type = from_low_cardinality->getDictionaryType(); /// TODO: Make map with defaults conversion. src_converted_to_full_column = !removeNullable(arg.type)->equals(*removeNullable(res.type)); if (src_converted_to_full_column) - arg.column = arg.column->index(col_with_dict->getIndexes(), 0); + arg.column = arg.column->index(col_low_cardinality->getIndexes(), 0); else - res_indexes = col_with_dict->getIndexesPtr(); + res_indexes = col_low_cardinality->getIndexesPtr(); tmp_rows_count = arg.column->size(); } @@ -1817,18 +1817,18 @@ private: res.type = prev_res_type; } - if (to_with_dict) + if (to_low_cardinality) { - auto res_column = to_with_dict->createColumn(); - auto * col_with_dict = typeid_cast(res_column.get()); + auto res_column = to_low_cardinality->createColumn(); + auto * col_low_cardinality = typeid_cast(res_column.get()); - if (from_with_dict && !src_converted_to_full_column) + if (from_low_cardinality && !src_converted_to_full_column) { auto res_keys = std::move(res.column); - col_with_dict->insertRangeFromDictionaryEncodedColumn(*res_keys, *res_indexes); + col_low_cardinality->insertRangeFromDictionaryEncodedColumn(*res_keys, *res_indexes); } else - col_with_dict->insertRangeFromFullColumn(*res.column, 0, res.column->size()); + col_low_cardinality->insertRangeFromFullColumn(*res.column, 0, res.column->size()); res.column = std::move(res_column); } @@ -2026,7 +2026,7 @@ protected: } bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } private: template diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index f74eb26cd60..8d6142bc287 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -8,12 +8,12 @@ #include #include #include -#include +#include #include #include #include #include -#include +#include #include #include #include @@ -123,7 +123,7 @@ static DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type) return std::make_shared(elements); } - if (const auto * low_cardinality_type = typeid_cast(type.get())) + if (const auto * low_cardinality_type = typeid_cast(type.get())) return low_cardinality_type->getDictionaryType(); return type; @@ -148,7 +148,7 @@ static ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column) return ColumnTuple::create(columns); } - if (const auto * column_low_cardinality = typeid_cast(column.get())) + if (const auto * column_low_cardinality = typeid_cast(column.get())) return column_low_cardinality->convertToFullColumn(); return column; @@ -309,7 +309,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo for (size_t i = 0; i < arguments_size; ++i) temporary_argument_numbers[i] = i; - executeWithoutColumnsWithDictionary(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows()); + executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows()); block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column, input_rows_count); return true; @@ -333,7 +333,7 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co if (null_presence.has_nullable) { Block temporary_block = createBlockWithNestedColumns(block, args, result); - executeWithoutColumnsWithDictionary(temporary_block, args, result, temporary_block.rows()); + executeWithoutLowCardinalityColumns(temporary_block, args, result, temporary_block.rows()); block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args, result, input_rows_count); return true; @@ -342,7 +342,7 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co return false; } -void PreparedFunctionImpl::executeWithoutColumnsWithDictionary(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) +void PreparedFunctionImpl::executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) { if (defaultImplementationForConstantArguments(block, args, result, input_rows_count)) return; @@ -353,14 +353,14 @@ void PreparedFunctionImpl::executeWithoutColumnsWithDictionary(Block & block, co executeImpl(block, args, result, input_rows_count); } -static const ColumnWithDictionary * findLowCardinalityArgument(const Block & block, const ColumnNumbers & args) +static const ColumnLowCardinality * findLowCardinalityArgument(const Block & block, const ColumnNumbers & args) { - const ColumnWithDictionary * result_column = nullptr; + const ColumnLowCardinality * result_column = nullptr; for (auto arg : args) { const ColumnWithTypeAndName & column = block.getByPosition(arg); - if (auto * low_cardinality_column = checkAndGetColumn(column.column.get())) + if (auto * low_cardinality_column = checkAndGetColumn(column.column.get())) { if (result_column) throw Exception("Expected single dictionary argument for function.", ErrorCodes::LOGICAL_ERROR); @@ -372,7 +372,7 @@ static const ColumnWithDictionary * findLowCardinalityArgument(const Block & blo return result_column; } -static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes( +static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( Block & block, const ColumnNumbers & args, bool can_be_executed_on_default_arguments) { size_t num_rows = 0; @@ -381,13 +381,13 @@ static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes( for (auto arg : args) { ColumnWithTypeAndName & column = block.getByPosition(arg); - if (auto * column_with_dict = checkAndGetColumn(column.column.get())) + if (auto * low_cardinality_column = checkAndGetColumn(column.column.get())) { if (indexes) throw Exception("Expected single dictionary argument for function.", ErrorCodes::LOGICAL_ERROR); - indexes = column_with_dict->getIndexesPtr(); - num_rows = column_with_dict->getDictionary().size(); + indexes = low_cardinality_column->getIndexesPtr(); + num_rows = low_cardinality_column->getDictionary().size(); } } @@ -396,30 +396,30 @@ static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes( ColumnWithTypeAndName & column = block.getByPosition(arg); if (auto * column_const = checkAndGetColumn(column.column.get())) column.column = column_const->removeLowCardinality()->cloneResized(num_rows); - else if (auto * column_with_dict = checkAndGetColumn(column.column.get())) + else if (auto * low_cardinality_column = checkAndGetColumn(column.column.get())) { - auto * type_with_dict = checkAndGetDataType(column.type.get()); + auto * low_cardinality_type = checkAndGetDataType(column.type.get()); - if (!type_with_dict) - throw Exception("Incompatible type for column with dictionary: " + column.type->getName(), + if (!low_cardinality_type) + throw Exception("Incompatible type for low cardinality column: " + column.type->getName(), ErrorCodes::LOGICAL_ERROR); if (can_be_executed_on_default_arguments) - column.column = column_with_dict->getDictionary().getNestedColumn(); + column.column = low_cardinality_column->getDictionary().getNestedColumn(); else { - auto dict_encoded = column_with_dict->getMinimalDictionaryEncodedColumn(0, column_with_dict->size()); + auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size()); column.column = dict_encoded.dictionary; indexes = dict_encoded.indexes; } - column.type = type_with_dict->getDictionaryType(); + column.type = low_cardinality_type->getDictionaryType(); } } return indexes; } -static void convertColumnsWithDictionaryToFull(Block & block, const ColumnNumbers & args) +static void convertLowCardinalityColumnsToFull(Block & block, const ColumnNumbers & args) { for (auto arg : args) { @@ -432,15 +432,15 @@ static void convertColumnsWithDictionaryToFull(Block & block, const ColumnNumber void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) { - if (useDefaultImplementationForColumnsWithDictionary()) + if (useDefaultImplementationForLowCardinalityColumns()) { auto & res = block.safeGetByPosition(result); - Block block_without_dicts = block.cloneWithoutColumns(); + Block block_without_low_cardinality = block.cloneWithoutColumns(); for (auto arg : args) - block_without_dicts.safeGetByPosition(arg).column = block.safeGetByPosition(arg).column; + block_without_low_cardinality.safeGetByPosition(arg).column = block.safeGetByPosition(arg).column; - if (auto * res_type_with_dict = typeid_cast(res.type.get())) + if (auto * res_low_cardinality_type = typeid_cast(res.type.get())) { const auto * low_cardinality_column = findLowCardinalityArgument(block, args); bool can_be_executed_on_default_arguments = canBeExecutedOnDefaultArguments(); @@ -457,22 +457,22 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si if (cached_values) { auto indexes = cached_values->index_mapping->index(low_cardinality_column->getIndexes(), 0); - res.column = ColumnWithDictionary::create(cached_values->function_result, indexes, true); + res.column = ColumnLowCardinality::create(cached_values->function_result, indexes, true); return; } } - block_without_dicts.safeGetByPosition(result).type = res_type_with_dict->getDictionaryType(); - ColumnPtr indexes = replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes( - block_without_dicts, args, can_be_executed_on_default_arguments); + block_without_low_cardinality.safeGetByPosition(result).type = res_low_cardinality_type->getDictionaryType(); + ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( + block_without_low_cardinality, args, can_be_executed_on_default_arguments); - executeWithoutColumnsWithDictionary(block_without_dicts, args, result, block_without_dicts.rows()); + executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, block_without_low_cardinality.rows()); - auto & keys = block_without_dicts.safeGetByPosition(result).column; + auto & keys = block_without_low_cardinality.safeGetByPosition(result).column; if (auto full_column = keys->convertToFullColumnIfConst()) keys = full_column; - auto res_mut_dictionary = DataTypeWithDictionary::createColumnUnique(*res_type_with_dict->getDictionaryType()); + auto res_mut_dictionary = DataTypeLowCardinality::createColumnUnique(*res_low_cardinality_type->getDictionaryType()); ColumnPtr res_indexes = res_mut_dictionary->uniqueInsertRangeFrom(*keys, 0, keys->size()); ColumnUniquePtr res_dictionary = std::move(res_mut_dictionary); @@ -490,22 +490,22 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si res_indexes = cache_values->index_mapping; } - res.column = ColumnWithDictionary::create(res_dictionary, res_indexes->index(*indexes, 0), use_cache); + res.column = ColumnLowCardinality::create(res_dictionary, res_indexes->index(*indexes, 0), use_cache); } else { - res.column = ColumnWithDictionary::create(res_dictionary, res_indexes); + res.column = ColumnLowCardinality::create(res_dictionary, res_indexes); } } else { - convertColumnsWithDictionaryToFull(block_without_dicts, args); - executeWithoutColumnsWithDictionary(block_without_dicts, args, result, input_rows_count); - res.column = block_without_dicts.safeGetByPosition(result).column; + convertLowCardinalityColumnsToFull(block_without_low_cardinality, args); + executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, input_rows_count); + res.column = block_without_low_cardinality.safeGetByPosition(result).column; } } else - executeWithoutColumnsWithDictionary(block, args, result, input_rows_count); + executeWithoutLowCardinalityColumns(block, args, result, input_rows_count); } void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const @@ -521,7 +521,7 @@ void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) con ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } -DataTypePtr FunctionBuilderImpl::getReturnTypeWithoutDictionary(const ColumnsWithTypeAndName & arguments) const +DataTypePtr FunctionBuilderImpl::getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const { checkNumberOfArguments(arguments.size()); @@ -613,23 +613,23 @@ llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const DataTypes DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & arguments) const { - if (useDefaultImplementationForColumnsWithDictionary()) + if (useDefaultImplementationForLowCardinalityColumns()) { bool has_low_cardinality = false; size_t num_full_low_cardinality_columns = 0; size_t num_full_ordinary_columns = 0; - ColumnsWithTypeAndName args_without_dictionary(arguments); + ColumnsWithTypeAndName args_without_low_cardinality(arguments); - for (ColumnWithTypeAndName & arg : args_without_dictionary) + for (ColumnWithTypeAndName & arg : args_without_low_cardinality) { bool is_const = arg.column && arg.column->isColumnConst(); if (is_const) arg.column = static_cast(*arg.column).removeLowCardinality(); - if (auto * type_with_dictionary = typeid_cast(arg.type.get())) + if (auto * low_cardinality_type = typeid_cast(arg.type.get())) { - arg.type = type_with_dictionary->getDictionaryType(); + arg.type = low_cardinality_type->getDictionaryType(); has_low_cardinality = true; if (!is_const) @@ -639,7 +639,7 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar ++num_full_ordinary_columns; } - for (auto & arg : args_without_dictionary) + for (auto & arg : args_without_low_cardinality) { arg.column = recursiveRemoveLowCardinality(arg.column); arg.type = recursiveRemoveLowCardinality(arg.type); @@ -647,11 +647,11 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar if (canBeExecutedOnLowCardinalityDictionary() && has_low_cardinality && num_full_low_cardinality_columns <= 1 && num_full_ordinary_columns == 0) - return std::make_shared(getReturnTypeWithoutDictionary(args_without_dictionary)); + return std::make_shared(getReturnTypeWithoutLowCardinality(args_without_low_cardinality)); else - return getReturnTypeWithoutDictionary(args_without_dictionary); + return getReturnTypeWithoutLowCardinality(args_without_low_cardinality); } - return getReturnTypeWithoutDictionary(arguments); + return getReturnTypeWithoutLowCardinality(arguments); } } diff --git a/dbms/src/Functions/IFunction.h b/dbms/src/Functions/IFunction.h index db6cf0ead9d..4028a61ef7b 100644 --- a/dbms/src/Functions/IFunction.h +++ b/dbms/src/Functions/IFunction.h @@ -77,11 +77,11 @@ protected: */ virtual bool useDefaultImplementationForConstants() const { return false; } - /** If function arguments has single column with dictionary and all other arguments are constants, call function on nested column. - * Otherwise, convert all columns with dictionary to ordinary columns. - * Returns ColumnWithDictionary if at least one argument is ColumnWithDictionary. + /** If function arguments has single low cardinality column and all other arguments are constants, call function on nested column. + * Otherwise, convert all low cardinality columns to ordinary columns. + * Returns ColumnLowCardinality if at least one argument is ColumnLowCardinality. */ - virtual bool useDefaultImplementationForColumnsWithDictionary() const { return true; } + virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; } /** Some arguments could remain constant during this implementation. */ @@ -97,7 +97,7 @@ private: size_t input_rows_count); bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count); - void executeWithoutColumnsWithDictionary(Block & block, const ColumnNumbers & arguments, size_t result, + void executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count); /// Cache is created by function createLowCardinalityResultCache() @@ -292,12 +292,12 @@ protected: virtual bool useDefaultImplementationForNulls() const { return true; } /** If useDefaultImplementationForNulls() is true, than change arguments for getReturnType() and buildImpl(). - * If function arguments has types with dictionary, convert them to ordinary types. - * getReturnType returns ColumnWithDictionary if at least one argument type is ColumnWithDictionary. + * If function arguments has low cardinality types, convert them to ordinary types. + * getReturnType returns ColumnLowCardinality if at least one argument type is ColumnLowCardinality. */ - virtual bool useDefaultImplementationForColumnsWithDictionary() const { return true; } + virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; } - /// If it isn't, will convert all ColumnWithDictionary arguments to full columns. + /// If it isn't, will convert all ColumnLowCardinality arguments to full columns. virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; } virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const = 0; @@ -309,7 +309,7 @@ protected: private: - DataTypePtr getReturnTypeWithoutDictionary(const ColumnsWithTypeAndName & arguments) const; + DataTypePtr getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const; }; /// Previous function interface. @@ -324,7 +324,7 @@ public: /// Override this functions to change default implementation behavior. See details in IMyFunction. bool useDefaultImplementationForNulls() const override { return true; } bool useDefaultImplementationForConstants() const override { return false; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } bool canBeExecutedOnDefaultArguments() const override { return true; } bool canBeExecutedOnLowCardinalityDictionary() const override { return isDeterministicInScopeOfQuery(); } @@ -406,7 +406,7 @@ protected: } bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); } bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); } - bool useDefaultImplementationForColumnsWithDictionary() const final { return function->useDefaultImplementationForColumnsWithDictionary(); } + bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); } bool canBeExecutedOnDefaultArguments() const override { return function->canBeExecutedOnDefaultArguments(); } @@ -477,7 +477,7 @@ protected: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); } bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); } - bool useDefaultImplementationForColumnsWithDictionary() const override { return function->useDefaultImplementationForColumnsWithDictionary(); } + bool useDefaultImplementationForLowCardinalityColumns() const override { return function->useDefaultImplementationForLowCardinalityColumns(); } bool canBeExecutedOnLowCardinalityDictionary() const override { return function->canBeExecutedOnLowCardinalityDictionary(); } FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override diff --git a/dbms/src/Functions/lowCardinalityIndices.cpp b/dbms/src/Functions/lowCardinalityIndices.cpp index 1a0c05d738c..23cd1594463 100644 --- a/dbms/src/Functions/lowCardinalityIndices.cpp +++ b/dbms/src/Functions/lowCardinalityIndices.cpp @@ -1,9 +1,9 @@ #include #include #include -#include +#include #include -#include +#include #include @@ -27,13 +27,13 @@ public: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - auto * type = typeid_cast(arguments[0].get()); + auto * type = typeid_cast(arguments[0].get()); if (!type) - throw Exception("First first argument of function lowCardinalityIndexes must be ColumnWithDictionary, but got" + throw Exception("First first argument of function lowCardinalityIndexes must be ColumnLowCardinality, but got" + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); @@ -44,7 +44,7 @@ public: auto arg_num = arguments[0]; const auto & arg = block.getByPosition(arg_num); auto & res = block.getByPosition(result); - auto indexes_col = typeid_cast(arg.column.get())->getIndexesPtr(); + auto indexes_col = typeid_cast(arg.column.get())->getIndexesPtr(); auto new_indexes_col = ColumnUInt64::create(indexes_col->size()); auto & data = new_indexes_col->getData(); for (size_t i = 0; i < data.size(); ++i) diff --git a/dbms/src/Functions/lowCardinalityKeys.cpp b/dbms/src/Functions/lowCardinalityKeys.cpp index e4654a35a68..1b1ebc41410 100644 --- a/dbms/src/Functions/lowCardinalityKeys.cpp +++ b/dbms/src/Functions/lowCardinalityKeys.cpp @@ -1,7 +1,7 @@ #include #include -#include -#include +#include +#include #include @@ -26,13 +26,13 @@ public: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - auto * type = typeid_cast(arguments[0].get()); + auto * type = typeid_cast(arguments[0].get()); if (!type) - throw Exception("First first argument of function lowCardinalityKeys must be ColumnWithDictionary, but got" + throw Exception("First first argument of function lowCardinalityKeys must be ColumnLowCardinality, but got" + arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return type->getDictionaryType(); @@ -43,8 +43,8 @@ public: auto arg_num = arguments[0]; const auto & arg = block.getByPosition(arg_num); auto & res = block.getByPosition(result); - const auto * column_with_dictionary = typeid_cast(arg.column.get()); - res.column = column_with_dictionary->getDictionary().getNestedColumn()->cloneResized(arg.column->size()); + const auto * low_cardinality_column = typeid_cast(arg.column.get()); + res.column = low_cardinality_column->getDictionary().getNestedColumn()->cloneResized(arg.column->size()); } }; diff --git a/dbms/src/Functions/toLowCardinality.cpp b/dbms/src/Functions/toLowCardinality.cpp index 046eb717114..1e67de15162 100644 --- a/dbms/src/Functions/toLowCardinality.cpp +++ b/dbms/src/Functions/toLowCardinality.cpp @@ -1,9 +1,9 @@ #include #include #include -#include +#include #include -#include +#include #include @@ -22,14 +22,14 @@ public: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments[0]->withDictionary()) + if (arguments[0]->lowCardinality()) return arguments[0]; - return std::make_shared(arguments[0]); + return std::make_shared(arguments[0]); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override @@ -38,12 +38,12 @@ public: const auto & arg = block.getByPosition(arg_num); auto & res = block.getByPosition(result); - if (arg.type->withDictionary()) + if (arg.type->lowCardinality()) res.column = arg.column; else { auto column = res.type->createColumn(); - typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); + typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); res.column = std::move(column); } } diff --git a/dbms/src/Functions/toTypeName.cpp b/dbms/src/Functions/toTypeName.cpp index 560fdaf8dd4..a631c7f4044 100644 --- a/dbms/src/Functions/toTypeName.cpp +++ b/dbms/src/Functions/toTypeName.cpp @@ -24,7 +24,7 @@ public: } bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForColumnsWithDictionary() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } size_t getNumberOfArguments() const override { diff --git a/dbms/src/Interpreters/AggregationCommon.h b/dbms/src/Interpreters/AggregationCommon.h index e62e5b7a81c..c101d5b8e27 100644 --- a/dbms/src/Interpreters/AggregationCommon.h +++ b/dbms/src/Interpreters/AggregationCommon.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include template <> diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index df621ae9390..82654c30bb9 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -24,8 +24,8 @@ #include #if __has_include() #include -#include -#include +#include +#include #endif @@ -405,7 +405,7 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod() { DataTypePtr type = (params.src_header ? params.src_header : params.intermediate_header).safeGetByPosition(pos).type; - if (type->withDictionary()) + if (type->lowCardinality()) { has_low_cardinality = true; type = removeLowCardinality(type); @@ -748,7 +748,6 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re * To make them work anyway, we materialize them. */ Columns materialized_columns; - // ColumnRawPtrs key_counts; /// Remember the columns we will work with for (size_t i = 0; i < params.keys_size; ++i) @@ -761,14 +760,13 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re key_columns[i] = materialized_columns.back().get(); } - if (const auto * column_with_dictionary = typeid_cast(key_columns[i])) + if (const auto * low_cardinality_column = typeid_cast(key_columns[i])) { if (!result.isLowCardinality()) { - materialized_columns.push_back(column_with_dictionary->convertToFullColumn()); + materialized_columns.push_back(low_cardinality_column->convertToFullColumn()); key_columns[i] = materialized_columns.back().get(); } - //key_counts.push_back(materialized_columns.back().get()); } } @@ -787,9 +785,9 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re aggregate_columns[i][j] = materialized_columns.back().get(); } - if (auto * col_with_dict = typeid_cast(aggregate_columns[i][j])) + if (auto * col_low_cardinality = typeid_cast(aggregate_columns[i][j])) { - materialized_columns.push_back(col_with_dict->convertToFullColumn()); + materialized_columns.push_back(col_low_cardinality->convertToFullColumn()); aggregate_columns[i][j] = materialized_columns.back().get(); } } diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index deeebe5b41f..f32c0e9940f 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -28,7 +28,7 @@ #include #include #include -#include +#include namespace DB @@ -413,7 +413,7 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod void init(ColumnRawPtrs & key_columns, const AggregationStateCachePtr & cache_ptr) { - auto column = typeid_cast(key_columns[0]); + auto column = typeid_cast(key_columns[0]); if (!column) throw Exception("Invalid aggregation key type for AggregationMethodSingleLowCardinalityColumn method. " "Excepted LowCardinality, got " + key_columns[0]->getName(), ErrorCodes::LOGICAL_ERROR); @@ -583,7 +583,7 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t /*keys_size*/, const Sizes & /*key_sizes*/) { auto ref = Base::getValueRef(value); - static_cast(key_columns[0].get())->insertData(ref.data, ref.size); + static_cast(key_columns[0].get())->insertData(ref.data, ref.size); } }; @@ -732,7 +732,7 @@ struct AggregationMethodKeysFixed low_cardinality_keys.position_sizes.resize(key_columns.size()); for (size_t i = 0; i < key_columns.size(); ++i) { - if (auto * low_cardinality_col = typeid_cast(key_columns[i])) + if (auto * low_cardinality_col = typeid_cast(key_columns[i])) { low_cardinality_keys.nested_columns[i] = low_cardinality_col->getDictionary().getNestedColumn().get(); low_cardinality_keys.positions[i] = &low_cardinality_col->getIndexes(); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 1924608b676..466473b1f83 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -69,7 +69,7 @@ #include #include #include -#include +#include namespace DB @@ -1507,7 +1507,7 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block & set_element_types = left_tuple_type->getElements(); for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) element_type = low_cardinality_type->getDictionaryType(); ASTPtr elements_ast = nullptr; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index e03ebb3d50c..337ad39abef 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -373,7 +373,7 @@ void InterpreterCreateQuery::checkSupportedTypes(const ColumnsDescription & colu { for (const auto & column : list) { - if (!allow_low_cardinality && column.type && column.type->withDictionary()) + if (!allow_low_cardinality && column.type && column.type->lowCardinality()) { String message = "Cannot create table with column '" + column.name + "' which type is '" + column.type->getName() + "' because LowCardinality type is not allowed. " diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 45edccf723c..618df895c66 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include #include @@ -962,8 +962,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre { pipeline.transform([&](auto & stream) { - stream = //std::make_shared( - std::make_shared(stream, expression); //); + stream = std::make_shared(stream, expression); }); Names key_names; diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index b1f67820fad..6bc99f3355d 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -27,7 +27,7 @@ #include #include -#include +#include namespace DB @@ -131,10 +131,10 @@ void Set::setHeader(const Block & block) } /// Convert low cardinality column to full. - if (auto * low_cardinality_type = typeid_cast(data_types.back().get())) + if (auto * low_cardinality_type = typeid_cast(data_types.back().get())) { data_types.back() = low_cardinality_type->getDictionaryType(); - materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfWithDictionary()); + materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfLowCardinality()); key_columns.back() = materialized_columns.back().get(); } } @@ -184,9 +184,9 @@ bool Set::insertFromBlock(const Block & block) } /// Convert low cardinality column to full. - if (key_columns.back()->withDictionary()) + if (key_columns.back()->lowCardinality()) { - materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfWithDictionary()); + materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfLowCardinality()); key_columns.back() = materialized_columns.back().get(); } } diff --git a/dbms/src/Interpreters/convertFieldToType.cpp b/dbms/src/Interpreters/convertFieldToType.cpp index 517caa0edf6..58b0c164c35 100644 --- a/dbms/src/Interpreters/convertFieldToType.cpp +++ b/dbms/src/Interpreters/convertFieldToType.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include @@ -257,8 +257,8 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co if (from_type_hint && from_type_hint->equals(to_type)) return from_value; - if (auto * with_dict_type = typeid_cast(&to_type)) - return convertFieldToType(from_value, *with_dict_type->getDictionaryType(), from_type_hint); + if (auto * low_cardinality_type = typeid_cast(&to_type)) + return convertFieldToType(from_value, *low_cardinality_type->getDictionaryType(), from_type_hint); else if (auto * nullable_type = typeid_cast(&to_type)) return convertFieldToTypeImpl(from_value, *nullable_type->getNestedType(), from_type_hint); else diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index d9be484d5d6..fe40d47712d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -200,7 +200,7 @@ MergeTreeReader::Stream::Stream( getMark(right).offset_in_compressed_file - getMark(all_mark_ranges[i].begin).offset_in_compressed_file); } - /// Avoid empty buffer. May happen while reading dictionary for DataTypeWithDictionary. + /// Avoid empty buffer. May happen while reading dictionary for DataTypeLowCardinality. /// For example: part has single dictionary and all marks point to the same position. if (max_mark_range == 0) max_mark_range = max_read_buffer_size; From 6f96dfc7dab100194dfdbc50bc4dc03e40610d25 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Sep 2018 14:58:24 +0300 Subject: [PATCH 14/52] Auto version update to [18.14.1] [54409] --- dbms/cmake/version.cmake | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 6c75235b815..7f720afa804 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54409 CACHE STRING "") set(VERSION_MAJOR 18 CACHE STRING "") set(VERSION_MINOR 14 CACHE STRING "") -set(VERSION_PATCH 0 CACHE STRING "") -set(VERSION_GITHASH f09970ff704a32c41ae207e3db152541800dc580 CACHE STRING "") -set(VERSION_DESCRIBE v18.14.0-testing CACHE STRING "") -set(VERSION_STRING 18.14.0 CACHE STRING "") +set(VERSION_PATCH 1 CACHE STRING "") +set(VERSION_GITHASH 4452a1f49bf9ae93838da62694b7f1b1523470e3 CACHE STRING "") +set(VERSION_DESCRIBE v18.14.1-testing CACHE STRING "") +set(VERSION_STRING 18.14.1 CACHE STRING "") # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/debian/changelog b/debian/changelog index 20cf12781dc..b2654d91dbd 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (18.14.0) unstable; urgency=low +clickhouse (18.14.1) unstable; urgency=low * Modified source code - -- Mon, 24 Sep 2018 20:52:21 +0300 + -- Fri, 28 Sep 2018 14:58:24 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 426ccebb976..e2209f79b82 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.0 +ARG version=18.14.1 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index a8b47159aab..7bed030ee7c 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.0 +ARG version=18.14.1 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index bb80f4a39ec..febec335824 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.0 +ARG version=18.14.1 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ From f9fb73ce2e62828ef4c9d934794c7cb9836ec423 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 28 Sep 2018 16:44:39 +0300 Subject: [PATCH 15/52] fix distributed "create table as select" query --- dbms/src/Interpreters/DDLWorker.cpp | 21 ++++++++++++++++++++- dbms/src/Interpreters/DDLWorker.h | 6 ++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 69e951f18e4..b3baf0731ff 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -178,7 +178,7 @@ struct DDLTask Cluster::Address address_in_cluster; size_t host_shard_num; size_t host_replica_num; - +// /// Stage 3.3: execute query ExecutionStatus execution_status; bool was_executed = false; @@ -335,6 +335,7 @@ void DDLWorker::processTasks() { LOG_DEBUG(log, "Processing tasks"); + Strings queue_nodes = zookeeper->getChildren(queue_dir, nullptr, event_queue_updated); filterAndSortQueueNodes(queue_nodes); if (queue_nodes.empty()) @@ -536,11 +537,29 @@ bool DDLWorker::tryExecuteQuery(const String & query, const DDLTask & task, Exec return true; } +void DDLWorker::attachToThreadGroup() +{ + std::lock_guard lock(thread_group_mutex); + + if (thread_group) + { + /// Put all threads to one thread pool + CurrentThread::attachToIfDetached(thread_group); + } + else + { + CurrentThread::initializeQuery(); + thread_group = CurrentThread::getGroup(); + } +} + void DDLWorker::processTask(DDLTask & task) { LOG_DEBUG(log, "Processing task " << task.entry_name << " (" << task.entry.query << ")"); + attachToThreadGroup(); + String dummy; String active_node_path = task.entry_path + "/active/" + task.host_id_str; String finished_node_path = task.entry_path + "/finished/" + task.host_id_str; diff --git a/dbms/src/Interpreters/DDLWorker.h b/dbms/src/Interpreters/DDLWorker.h index 5db27c49f10..76fa960b3ec 100644 --- a/dbms/src/Interpreters/DDLWorker.h +++ b/dbms/src/Interpreters/DDLWorker.h @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -67,6 +68,8 @@ private: void run(); + void attachToThreadGroup(); + private: Context & context; Logger * log; @@ -98,6 +101,9 @@ private: /// How many tasks could be in the queue size_t max_tasks_in_queue = 1000; + ThreadGroupStatusPtr thread_group; + std::mutex thread_group_mutex; + friend class DDLQueryStatusInputSream; friend struct DDLTask; }; From 009c9733129bf66cea08256560258784bd167c8b Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 28 Sep 2018 16:47:34 +0300 Subject: [PATCH 16/52] remove redundant comment --- dbms/src/Interpreters/DDLWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index b3baf0731ff..971fc08856f 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -178,7 +178,7 @@ struct DDLTask Cluster::Address address_in_cluster; size_t host_shard_num; size_t host_replica_num; -// + /// Stage 3.3: execute query ExecutionStatus execution_status; bool was_executed = false; From ec11bef6337b23d9b41f9beb9cbefac5748b3b86 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 28 Sep 2018 17:53:20 +0300 Subject: [PATCH 17/52] Build fixes (#3181) --- cmake/find_cpuid.cmake | 19 ++++----- cmake/find_zlib.cmake | 4 +- cmake/test_cpu.cmake | 15 ++++--- contrib/CMakeLists.txt | 1 + dbms/CMakeLists.txt | 2 +- dbms/programs/clang/CMakeLists.txt | 2 +- .../odbc-bridge/ColumnInfoHandler.cpp | 6 +-- dbms/src/Client/TimeoutSetter.cpp | 41 +++++++++++++++++++ dbms/src/Client/TimeoutSetter.h | 35 ++++------------ dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- dbms/src/IO/CascadeWriteBuffer.h | 2 +- dbms/src/Interpreters/DDLWorker.cpp | 2 +- dbms/tests/integration/CMakeLists.txt | 10 ++++- dbms/tests/integration/helpers/cluster.py | 39 +++++++++++------- .../configs/conf.d/clusters.xml | 2 +- .../configs/config-copier.xml | 2 +- debian/.pbuilderrc | 2 +- debian/pbuilder-hooks/A00ccache | 6 +-- debian/pbuilder-hooks/C99kill-make | 2 +- debian/rules | 12 ++++-- libs/libcommon/cmake/find_jemalloc.cmake | 9 +++- 21 files changed, 138 insertions(+), 77 deletions(-) create mode 100644 dbms/src/Client/TimeoutSetter.cpp diff --git a/cmake/find_cpuid.cmake b/cmake/find_cpuid.cmake index d02336021bb..cda88433a1c 100644 --- a/cmake/find_cpuid.cmake +++ b/cmake/find_cpuid.cmake @@ -1,18 +1,11 @@ -# Freebsd: /usr/local/include/libcpuid/libcpuid_types.h:61:29: error: conflicting declaration 'typedef long long int int64_t' -# TODO: test new libcpuid - maybe already fixed - if (NOT ARCH_ARM) - if (OS_FREEBSD) - set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY 1) - else () - set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY ${NOT_UNBUNDLED}) - endif () - option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${DEFAULT_USE_INTERNAL_CPUID_LIBRARY}) + option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${NOT_UNBUNDLED}) endif () #if (USE_INTERNAL_CPUID_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcpuid/include/cpuid/libcpuid.h") # message (WARNING "submodule contrib/libcpuid is missing. to fix try run: \n git submodule update --init --recursive") # set (USE_INTERNAL_CPUID_LIBRARY 0) +# set (MISSING_INTERNAL_CPUID_LIBRARY 1) #endif () if (NOT USE_INTERNAL_CPUID_LIBRARY) @@ -21,7 +14,13 @@ if (NOT USE_INTERNAL_CPUID_LIBRARY) endif () if (CPUID_LIBRARY AND CPUID_INCLUDE_DIR) -else () + if (OS_FREEBSD) + # need in /usr/local/include/libcpuid/libcpuid_types.h + # Freebsd: /usr/local/include/libcpuid/libcpuid_types.h:61:29: error: conflicting declaration 'typedef long long int int64_t' + add_definitions(-DHAVE_STDINT_H) + # TODO: make virtual target cpuid:cpuid with COMPILE_DEFINITIONS property + endif () +elseif (NOT MISSING_INTERNAL_CPUID_LIBRARY) set (CPUID_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcpuid/include) set (USE_INTERNAL_CPUID_LIBRARY 1) set (CPUID_LIBRARY cpuid) diff --git a/cmake/find_zlib.cmake b/cmake/find_zlib.cmake index 6bf39561487..f6b2d268291 100644 --- a/cmake/find_zlib.cmake +++ b/cmake/find_zlib.cmake @@ -1,4 +1,6 @@ -option (USE_INTERNAL_ZLIB_LIBRARY "Set to FALSE to use system zlib library instead of bundled" ${NOT_UNBUNDLED}) +if (NOT OS_FREEBSD) + option (USE_INTERNAL_ZLIB_LIBRARY "Set to FALSE to use system zlib library instead of bundled" ${NOT_UNBUNDLED}) +endif () if (NOT USE_INTERNAL_ZLIB_LIBRARY) find_package (ZLIB) diff --git a/cmake/test_cpu.cmake b/cmake/test_cpu.cmake index 8a301983963..6894c58703d 100644 --- a/cmake/test_cpu.cmake +++ b/cmake/test_cpu.cmake @@ -14,11 +14,12 @@ cmake_push_check_state () #define __SSE4_1__ 1 set (TEST_FLAG "-msse4.1") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}") +set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" #include int main() { - _mm_insert_epi8(__m128i(), 0, 0); + auto a = _mm_insert_epi8(__m128i(), 0, 0); + (void)a; return 0; } " HAVE_SSE41) @@ -31,11 +32,12 @@ endif () #define __SSE4_2__ 1 set (TEST_FLAG "-msse4.2") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}") +set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" #include int main() { - _mm_crc32_u64(0, 0); + auto a = _mm_crc32_u64(0, 0); + (void)a; return 0; } " HAVE_SSE42) @@ -49,10 +51,11 @@ endif () set (TEST_FLAG "-mpopcnt") -set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}") +set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" int main() { - __builtin_popcountll(0); + auto a = __builtin_popcountll(0); + (void)a; return 0; } " HAVE_POPCNT) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index bea55e30b35..1c653f86808 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -142,6 +142,7 @@ if (USE_INTERNAL_POCO_LIBRARY) set (save_CMAKE_C_FLAGS ${CMAKE_C_FLAGS}) set (_save ${ENABLE_TESTS}) set (ENABLE_TESTS 0) + set (POCO_ENABLE_TESTS 0) set (CMAKE_DISABLE_FIND_PACKAGE_ZLIB 1) if (MSVC) set (ENABLE_DATA_ODBC 0 CACHE INTERNAL "") # TODO (build fail) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 6c8d65c0a22..69fe7f46a4b 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -155,7 +155,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELW PROPERTIES COMPILE_FLAGS -g0) endif () -if (NOT ARCH_ARM) +if (NOT ARCH_ARM AND CPUID_LIBRARY) set (LINK_LIBRARIES_ONLY_ON_X86_64 ${CPUID_LIBRARY}) endif() diff --git a/dbms/programs/clang/CMakeLists.txt b/dbms/programs/clang/CMakeLists.txt index 802ec00168c..ca06f27546d 100644 --- a/dbms/programs/clang/CMakeLists.txt +++ b/dbms/programs/clang/CMakeLists.txt @@ -26,7 +26,7 @@ elseif (EXISTS ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}) set (COPY_HEADERS_COMPILER "${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}") endif () -if (COPY_HEADERS_COMPILER AND NOT APPLE) +if (COPY_HEADERS_COMPILER AND OS_LINUX) add_custom_target (copy-headers ALL env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh) if (USE_INTERNAL_LLVM_LIBRARY) diff --git a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp index 014d09d9067..820697b3935 100644 --- a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -2,9 +2,9 @@ #if USE_POCO_SQLODBC || USE_POCO_DATAODBC #if USE_POCO_SQLODBC -#include -#include -#include +#include // Y_IGNORE +#include // Y_IGNORE +#include // Y_IGNORE #define POCO_SQL_ODBC_CLASS Poco::SQL::ODBC #endif #if USE_POCO_DATAODBC diff --git a/dbms/src/Client/TimeoutSetter.cpp b/dbms/src/Client/TimeoutSetter.cpp new file mode 100644 index 00000000000..f61c505390d --- /dev/null +++ b/dbms/src/Client/TimeoutSetter.cpp @@ -0,0 +1,41 @@ +#include "TimeoutSetter.h" + +#include + +namespace DB +{ +TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_, + const Poco::Timespan & send_timeout_, + const Poco::Timespan & receive_timeout_, + bool limit_max_timeout) + : socket(socket_), send_timeout(send_timeout_), receive_timeout(receive_timeout_) +{ + old_send_timeout = socket.getSendTimeout(); + old_receive_timeout = socket.getReceiveTimeout(); + + if (!limit_max_timeout || old_send_timeout > send_timeout) + socket.setSendTimeout(send_timeout); + + if (!limit_max_timeout || old_receive_timeout > receive_timeout) + socket.setReceiveTimeout(receive_timeout); +} + +TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout) + : TimeoutSetter(socket_, timeout_, timeout_, limit_max_timeout) +{ +} + +TimeoutSetter::~TimeoutSetter() +{ + try + { + socket.setSendTimeout(old_send_timeout); + socket.setReceiveTimeout(old_receive_timeout); + } + catch (std::exception & e) + { + // Sometimes catched on macos + LOG_ERROR(&Logger::get("Client"), std::string{"TimeoutSetter: Can't reset timeouts: "} + e.what()); + } +} +} diff --git a/dbms/src/Client/TimeoutSetter.h b/dbms/src/Client/TimeoutSetter.h index 30ce28e889c..ce6c8fe3ce8 100644 --- a/dbms/src/Client/TimeoutSetter.h +++ b/dbms/src/Client/TimeoutSetter.h @@ -1,47 +1,30 @@ #pragma once -#include #include +#include namespace DB { - -/// Temporarily overrides socket send/recieve timeouts and reset them back into destructor +/// Temporarily overrides socket send/receive timeouts and reset them back into destructor /// If "limit_max_timeout" is true, timeouts could be only decreased (maxed by previous value). struct TimeoutSetter { - TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & send_timeout_, const Poco::Timespan & recieve_timeout_, - bool limit_max_timeout = false) - : socket(socket_), send_timeout(send_timeout_), recieve_timeout(recieve_timeout_) - { - old_send_timeout = socket.getSendTimeout(); - old_receive_timeout = socket.getReceiveTimeout(); + TimeoutSetter(Poco::Net::StreamSocket & socket_, + const Poco::Timespan & send_timeout_, + const Poco::Timespan & receive_timeout_, + bool limit_max_timeout = false); - if (!limit_max_timeout || old_send_timeout > send_timeout) - socket.setSendTimeout(send_timeout); + TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout = false); - if (!limit_max_timeout || old_receive_timeout > recieve_timeout) - socket.setReceiveTimeout(recieve_timeout); - } - - TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout = false) - : TimeoutSetter(socket_, timeout_, timeout_, limit_max_timeout) {} - - ~TimeoutSetter() - { - socket.setSendTimeout(old_send_timeout); - socket.setReceiveTimeout(old_receive_timeout); - } + ~TimeoutSetter(); Poco::Net::StreamSocket & socket; Poco::Timespan send_timeout; - Poco::Timespan recieve_timeout; + Poco::Timespan receive_timeout; Poco::Timespan old_send_timeout; Poco::Timespan old_receive_timeout; }; - - } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index b47452cedb6..9a4d059d461 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1027,7 +1027,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) read(err); if (read_xid != auth_xid) - throw Exception("Unexpected event recieved in reply to auth request: " + toString(read_xid), + throw Exception("Unexpected event received in reply to auth request: " + toString(read_xid), ZMARSHALLINGERROR); int32_t actual_length = in->count() - count_before_event; diff --git a/dbms/src/IO/CascadeWriteBuffer.h b/dbms/src/IO/CascadeWriteBuffer.h index 58e21169b4c..dc8678ffdd1 100644 --- a/dbms/src/IO/CascadeWriteBuffer.h +++ b/dbms/src/IO/CascadeWriteBuffer.h @@ -17,7 +17,7 @@ namespace ErrorCodes * (lazy_sources contains not pointers themself, but their delayed constructors) * * Firtly, CascadeWriteBuffer redirects data to first buffer of the sequence - * If current WriteBuffer cannot recieve data anymore, it throws special exception CURRENT_WRITE_BUFFER_IS_EXHAUSTED in nextImpl() body, + * If current WriteBuffer cannot receive data anymore, it throws special exception CURRENT_WRITE_BUFFER_IS_EXHAUSTED in nextImpl() body, * CascadeWriteBuffer prepare next buffer and continuously redirects data to it. * If there are no buffers anymore CascadeWriteBuffer throws an exception. * diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 69e951f18e4..ee105ea5fe6 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -424,7 +424,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task) } if (!task.query || !(task.query_on_cluster = dynamic_cast(task.query.get()))) - throw Exception("Recieved unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY); + throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY); task.cluster_name = task.query_on_cluster->cluster; task.cluster = context.tryGetCluster(task.cluster_name); diff --git a/dbms/tests/integration/CMakeLists.txt b/dbms/tests/integration/CMakeLists.txt index b0437d45a4a..e2dae4634ca 100644 --- a/dbms/tests/integration/CMakeLists.txt +++ b/dbms/tests/integration/CMakeLists.txt @@ -1,2 +1,10 @@ +if (CLICKHOUSE_SPLIT_BINARY) + set (TEST_USE_BINARIES CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse-server CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse-client) +else() + set (TEST_USE_BINARIES CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse) +endif() -add_test(NAME integration WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/server/clickhouse-server CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/server/clickhouse-client "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=\${CLICKHOUSE_TESTS_BASE_CONFIG_DIR:=${ClickHouse_SOURCE_DIR}/dbms/programs/server/}" ${PYTEST_STARTER} pytest ${PYTEST_OPT}) +# will mount only one binary to docker container - build with .so cant work +if (MAKE_STATIC_LIBRARIES) + add_test (NAME integration WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/dbms/programs/server/" ${PYTEST_STARTER} pytest ${PYTEST_OPT}) +endif() diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 1333a640938..6c4fdcbc7f7 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -30,6 +30,16 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): f.write("=".join([var, value]) + "\n") return full_path +def subprocess_check_call(args): + # Uncomment for debugging + # print('run:', ' ' . join(args)) + subprocess.check_call(args) + +def subprocess_call(args): + # Uncomment for debugging + # print('run:', ' ' . join(args)) + subprocess.call(args) + class ClickHouseCluster: """ClickHouse cluster with several instances and (possibly) ZooKeeper. @@ -45,8 +55,8 @@ class ClickHouseCluster: self.name = name if name is not None else '' self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', '/etc/clickhouse-server/') - self.server_bin_path = server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse') - self.client_bin_path = client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client') + self.server_bin_path = p.realpath(server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')) + self.client_bin_path = p.realpath(client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client')) self.zookeeper_config_path = p.join(self.base_dir, zookeeper_config_path) if zookeeper_config_path else p.join(HELPERS_DIR, 'zookeeper_config.xml') self.project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name @@ -179,8 +189,8 @@ class ClickHouseCluster: # Just in case kill unstopped containers from previous launch try: - if not subprocess.call(['docker-compose', 'kill']): - subprocess.call(['docker-compose', 'down', '--volumes']) + if not subprocess_call(['docker-compose', 'kill']): + subprocess_call(['docker-compose', 'down', '--volumes']) except: pass @@ -194,23 +204,20 @@ class ClickHouseCluster: self.docker_client = docker.from_env(version=self.docker_api_version) if self.with_zookeeper and self.base_zookeeper_cmd: - subprocess.check_call(self.base_zookeeper_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) + subprocess_check_call(self.base_zookeeper_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) for command in self.pre_zookeeper_commands: self.run_kazoo_commands_with_retries(command, repeats=5) self.wait_zookeeper_to_start(120) if self.with_mysql and self.base_mysql_cmd: - subprocess.check_call(self.base_mysql_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) + subprocess_check_call(self.base_mysql_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) self.wait_mysql_to_start(120) if self.with_kafka and self.base_kafka_cmd: - subprocess.check_call(self.base_kafka_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) + subprocess_check_call(self.base_kafka_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) self.kafka_docker_id = self.get_instance_docker_id('kafka1') - # Uncomment for debugging - #print ' '.join(self.base_cmd + ['up', '--no-recreate']) - - subprocess.check_call(self.base_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) + subprocess_check_call(self.base_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) start_deadline = time.time() + 20.0 # seconds for instance in self.instances.itervalues(): @@ -226,8 +233,8 @@ class ClickHouseCluster: def shutdown(self, kill=True): if kill: - subprocess.check_call(self.base_cmd + ['kill']) - subprocess.check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) + subprocess_check_call(self.base_cmd + ['kill']) + subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) self.is_up = False self.docker_client = None @@ -468,8 +475,12 @@ class ClickHouseInstance: shutil.copy(p.join(self.base_configs_dir, 'config.xml'), configs_dir) shutil.copy(p.join(self.base_configs_dir, 'users.xml'), configs_dir) + # used by all utils with any config + conf_d_dir = p.abspath(p.join(configs_dir, 'conf.d')) + # used by server with main config.xml config_d_dir = p.abspath(p.join(configs_dir, 'config.d')) users_d_dir = p.abspath(p.join(configs_dir, 'users.d')) + os.mkdir(conf_d_dir) os.mkdir(config_d_dir) os.mkdir(users_d_dir) @@ -483,7 +494,7 @@ class ClickHouseInstance: # Put ZooKeeper config if self.with_zookeeper: - shutil.copy(self.zookeeper_config_path, config_d_dir) + shutil.copy(self.zookeeper_config_path, conf_d_dir) # Copy config dir if self.custom_config_dir: diff --git a/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml b/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml index e2304d5a936..f00cf1cf351 100644 --- a/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml +++ b/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml @@ -58,4 +58,4 @@ - \ No newline at end of file + diff --git a/dbms/tests/integration/test_cluster_copier/configs/config-copier.xml b/dbms/tests/integration/test_cluster_copier/configs/config-copier.xml index e2583e0ca00..12640034104 100644 --- a/dbms/tests/integration/test_cluster_copier/configs/config-copier.xml +++ b/dbms/tests/integration/test_cluster_copier/configs/config-copier.xml @@ -8,4 +8,4 @@ /var/log/clickhouse-server/copier/stderr.log /var/log/clickhouse-server/copier/stdout.log - \ No newline at end of file + diff --git a/debian/.pbuilderrc b/debian/.pbuilderrc index 8b787718d76..cfdebbfc979 100644 --- a/debian/.pbuilderrc +++ b/debian/.pbuilderrc @@ -179,7 +179,7 @@ fi # For killall in pbuilder-hooks: EXTRAPACKAGES+=" psmisc " -[[ $CCACHE_PREFIX == 'distcc' ]] && EXTRAPACKAGES+=" $CCACHE_PREFIX " +[[ $CCACHE_PREFIX == 'distcc' ]] && EXTRAPACKAGES+=" $CCACHE_PREFIX " && USENETWORK=yes && export DISTCC_DIR=/var/cache/pbuilder/distcc export DEB_BUILD_OPTIONS=parallel=`nproc` diff --git a/debian/pbuilder-hooks/A00ccache b/debian/pbuilder-hooks/A00ccache index b8bf8d579c0..ab122ecf82f 100755 --- a/debian/pbuilder-hooks/A00ccache +++ b/debian/pbuilder-hooks/A00ccache @@ -7,9 +7,9 @@ echo "CCACHEDIR=$CCACHEDIR CCACHE_DIR=$CCACHE_DIR SET_CCACHEDIR=$SET_CCACHEDIR" [ -z "$CCACHE_DIR" ] && export CCACHE_DIR=${CCACHEDIR:=${SET_CCACHEDIR=/var/cache/pbuilder/ccache}} if [ -n "$CCACHE_DIR" ]; then - mkdir -p $CCACHE_DIR || true - chown -R $BUILDUSERID:$BUILDUSERID $CCACHE_DIR || true - chmod -R a+rwx $CCACHE_DIR || true + mkdir -p $CCACHE_DIR $DISTCC_DIR || true + chown -R $BUILDUSERID:$BUILDUSERID $CCACHE_DIR $DISTCC_DIR || true + chmod -R a+rwx $CCACHE_DIR $DISTCC_DIR || true fi df -h diff --git a/debian/pbuilder-hooks/C99kill-make b/debian/pbuilder-hooks/C99kill-make index 34c33ee6138..863356d4724 100755 --- a/debian/pbuilder-hooks/C99kill-make +++ b/debian/pbuilder-hooks/C99kill-make @@ -2,4 +2,4 @@ # Try stop parallel build after timeout -killall make gcc gcc-7 g++-7 gcc-8 g++-8 clang-5.0 clang++-5.0 clang-6.0 clang++-6.0 clang-7 clang++-7 || true +killall make gcc gcc-7 g++-7 gcc-8 g++-8 clang clang-5.0 clang++-5.0 clang-6.0 clang++-6.0 clang-7 clang++-7 || true diff --git a/debian/rules b/debian/rules index 9cf58799610..06f4316f615 100755 --- a/debian/rules +++ b/debian/rules @@ -14,8 +14,10 @@ DESTDIR = $(CURDIR)/debian/tmp DEB_HOST_MULTIARCH ?= $(shell dpkg-architecture -qDEB_HOST_MULTIARCH) -#TODO: why it not working? (maybe works in debhelper 10+) -ifndef THREADS_COUNT +ifeq ($(CCACHE_PREFIX),distcc) + THREADS_COUNT=$(shell distcc -j) +endif +ifeq ($(THREADS_COUNT),) THREADS_COUNT=$(shell nproc || grep -c ^processor /proc/cpuinfo || sysctl -n hw.ncpu || echo 4) endif DEB_BUILD_OPTIONS+=parallel=$(THREADS_COUNT) @@ -73,6 +75,10 @@ else THREADS_COUNT = 1 endif +ifneq ($(THREADS_COUNT),) + THREADS_COUNT:=-j$(THREADS_COUNT) +endif + %: dh $@ $(DH_FLAGS) --buildsystem=cmake --builddirectory=$(BUILDDIR) @@ -81,7 +87,7 @@ override_dh_auto_configure: override_dh_auto_build: # Fix for ninja. Do not add -O. - cd $(BUILDDIR) && $(MAKE) -j$(THREADS_COUNT) $(MAKE_TARGET) + cd $(BUILDDIR) && $(MAKE) $(THREADS_COUNT) $(MAKE_TARGET) #cd $(BUILDDIR) && cmake --build . -- -j$(THREADS_COUNT) # cmake return true on error override_dh_auto_test: diff --git a/libs/libcommon/cmake/find_jemalloc.cmake b/libs/libcommon/cmake/find_jemalloc.cmake index eccebdce64f..f974688c5be 100644 --- a/libs/libcommon/cmake/find_jemalloc.cmake +++ b/libs/libcommon/cmake/find_jemalloc.cmake @@ -12,11 +12,18 @@ elseif () endif() if (ENABLE_JEMALLOC) + + if (USE_INTERNAL_JEMALLOC_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/jemalloc/src/jemalloc.c") + message (WARNING "submodule contrib/jemalloc is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_JEMALLOC_LIBRARY 0) + set (MISSING_INTERNAL_JEMALLOC_LIBRARY 1) + endif () + if (NOT USE_INTERNAL_JEMALLOC_LIBRARY) find_package (JeMalloc) endif () - if (NOT JEMALLOC_LIBRARIES) + if (NOT JEMALLOC_LIBRARIES AND NOT MISSING_INTERNAL_JEMALLOC_LIBRARY) set (JEMALLOC_LIBRARIES "jemalloc") set (USE_INTERNAL_JEMALLOC_LIBRARY 1) endif () From 292a885533b8e3b41ce8993867069d14cbd5a664 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 28 Sep 2018 18:08:37 +0300 Subject: [PATCH 18/52] fix typo in quantiles func registration --- dbms/src/AggregateFunctions/AggregateFunctionQuantile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.cpp b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.cpp index 1b4301767fc..d8c96a42fcd 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.cpp @@ -20,7 +20,7 @@ namespace { template using FuncQuantile = AggregateFunctionQuantile, NameQuantile, false, Float64, false>; -template using FuncQuantiles = AggregateFunctionQuantile, NameQuantile, false, Float64, true>; +template using FuncQuantiles = AggregateFunctionQuantile, NameQuantiles, false, Float64, true>; template using FuncQuantileDeterministic = AggregateFunctionQuantile, NameQuantileDeterministic, true, Float64, false>; template using FuncQuantilesDeterministic = AggregateFunctionQuantile, NameQuantilesDeterministic, true, Float64, true>; From ed3c7440554efd29efd2d5c7d61315c196b6e067 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 28 Sep 2018 18:30:03 +0300 Subject: [PATCH 19/52] fixes --- dbms/src/Interpreters/DDLWorker.cpp | 8 +++----- dbms/src/Interpreters/DDLWorker.h | 1 - 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 971fc08856f..5b3c261ca76 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -178,7 +178,7 @@ struct DDLTask Cluster::Address address_in_cluster; size_t host_shard_num; size_t host_replica_num; - + /// Stage 3.3: execute query ExecutionStatus execution_status; bool was_executed = false; @@ -539,8 +539,6 @@ bool DDLWorker::tryExecuteQuery(const String & query, const DDLTask & task, Exec void DDLWorker::attachToThreadGroup() { - std::lock_guard lock(thread_group_mutex); - if (thread_group) { /// Put all threads to one thread pool @@ -558,8 +556,6 @@ void DDLWorker::processTask(DDLTask & task) { LOG_DEBUG(log, "Processing task " << task.entry_name << " (" << task.entry.query << ")"); - attachToThreadGroup(); - String dummy; String active_node_path = task.entry_path + "/active/" + task.host_id_str; String finished_node_path = task.entry_path + "/finished/" + task.host_id_str; @@ -900,6 +896,8 @@ void DDLWorker::run() { try { + attachToThreadGroup(); + processTasks(); LOG_DEBUG(log, "Waiting a watch"); diff --git a/dbms/src/Interpreters/DDLWorker.h b/dbms/src/Interpreters/DDLWorker.h index 76fa960b3ec..2fba83b7356 100644 --- a/dbms/src/Interpreters/DDLWorker.h +++ b/dbms/src/Interpreters/DDLWorker.h @@ -102,7 +102,6 @@ private: size_t max_tasks_in_queue = 1000; ThreadGroupStatusPtr thread_group; - std::mutex thread_group_mutex; friend class DDLQueryStatusInputSream; friend struct DDLTask; From 49438ed1a143b0f91678d9d3350c437b2724fa9d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 28 Sep 2018 18:35:30 +0300 Subject: [PATCH 20/52] Update DDLWorker.cpp --- dbms/src/Interpreters/DDLWorker.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 5b3c261ca76..a02b51ebaa7 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -178,7 +178,7 @@ struct DDLTask Cluster::Address address_in_cluster; size_t host_shard_num; size_t host_replica_num; - + /// Stage 3.3: execute query ExecutionStatus execution_status; bool was_executed = false; @@ -335,7 +335,6 @@ void DDLWorker::processTasks() { LOG_DEBUG(log, "Processing tasks"); - Strings queue_nodes = zookeeper->getChildren(queue_dir, nullptr, event_queue_updated); filterAndSortQueueNodes(queue_nodes); if (queue_nodes.empty()) From d0b2e58d61a0f93726609580c0d768e63d26bd6a Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 28 Sep 2018 19:05:58 +0300 Subject: [PATCH 21/52] add test --- dbms/tests/integration/test_distributed_ddl/test.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbms/tests/integration/test_distributed_ddl/test.py b/dbms/tests/integration/test_distributed_ddl/test.py index cf5527eb0b6..00233335396 100755 --- a/dbms/tests/integration/test_distributed_ddl/test.py +++ b/dbms/tests/integration/test_distributed_ddl/test.py @@ -355,6 +355,12 @@ def test_optimize_query(started_cluster): ddl_check_query(instance, "CREATE TABLE test_optimize ON CLUSTER cluster (p Date, i Int32) ENGINE = MergeTree(p, p, 8192)") ddl_check_query(instance, "OPTIMIZE TABLE test_optimize ON CLUSTER cluster FORMAT TSV") +def test_create_as_select(started_cluster): + instance = cluster.instances['ch2'] + ddl_check_query(instance, "CREATE TABLE test_as_select ON CLUSTER cluster ENGINE = Memory AS (SELECT number FROM system.numbers_mt LIMIT 5)") + assert TSV(instance.query("SELECT number FROM test_as_select ORDER BY number")) == TSV("0\n1\n2\n3\n4\n") + + if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: for name, instance in cluster.instances.items(): From c2bec8b2c8c2df455b0d618b50424133f987ed08 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 28 Sep 2018 19:11:56 +0300 Subject: [PATCH 22/52] update test --- dbms/tests/integration/test_distributed_ddl/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/tests/integration/test_distributed_ddl/test.py b/dbms/tests/integration/test_distributed_ddl/test.py index 00233335396..375d07e0c05 100755 --- a/dbms/tests/integration/test_distributed_ddl/test.py +++ b/dbms/tests/integration/test_distributed_ddl/test.py @@ -354,11 +354,13 @@ def test_optimize_query(started_cluster): ddl_check_query(instance, "DROP TABLE IF EXISTS test_optimize ON CLUSTER cluster FORMAT TSV") ddl_check_query(instance, "CREATE TABLE test_optimize ON CLUSTER cluster (p Date, i Int32) ENGINE = MergeTree(p, p, 8192)") ddl_check_query(instance, "OPTIMIZE TABLE test_optimize ON CLUSTER cluster FORMAT TSV") + def test_create_as_select(started_cluster): instance = cluster.instances['ch2'] ddl_check_query(instance, "CREATE TABLE test_as_select ON CLUSTER cluster ENGINE = Memory AS (SELECT number FROM system.numbers_mt LIMIT 5)") assert TSV(instance.query("SELECT number FROM test_as_select ORDER BY number")) == TSV("0\n1\n2\n3\n4\n") + ddl_check_query(instance, "DROP TABLE IF EXISTS test_as_select ON CLUSTER cluster") if __name__ == '__main__': From be3b845baf4398b52bb875793eebb5714b86f992 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 28 Sep 2018 19:33:12 +0300 Subject: [PATCH 23/52] add test for remote quantiles --- dbms/tests/queries/0_stateless/00273_quantiles.reference | 6 ++++++ dbms/tests/queries/0_stateless/00273_quantiles.sql | 7 +++++++ 2 files changed, 13 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00273_quantiles.reference b/dbms/tests/queries/0_stateless/00273_quantiles.reference index 73dc42a93fa..ffa9c3ff0a9 100644 --- a/dbms/tests/queries/0_stateless/00273_quantiles.reference +++ b/dbms/tests/queries/0_stateless/00273_quantiles.reference @@ -1,8 +1,10 @@ [500] [500] [500] +[500] [0,1,10,50,100,200,300,400,500,600,700,800,900,950,990,999,1000] [0,1,10,50,100,200,300,400,500,600,700,800,900,950,990,999,1000] +[0,0.50100005,9.51,49.55,99.6,199.7,299.8,399.9,500,600.1,700.2,800.3,900.4,950.45,990.49,999.499,1000] [0,1,10,50,100,200,300,400,500,600,700,800,900,950,990,999,1000] 1 333334 [699140.3,835642,967430.8] [699999,833333,966666] 2 266667 [426549.5,536255.5,638957.6] [426665,533332,639999] @@ -2003,3 +2005,7 @@ 333333 1 [2,2,2] [2,2,2] 500000 1 [1,1,1] [1,1,1] 1000000 1 [0,0,0] [0,0,0] +[4.5,8.100000000000001] +[5,9] +[4.5,8.5] +[4.5,8.100000000000001] diff --git a/dbms/tests/queries/0_stateless/00273_quantiles.sql b/dbms/tests/queries/0_stateless/00273_quantiles.sql index d1862a4c1ce..4c8e22a660e 100644 --- a/dbms/tests/queries/0_stateless/00273_quantiles.sql +++ b/dbms/tests/queries/0_stateless/00273_quantiles.sql @@ -1,9 +1,16 @@ SELECT quantiles(0.5)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001); SELECT quantilesExact(0.5)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001); +SELECT quantilesTDigest(0.5)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001); SELECT quantilesDeterministic(0.5)(x, x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001); SELECT quantiles(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001); SELECT quantilesExact(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001); +SELECT quantilesTDigest(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001); SELECT quantilesDeterministic(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x, x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001); SELECT round(1000000 / (number + 1)) AS k, count() AS c, quantilesDeterministic(0.1, 0.5, 0.9)(number, intHash64(number)) AS q1, quantilesExact(0.1, 0.5, 0.9)(number) AS q2 FROM (SELECT number FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k; + +SELECT quantiles(0.5, 0.9)(number) FROM remote('127.0.0.{1,2}', numbers(10)); +SELECT quantilesExact(0.5, 0.9)(number) FROM remote('127.0.0.{1,2}', numbers(10)); +SELECT quantilesTDigest(0.5, 0.9)(number) FROM remote('127.0.0.{1,2}', numbers(10)); +SELECT quantilesDeterministic(0.5, 0.9)(number, number) FROM remote('127.0.0.{1,2}', numbers(10)); From a4fac0907325b3a27eafb6446e675856eeeb3463 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 28 Sep 2018 19:43:41 +0300 Subject: [PATCH 24/52] update test --- dbms/tests/integration/test_distributed_ddl/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/integration/test_distributed_ddl/test.py b/dbms/tests/integration/test_distributed_ddl/test.py index 375d07e0c05..714f179d9fa 100755 --- a/dbms/tests/integration/test_distributed_ddl/test.py +++ b/dbms/tests/integration/test_distributed_ddl/test.py @@ -354,12 +354,12 @@ def test_optimize_query(started_cluster): ddl_check_query(instance, "DROP TABLE IF EXISTS test_optimize ON CLUSTER cluster FORMAT TSV") ddl_check_query(instance, "CREATE TABLE test_optimize ON CLUSTER cluster (p Date, i Int32) ENGINE = MergeTree(p, p, 8192)") ddl_check_query(instance, "OPTIMIZE TABLE test_optimize ON CLUSTER cluster FORMAT TSV") - + def test_create_as_select(started_cluster): instance = cluster.instances['ch2'] - ddl_check_query(instance, "CREATE TABLE test_as_select ON CLUSTER cluster ENGINE = Memory AS (SELECT number FROM system.numbers_mt LIMIT 5)") - assert TSV(instance.query("SELECT number FROM test_as_select ORDER BY number")) == TSV("0\n1\n2\n3\n4\n") + ddl_check_query(instance, "CREATE TABLE test_as_select ON CLUSTER cluster ENGINE = Memory AS (SELECT 1 AS x UNION ALL SELECT 2 AS x)") + assert TSV(instance.query("SELECT x FROM test_as_select ORDER BY x")) == TSV("1\n2\n") ddl_check_query(instance, "DROP TABLE IF EXISTS test_as_select ON CLUSTER cluster") From 99b4536b2ace0af69ce873d8fd0efb6541b0d09b Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 28 Sep 2018 22:07:29 +0300 Subject: [PATCH 25/52] add test --- .../integration/test_distributed_ddl/test.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/dbms/tests/integration/test_distributed_ddl/test.py b/dbms/tests/integration/test_distributed_ddl/test.py index cf5527eb0b6..71c04b894d7 100755 --- a/dbms/tests/integration/test_distributed_ddl/test.py +++ b/dbms/tests/integration/test_distributed_ddl/test.py @@ -317,6 +317,23 @@ def test_macro(started_cluster): ddl_check_query(instance, "DROP TABLE IF EXISTS distr ON CLUSTER '{cluster}'") ddl_check_query(instance, "DROP TABLE IF EXISTS tab ON CLUSTER '{cluster}'") + # Temporarily disable random ZK packet drops, they might broke creation if ReplicatedMergeTree replicas + firewall_drops_rules = cluster.pm_random_drops.pop_rules() + + ddl_check_query(instance, "DROP TABLE IF EXISTS some_kek ON CLUSTER '{cluster}'") + ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test_db ON CLUSTER '{cluster}'") + + ddl_check_query(instance, """ +CREATE TABLE IF NOT EXISTS test_db.test_macro ON CLUSTER '{cluster}' (p Date, i Int32) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/{layer}-{shard}/{table}', '{replica}', p, p, 1) +""") + + # Check that table was created at correct path in zookeeper + assert cluster.get_kazoo_client('zoo1').exists('/clickhouse/tables/test_db/0-1/test_macro') is not None + + # Enable random ZK packet drops + cluster.pm_random_drops.push_rules(firewall_drops_rules) + def test_allowed_databases(started_cluster): instance = cluster.instances['ch2'] From e0d3aed0c4c1564f823d8b5826bcf885298be481 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 28 Sep 2018 22:10:16 +0300 Subject: [PATCH 26/52] remove redundant code --- dbms/src/Common/Macros.cpp | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/dbms/src/Common/Macros.cpp b/dbms/src/Common/Macros.cpp index 0748e9bc1d1..765c6427a59 100644 --- a/dbms/src/Common/Macros.cpp +++ b/dbms/src/Common/Macros.cpp @@ -2,9 +2,6 @@ #include #include -#include -#include - namespace DB { @@ -14,10 +11,7 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -Macros::Macros() -{ - LOG_DEBUG(&Logger::get("Macros"), "creating in empty contructor"); -} +Macros::Macros() {} Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key) { From b7b7314ecb1f04a41ab5502e5e92256fae1640ec Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 28 Sep 2018 22:13:16 +0300 Subject: [PATCH 27/52] remove redundant code --- dbms/src/Common/Macros.cpp | 1 - dbms/src/Common/Macros.h | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Common/Macros.cpp b/dbms/src/Common/Macros.cpp index 765c6427a59..a5a82dfa833 100644 --- a/dbms/src/Common/Macros.cpp +++ b/dbms/src/Common/Macros.cpp @@ -21,7 +21,6 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & { macros[key] = config.getString(root_key + "." + key); } - } String Macros::expand(const String & s, size_t level, const String & database_name, const String & table_name) const diff --git a/dbms/src/Common/Macros.h b/dbms/src/Common/Macros.h index 7fdc86b9ed0..8f1efdc094d 100644 --- a/dbms/src/Common/Macros.h +++ b/dbms/src/Common/Macros.h @@ -2,7 +2,7 @@ #include #include -#include + #include From d4de76346c212f78b8f244d2f072cc3d0e447e9f Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Sat, 29 Sep 2018 16:12:53 +0300 Subject: [PATCH 28/52] fix test --- dbms/tests/integration/test_distributed_ddl/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/tests/integration/test_distributed_ddl/test.py b/dbms/tests/integration/test_distributed_ddl/test.py index 71c04b894d7..81a5c558236 100755 --- a/dbms/tests/integration/test_distributed_ddl/test.py +++ b/dbms/tests/integration/test_distributed_ddl/test.py @@ -320,7 +320,8 @@ def test_macro(started_cluster): # Temporarily disable random ZK packet drops, they might broke creation if ReplicatedMergeTree replicas firewall_drops_rules = cluster.pm_random_drops.pop_rules() - ddl_check_query(instance, "DROP TABLE IF EXISTS some_kek ON CLUSTER '{cluster}'") + ddl_check_query(instance, "DROP DATABASE IF EXISTS test_db ON CLUSTER '{cluster}'") + ddl_check_query(instance, "DROP TABLE IF EXISTS test_db.test_macro ON CLUSTER '{cluster}'") ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test_db ON CLUSTER '{cluster}'") ddl_check_query(instance, """ From 85ca2006b0e4065e01f18311d8747a7c23641dae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Sep 2018 22:33:23 +0300 Subject: [PATCH 29/52] Fixed bug in client (double whitespace) [#CLICKHOUSE-4038] --- dbms/programs/client/clickhouse-client.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/client/clickhouse-client.xml b/dbms/programs/client/clickhouse-client.xml index 083f035d908..c073ab38aea 100644 --- a/dbms/programs/client/clickhouse-client.xml +++ b/dbms/programs/client/clickhouse-client.xml @@ -26,7 +26,7 @@ --> {display_name} :) - {display_name} \x01\e[1;32m\x02:)\x01\e[0m\x02 + {display_name} \x01\e[1;32m\x02:)\x01\e[0m\x02 {display_name} \x01\e[1;31m\x02:)\x01\e[0m\x02 From e4ec8ffc490eae7abf5fe77d2a858bf3396002b5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Sep 2018 22:50:12 +0300 Subject: [PATCH 30/52] Fixed test; tests that require server to listen something other than 127.0.0.1, must be tagged as "shard" in their names [#CLICKHOUSE-2] --- dbms/tests/queries/0_stateless/00273_quantiles.reference | 4 ---- dbms/tests/queries/0_stateless/00273_quantiles.sql | 5 ----- .../queries/0_stateless/00725_quantiles_shard.reference | 4 ++++ dbms/tests/queries/0_stateless/00725_quantiles_shard.sql | 4 ++++ 4 files changed, 8 insertions(+), 9 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00725_quantiles_shard.reference create mode 100644 dbms/tests/queries/0_stateless/00725_quantiles_shard.sql diff --git a/dbms/tests/queries/0_stateless/00273_quantiles.reference b/dbms/tests/queries/0_stateless/00273_quantiles.reference index ffa9c3ff0a9..d75d11191f4 100644 --- a/dbms/tests/queries/0_stateless/00273_quantiles.reference +++ b/dbms/tests/queries/0_stateless/00273_quantiles.reference @@ -2005,7 +2005,3 @@ 333333 1 [2,2,2] [2,2,2] 500000 1 [1,1,1] [1,1,1] 1000000 1 [0,0,0] [0,0,0] -[4.5,8.100000000000001] -[5,9] -[4.5,8.5] -[4.5,8.100000000000001] diff --git a/dbms/tests/queries/0_stateless/00273_quantiles.sql b/dbms/tests/queries/0_stateless/00273_quantiles.sql index 4c8e22a660e..a3609834e3c 100644 --- a/dbms/tests/queries/0_stateless/00273_quantiles.sql +++ b/dbms/tests/queries/0_stateless/00273_quantiles.sql @@ -9,8 +9,3 @@ SELECT quantilesTDigest(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, SELECT quantilesDeterministic(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x, x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001); SELECT round(1000000 / (number + 1)) AS k, count() AS c, quantilesDeterministic(0.1, 0.5, 0.9)(number, intHash64(number)) AS q1, quantilesExact(0.1, 0.5, 0.9)(number) AS q2 FROM (SELECT number FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k; - -SELECT quantiles(0.5, 0.9)(number) FROM remote('127.0.0.{1,2}', numbers(10)); -SELECT quantilesExact(0.5, 0.9)(number) FROM remote('127.0.0.{1,2}', numbers(10)); -SELECT quantilesTDigest(0.5, 0.9)(number) FROM remote('127.0.0.{1,2}', numbers(10)); -SELECT quantilesDeterministic(0.5, 0.9)(number, number) FROM remote('127.0.0.{1,2}', numbers(10)); diff --git a/dbms/tests/queries/0_stateless/00725_quantiles_shard.reference b/dbms/tests/queries/0_stateless/00725_quantiles_shard.reference new file mode 100644 index 00000000000..6974bee9735 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00725_quantiles_shard.reference @@ -0,0 +1,4 @@ +[4.5,8.100000000000001] +[5,9] +[4.5,8.5] +[4.5,8.100000000000001] diff --git a/dbms/tests/queries/0_stateless/00725_quantiles_shard.sql b/dbms/tests/queries/0_stateless/00725_quantiles_shard.sql new file mode 100644 index 00000000000..bb8fcfcd2d9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00725_quantiles_shard.sql @@ -0,0 +1,4 @@ +SELECT quantiles(0.5, 0.9)(number) FROM remote('127.0.0.{1,2}', numbers(10)); +SELECT quantilesExact(0.5, 0.9)(number) FROM remote('127.0.0.{1,2}', numbers(10)); +SELECT quantilesTDigest(0.5, 0.9)(number) FROM remote('127.0.0.{1,2}', numbers(10)); +SELECT quantilesDeterministic(0.5, 0.9)(number, number) FROM remote('127.0.0.{1,2}', numbers(10)); From 6778220ffdad18595c73c519cb2e8896b6dbbb4c Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 29 Sep 2018 19:29:23 +0800 Subject: [PATCH 31/52] ISSUES-3232 remake interpreter_subquery when rewrite subqueries --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 10 ++++++++-- .../0_stateless/00597_push_down_predicate.reference | 1 + .../queries/0_stateless/00597_push_down_predicate.sql | 1 + 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 618df895c66..8a9f7a64e85 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -202,8 +202,14 @@ InterpreterSelectQuery::InterpreterSelectQuery( context.addExternalTable(it.first, it.second); if (query_analyzer->isRewriteSubqueriesPredicate()) - interpreter_subquery = std::make_unique( - table_expression, getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1, only_analyze); + { + /// remake interpreter_subquery when PredicateOptimizer is rewrite subqueries and main table is subquery + if (typeid_cast(table_expression.get())) + interpreter_subquery = std::make_unique( + table_expression, getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1, + only_analyze); + } + } if (interpreter_subquery) diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index 484107c0947..f273a3710c1 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -3,6 +3,7 @@ 1 1 -------Need push down------- +0 0 1 1 1 diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql index 36378bda64a..95c67b0dcda 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -17,6 +17,7 @@ SELECT 1 AS id WHERE id = 1; SELECT arrayJoin([1,2,3]) AS id WHERE id = 1; SELECT '-------Need push down-------'; +SELECT * FROM system.one ANY LEFT JOIN (SELECT 0 AS dummy) USING dummy WHERE 1; SELECT toString(value) AS value FROM (SELECT 1 AS value) WHERE value = '1'; SELECT * FROM (SELECT 1 AS id UNION ALL SELECT 2) WHERE id = 1; SELECT * FROM (SELECT arrayJoin([1, 2, 3]) AS id) WHERE id = 1; From b9e6b0271987a68e2cad6f7ec52d26ba4d8bd49d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Oct 2018 04:26:52 +0300 Subject: [PATCH 32/52] Update StorageKafka.cpp --- dbms/src/Storages/Kafka/StorageKafka.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index a9c9d9d85db..c1198315b14 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -132,9 +132,8 @@ public: : ReadBuffer(nullptr, 0), consumer(consumer_), current(nullptr), current_pending(false), log(log_), read_messages(0), row_delimiter(row_delimiter_) { - if (row_delimiter != '\0') { + if (row_delimiter != '\0') LOG_TRACE(log, "Row delimiter is: " << row_delimiter); - } } ~ReadBufferFromKafkaConsumer() override { reset(); } @@ -169,9 +168,8 @@ public: ~KafkaBlockInputStream() override { - if (!hasClaimed()) { + if (!hasClaimed()) return; - } // An error was thrown during the stream or it did not finish successfully // The read offsets weren't comitted, so consumer must rejoin the group from the original starting point @@ -204,7 +202,8 @@ public: void readPrefixImpl() override { - if (!hasClaimed()) { + if (!hasClaimed()) + { // Create a formatted reader on Kafka messages LOG_TRACE(storage.log, "Creating formatted reader"); consumer = storage.tryClaimConsumer(context.getSettingsRef().queue_max_wait_ms.totalMilliseconds()); @@ -222,7 +221,8 @@ public: void readSuffixImpl() override { - if (hasClaimed()) { + if (hasClaimed()) + { reader->readSuffix(); // Store offsets read in this stream read_buf->commit(); From bb0f76dd2559c65431aa6862f668ba9c13470c80 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Oct 2018 04:29:17 +0300 Subject: [PATCH 33/52] Update StorageKafka.cpp --- dbms/src/Storages/Kafka/StorageKafka.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index c1198315b14..10c77de58a3 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -508,11 +508,10 @@ bool StorageKafka::streamToViews() // Join multiple streams if necessary BlockInputStreamPtr in; - if (streams.size() > 1) { + if (streams.size() > 1) in = std::make_shared>(streams, nullptr, num_consumers); - } else { + else in = streams[0]; - } // Execute the query InterpreterInsertQuery interpreter{insert, context}; @@ -521,7 +520,8 @@ bool StorageKafka::streamToViews() // Check whether the limits were applied during query execution bool limits_applied = false; - if (IProfilingBlockInputStream * p_stream = dynamic_cast(in.get())) { + if (IProfilingBlockInputStream * p_stream = dynamic_cast(in.get())) + { const BlockStreamProfileInfo & info = p_stream->getProfileInfo(); limits_applied = info.hasAppliedLimit(); } From fac040c39a16572c3a49906857cd25d7969ffa91 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Oct 2018 04:31:48 +0300 Subject: [PATCH 34/52] Update CapnProtoRowInputStream.cpp --- dbms/src/Formats/CapnProtoRowInputStream.cpp | 28 ++++++++++++++------ 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/dbms/src/Formats/CapnProtoRowInputStream.cpp b/dbms/src/Formats/CapnProtoRowInputStream.cpp index 598f33f49cb..cd28a673928 100644 --- a/dbms/src/Formats/CapnProtoRowInputStream.cpp +++ b/dbms/src/Formats/CapnProtoRowInputStream.cpp @@ -101,6 +101,7 @@ capnp::StructSchema::Field getFieldOrThrow(capnp::StructSchema node, const std:: else throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr()); } + void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields, capnp::StructSchema reader) { String last; @@ -121,25 +122,32 @@ void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields for (; level < field.tokens.size() - 1; ++level) { auto node = getFieldOrThrow(reader, field.tokens[level]); - if (node.getType().isStruct()) { + if (node.getType().isStruct()) + { // Descend to field structure last = field.tokens[level]; parent = node; reader = parent.getType().asStruct(); actions.push_back({Action::PUSH, parent}); - } else if (node.getType().isList()) { + } + else if (node.getType().isList()) + { break; // Collect list - } else + } + else throw Exception("Field " + field.tokens[level] + "is neither Struct nor List"); } // Read field from the structure auto node = getFieldOrThrow(reader, field.tokens[level]); - if (node.getType().isList() && actions.size() > 0 && actions.back().field == node) { + if (node.getType().isList() && actions.size() > 0 && actions.back().field == node) + { // The field list here flattens Nested elements into multiple arrays // In order to map Nested types in Cap'nProto back, they need to be collected actions.back().columns.push_back(field.pos); - } else { + } + else + { actions.push_back({Action::READ, node, {field.pos}}); } } @@ -219,16 +227,20 @@ bool CapnProtoRowInputStream::read(MutableColumns & columns) size_t size = collected.size(); // The flattened array contains an array of a part of the nested tuple Array flattened(size); - for (size_t column_index = 0; column_index < action.columns.size(); ++column_index) { + for (size_t column_index = 0; column_index < action.columns.size(); ++column_index) + { // Populate array with a single tuple elements - for (size_t off = 0; off < size; ++off) { + for (size_t off = 0; off < size; ++off) + { const TupleBackend & tuple = DB::get(collected[off]).toUnderType(); flattened[off] = tuple[column_index]; } auto & col = columns[action.columns[column_index]]; col->insert(flattened); } - } else { + } + else + { auto & col = columns[action.columns[0]]; col->insert(value); } From ff53aa4947ce3bf7fd58003d6f201a0bbaaf8b56 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Oct 2018 04:33:36 +0300 Subject: [PATCH 35/52] Update CapnProtoRowInputStream.cpp --- dbms/src/Formats/CapnProtoRowInputStream.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Formats/CapnProtoRowInputStream.cpp b/dbms/src/Formats/CapnProtoRowInputStream.cpp index cd28a673928..c30aa2c446a 100644 --- a/dbms/src/Formats/CapnProtoRowInputStream.cpp +++ b/dbms/src/Formats/CapnProtoRowInputStream.cpp @@ -220,7 +220,8 @@ bool CapnProtoRowInputStream::read(MutableColumns & columns) case Action::READ: { Field value = convertNodeToField(stack.back().get(action.field)); - if (action.columns.size() > 1) { + if (action.columns.size() > 1) + { // Nested columns must be flattened into several arrays // e.g. Array(Tuple(x ..., y ...)) -> Array(x ...), Array(y ...) const Array & collected = DB::get(value); From c4939a1bec0602c4be5fce49540f072b2c00a455 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Oct 2018 04:39:00 +0300 Subject: [PATCH 36/52] Update PushingToViewsBlockOutputStream.cpp --- .../DataStreams/PushingToViewsBlockOutputStream.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index a4cd29c778c..ed20e27bad4 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -78,13 +78,15 @@ void PushingToViewsBlockOutputStream::write(const Block & block) // Insert data into materialized views only after successful insert into main table bool allow_concurrent_view_processing = context.getSettingsRef().allow_concurrent_view_processing; - if (allow_concurrent_view_processing && views.size() > 1) { + if (allow_concurrent_view_processing && views.size() > 1) + { // Push to views concurrently if enabled, and more than one view is attached ThreadPool pool(std::min(getNumberOfPhysicalCPUCores(), views.size())); for (size_t view_num = 0; view_num < views.size(); ++view_num) { auto thread_group = CurrentThread::getGroup(); - pool.schedule([=] () { + pool.schedule([=] () + { setThreadName("PushingToViewsBlockOutputStream"); CurrentThread::attachToIfDetached(thread_group); process(block, view_num); @@ -92,7 +94,9 @@ void PushingToViewsBlockOutputStream::write(const Block & block) } // Wait for concurrent view processing pool.wait(); - } else { + } + else + { // Process sequentially for (size_t view_num = 0; view_num < views.size(); ++view_num) process(block, view_num); From ed77e40dfea80761178a0eeb98ec35fe1c5c35be Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 1 Oct 2018 12:01:50 +0300 Subject: [PATCH 37/52] fix implicit macros --- dbms/src/Common/Macros.cpp | 21 +++++++++++-------- dbms/src/Common/Macros.h | 4 ++++ .../Storages/StorageReplicatedMergeTree.cpp | 4 ++-- .../integration/test_distributed_ddl/test.py | 5 ++++- 4 files changed, 22 insertions(+), 12 deletions(-) diff --git a/dbms/src/Common/Macros.cpp b/dbms/src/Common/Macros.cpp index a5a82dfa833..9198a72a69d 100644 --- a/dbms/src/Common/Macros.cpp +++ b/dbms/src/Common/Macros.cpp @@ -56,19 +56,17 @@ String Macros::expand(const String & s, size_t level, const String & database_na throw Exception("Unbalanced { and } in string with macros: '" + s + "'", ErrorCodes::SYNTAX_ERROR); String macro_name = s.substr(begin, end - begin); + auto it = macros.find(macro_name); - if (macro_name == "database") + /// Prefer explicit macros over implicit. + if (it != macros.end()) + res += it->second; + else if (macro_name == "database" && !database_name.empty()) res += database_name; - else if (macro_name == "table") + else if (macro_name == "table" && !table_name.empty()) res += table_name; else - { - auto it = macros.find(macro_name); - if (it == macros.end()) - throw Exception("No macro " + macro_name + " in config", ErrorCodes::SYNTAX_ERROR); - - res += it->second; - } + throw Exception("No macro " + macro_name + " in config", ErrorCodes::SYNTAX_ERROR); pos = end + 1; } @@ -76,6 +74,11 @@ String Macros::expand(const String & s, size_t level, const String & database_na return expand(res, level + 1, database_name, table_name); } +String Macros::expand(const String & s, const String & database_name, const String & table_name) const +{ + return expand(s, 0, database_name, table_name); +} + Names Macros::expand(const Names & source_names, size_t level) const { Names result_names; diff --git a/dbms/src/Common/Macros.h b/dbms/src/Common/Macros.h index 8f1efdc094d..4b28a9d2cd6 100644 --- a/dbms/src/Common/Macros.h +++ b/dbms/src/Common/Macros.h @@ -28,10 +28,14 @@ public: Macros(const Poco::Util::AbstractConfiguration & config, const String & key); /** Replace the substring of the form {macro_name} with the value for macro_name, obtained from the config file. + * If {database} and {table} macros aren`t defined explicitly, expand them as database_name and table_name respectively. * level - the level of recursion. */ String expand(const String & s, size_t level = 0, const String & database_name = "", const String & table_name = "") const; + String expand(const String & s, const String & database_name, const String & table_name) const; + + /** Apply expand for the list. */ Names expand(const Names & source_names, size_t level = 0) const; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index c96575d9156..ddb5d20599e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -207,8 +207,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( : context(context_), database_name(database_name_), table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'), - zookeeper_path(context.getMacros()->expand(zookeeper_path_, 0, database_name, table_name)), - replica_name(context.getMacros()->expand(replica_name_)), + zookeeper_path(context.getMacros()->expand(zookeeper_path_, database_name, table_name)), + replica_name(context.getMacros()->expand(replica_name_, database_name, table_name)), data(database_name, table_name, full_path, columns_, context_, primary_expr_ast_, secondary_sorting_expr_list_, date_column_name, partition_expr_ast_, diff --git a/dbms/tests/integration/test_distributed_ddl/test.py b/dbms/tests/integration/test_distributed_ddl/test.py index 81a5c558236..aa2b41e1228 100755 --- a/dbms/tests/integration/test_distributed_ddl/test.py +++ b/dbms/tests/integration/test_distributed_ddl/test.py @@ -317,11 +317,14 @@ def test_macro(started_cluster): ddl_check_query(instance, "DROP TABLE IF EXISTS distr ON CLUSTER '{cluster}'") ddl_check_query(instance, "DROP TABLE IF EXISTS tab ON CLUSTER '{cluster}'") + +def test_implicit_macros(started_cluster): # Temporarily disable random ZK packet drops, they might broke creation if ReplicatedMergeTree replicas firewall_drops_rules = cluster.pm_random_drops.pop_rules() + instance = cluster.instances['ch2'] + ddl_check_query(instance, "DROP DATABASE IF EXISTS test_db ON CLUSTER '{cluster}'") - ddl_check_query(instance, "DROP TABLE IF EXISTS test_db.test_macro ON CLUSTER '{cluster}'") ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test_db ON CLUSTER '{cluster}'") ddl_check_query(instance, """ From ca03cd6606a81aeeed938a133bc1e03ffedfc34b Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 1 Oct 2018 18:43:48 +0300 Subject: [PATCH 38/52] Sync with arcadia --- dbms/programs/client/Client.cpp | 2 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 4 ++-- libs/libmysqlxx/src/Connection.cpp | 2 +- libs/libmysqlxx/src/Exception.cpp | 2 +- libs/libmysqlxx/src/Pool.cpp | 4 ++-- libs/libmysqlxx/src/Query.cpp | 2 +- libs/libmysqlxx/src/ResultBase.cpp | 2 +- libs/libmysqlxx/src/Row.cpp | 2 +- libs/libmysqlxx/src/StoreQueryResult.cpp | 2 +- libs/libmysqlxx/src/UseQueryResult.cpp | 2 +- 10 files changed, 12 insertions(+), 12 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index ad9c36a3a45..6ff8899cd93 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -61,7 +61,7 @@ #include #if USE_READLINE -#include "Suggest.h" +#include "Suggest.h" // Y_IGNORE #endif #ifndef __clang__ diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 10c77de58a3..5b014aa7511 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -24,7 +24,7 @@ #include #include #include -#include // Y_IGNORE +#include #include #include #include @@ -32,7 +32,7 @@ #if __has_include() // maybe bundled #include // Y_IGNORE #else // system -#include // Y_IGNORE +#include #endif diff --git a/libs/libmysqlxx/src/Connection.cpp b/libs/libmysqlxx/src/Connection.cpp index 7e771c48bfb..c69c735dd72 100644 --- a/libs/libmysqlxx/src/Connection.cpp +++ b/libs/libmysqlxx/src/Connection.cpp @@ -1,5 +1,5 @@ #if __has_include() -#include +#include // Y_IGNORE #else #include #endif diff --git a/libs/libmysqlxx/src/Exception.cpp b/libs/libmysqlxx/src/Exception.cpp index dadd37e29e7..eb778e0c86f 100644 --- a/libs/libmysqlxx/src/Exception.cpp +++ b/libs/libmysqlxx/src/Exception.cpp @@ -1,5 +1,5 @@ #if __has_include() -#include +#include // Y_IGNORE #else #include #endif diff --git a/libs/libmysqlxx/src/Pool.cpp b/libs/libmysqlxx/src/Pool.cpp index 28eb4169a43..9e592b4aba0 100644 --- a/libs/libmysqlxx/src/Pool.cpp +++ b/libs/libmysqlxx/src/Pool.cpp @@ -1,6 +1,6 @@ #if __has_include() -#include -#include +#include // Y_IGNORE +#include // Y_IGNORE #else #include #include diff --git a/libs/libmysqlxx/src/Query.cpp b/libs/libmysqlxx/src/Query.cpp index 0bcafa04421..57609e8365a 100644 --- a/libs/libmysqlxx/src/Query.cpp +++ b/libs/libmysqlxx/src/Query.cpp @@ -1,5 +1,5 @@ #if __has_include() -#include +#include // Y_IGNORE #else #include #endif diff --git a/libs/libmysqlxx/src/ResultBase.cpp b/libs/libmysqlxx/src/ResultBase.cpp index b03f92e38f2..b9fd3110acb 100644 --- a/libs/libmysqlxx/src/ResultBase.cpp +++ b/libs/libmysqlxx/src/ResultBase.cpp @@ -1,5 +1,5 @@ #if __has_include() -#include +#include // Y_IGNORE #else #include #endif diff --git a/libs/libmysqlxx/src/Row.cpp b/libs/libmysqlxx/src/Row.cpp index e4baa681d69..d7c6e67d0e7 100644 --- a/libs/libmysqlxx/src/Row.cpp +++ b/libs/libmysqlxx/src/Row.cpp @@ -1,5 +1,5 @@ #if __has_include() -#include +#include // Y_IGNORE #else #include #endif diff --git a/libs/libmysqlxx/src/StoreQueryResult.cpp b/libs/libmysqlxx/src/StoreQueryResult.cpp index 05ad4299e17..9b516cb05fb 100644 --- a/libs/libmysqlxx/src/StoreQueryResult.cpp +++ b/libs/libmysqlxx/src/StoreQueryResult.cpp @@ -1,5 +1,5 @@ #if __has_include() -#include +#include // Y_IGNORE #else #include #endif diff --git a/libs/libmysqlxx/src/UseQueryResult.cpp b/libs/libmysqlxx/src/UseQueryResult.cpp index c5c52ffcb9c..7d200583f9c 100644 --- a/libs/libmysqlxx/src/UseQueryResult.cpp +++ b/libs/libmysqlxx/src/UseQueryResult.cpp @@ -1,5 +1,5 @@ #if __has_include() -#include +#include // Y_IGNORE #else #include #endif From 49bb0652a8d2034fd2c4afd60ff4d8317ba2bb16 Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 1 Oct 2018 18:55:21 +0300 Subject: [PATCH 39/52] enable_local_infile --- libs/libmysqlxx/include/mysqlxx/Connection.h | 13 ++++++++++--- libs/libmysqlxx/include/mysqlxx/Pool.h | 9 ++++++--- libs/libmysqlxx/src/Connection.cpp | 14 ++++++++------ libs/libmysqlxx/src/Pool.cpp | 12 ++++++++++-- 4 files changed, 34 insertions(+), 14 deletions(-) diff --git a/libs/libmysqlxx/include/mysqlxx/Connection.h b/libs/libmysqlxx/include/mysqlxx/Connection.h index 1eb6b93692c..46e37eaeb48 100644 --- a/libs/libmysqlxx/include/mysqlxx/Connection.h +++ b/libs/libmysqlxx/include/mysqlxx/Connection.h @@ -13,6 +13,9 @@ #define MYSQLXX_DEFAULT_TIMEOUT 60 #define MYSQLXX_DEFAULT_RW_TIMEOUT 1800 +/// Disable LOAD DATA LOCAL INFILE because it is insecure +#define MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE false + namespace mysqlxx { @@ -72,7 +75,8 @@ public: const char * ssl_cert = "", const char * ssl_key = "", unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, - unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT); + unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT, + bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); /// Creates connection. Can be used if Poco::Util::Application is using. /// All settings will be got from config_name section of configuration. @@ -91,7 +95,8 @@ public: const char* ssl_cert, const char* ssl_key, unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, - unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT); + unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT, + bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); void connect(const std::string & config_name) { @@ -106,6 +111,7 @@ public: std::string ssl_ca = cfg.getString(config_name + ".ssl_ca", ""); std::string ssl_cert = cfg.getString(config_name + ".ssl_cert", ""); std::string ssl_key = cfg.getString(config_name + ".ssl_key", ""); + bool enable_local_infile = cfg.getBool(config_name + ".enable_local_infile", MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); unsigned timeout = cfg.getInt(config_name + ".connect_timeout", @@ -128,7 +134,8 @@ public: ssl_cert.c_str(), ssl_key.c_str(), timeout, - rw_timeout); + rw_timeout, + enable_local_infile); } /// If MySQL connection was established. diff --git a/libs/libmysqlxx/include/mysqlxx/Pool.h b/libs/libmysqlxx/include/mysqlxx/Pool.h index 957d80418bf..bc7efcc06c3 100644 --- a/libs/libmysqlxx/include/mysqlxx/Pool.h +++ b/libs/libmysqlxx/include/mysqlxx/Pool.h @@ -162,10 +162,11 @@ public: unsigned connect_timeout_ = MYSQLXX_DEFAULT_TIMEOUT, unsigned rw_timeout_ = MYSQLXX_DEFAULT_RW_TIMEOUT, unsigned default_connections_ = MYSQLXX_POOL_DEFAULT_START_CONNECTIONS, - unsigned max_connections_ = MYSQLXX_POOL_DEFAULT_MAX_CONNECTIONS) + unsigned max_connections_ = MYSQLXX_POOL_DEFAULT_MAX_CONNECTIONS, + unsigned enable_local_infile_ = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE) : default_connections(default_connections_), max_connections(max_connections_), db(db_), server(server_), user(user_), password(password_), port(port_), socket(socket_), - connect_timeout(connect_timeout_), rw_timeout(rw_timeout_) {} + connect_timeout(connect_timeout_), rw_timeout(rw_timeout_), enable_local_infile(enable_local_infile_) {} Pool(const Pool & other) : default_connections{other.default_connections}, @@ -173,7 +174,8 @@ public: db{other.db}, server{other.server}, user{other.user}, password{other.password}, port{other.port}, socket{other.socket}, - connect_timeout{other.connect_timeout}, rw_timeout{other.rw_timeout} + connect_timeout{other.connect_timeout}, rw_timeout{other.rw_timeout}, + enable_local_infile{other.enable_local_infile} {} Pool & operator=(const Pool &) = delete; @@ -224,6 +226,7 @@ private: std::string ssl_ca; std::string ssl_cert; std::string ssl_key; + bool enable_local_infile; /// True if connection was established at least once. bool was_successful{false}; diff --git a/libs/libmysqlxx/src/Connection.cpp b/libs/libmysqlxx/src/Connection.cpp index c69c735dd72..e26a067f3d1 100644 --- a/libs/libmysqlxx/src/Connection.cpp +++ b/libs/libmysqlxx/src/Connection.cpp @@ -45,10 +45,11 @@ Connection::Connection( const char* ssl_cert, const char* ssl_key, unsigned timeout, - unsigned rw_timeout) + unsigned rw_timeout, + bool enable_local_infile) : Connection() { - connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout); + connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout, enable_local_infile); } Connection::Connection(const std::string & config_name) @@ -73,7 +74,8 @@ void Connection::connect(const char* db, const char * ssl_cert, const char * ssl_key, unsigned timeout, - unsigned rw_timeout) + unsigned rw_timeout, + bool enable_local_infile) { if (is_connected) disconnect(); @@ -92,9 +94,9 @@ void Connection::connect(const char* db, if (mysql_options(driver.get(), MYSQL_OPT_WRITE_TIMEOUT, &rw_timeout)) throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); - /// Disable LOAD DATA LOCAL INFILE because it is insecure. - unsigned enable_local_infile = 0; - if (mysql_options(driver.get(), MYSQL_OPT_LOCAL_INFILE, &enable_local_infile)) + /// Disable LOAD DATA LOCAL INFILE because it is insecure if necessary. + unsigned enable_local_infile_arg = static_cast(enable_local_infile); + if (mysql_options(driver.get(), MYSQL_OPT_LOCAL_INFILE, &enable_local_infile_arg)) throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); /// Specifies particular ssl key and certificate if it needs diff --git a/libs/libmysqlxx/src/Pool.cpp b/libs/libmysqlxx/src/Pool.cpp index 9e592b4aba0..494ae267ff7 100644 --- a/libs/libmysqlxx/src/Pool.cpp +++ b/libs/libmysqlxx/src/Pool.cpp @@ -69,6 +69,9 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co ssl_key = cfg.has(config_name + ".ssl_key") ? cfg.getString(config_name + ".ssl_key") : cfg.getString(parent_config_name + ".ssl_key", ""); + + enable_local_infile = cfg.getBool(config_name + ".enable_local_infile", + cfg.getBool(parent_config_name + ".enable_local_infile", MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE)); } else { @@ -84,6 +87,9 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co ssl_ca = cfg.getString(config_name + ".ssl_ca", ""); ssl_cert = cfg.getString(config_name + ".ssl_cert", ""); ssl_key = cfg.getString(config_name + ".ssl_key", ""); + + enable_local_infile = cfg.getBool( + config_name + ".enable_local_infile", MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); } connect_timeout = cfg.getInt(config_name + ".connect_timeout", @@ -192,7 +198,8 @@ void Pool::Entry::forceConnected() const pool->ssl_cert.c_str(), pool->ssl_key.c_str(), pool->connect_timeout, - pool->rw_timeout); + pool->rw_timeout, + pool->enable_local_infile); } while (!data->conn.ping()); } @@ -233,7 +240,8 @@ Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) ssl_cert.c_str(), ssl_key.c_str(), connect_timeout, - rw_timeout); + rw_timeout, + enable_local_infile); } catch (mysqlxx::ConnectionFailed & e) { From 6017be28f66121ab3599d0376ab0c8f7364ceb3c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Oct 2018 19:30:12 +0300 Subject: [PATCH 40/52] CLICKHOUSE-4007: Fix PerfTest substitution function --- .../performance-test/PerformanceTest.cpp | 48 ++++++++----------- 1 file changed, 19 insertions(+), 29 deletions(-) diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index 7a21617c97d..e63d40e2db0 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -1159,11 +1159,8 @@ private: StringToVector::iterator substitutions_first = substitutions.begin(); StringToVector::iterator substitutions_last = substitutions.end(); - --substitutions_last; - std::map substitutions_map; - - runThroughAllOptionsAndPush(substitutions_first, substitutions_last, query, queries, substitutions_map); + runThroughAllOptionsAndPush(substitutions_first, substitutions_last, query, queries); return queries; } @@ -1173,44 +1170,37 @@ private: void runThroughAllOptionsAndPush(StringToVector::iterator substitutions_left, StringToVector::iterator substitutions_right, const String & template_query, - std::vector & queries, - const StringKeyValue & template_substitutions_map = StringKeyValue()) + std::vector & queries) { - String name = substitutions_left->first; - std::vector values = substitutions_left->second; + if (substitutions_left == substitutions_right) + { + queries.push_back(template_query); /// completely substituted query + return; + } - for (const String & value : values) + String substitution_mask = "{" + substitutions_left->first + "}"; + + if (template_query.find(substitution_mask) == String::npos) /// nothing to substitute here + { + runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, template_query, queries); + return; + } + + for (const String & value : substitutions_left->second) { /// Copy query string for each unique permutation Query query = template_query; - StringKeyValue substitutions_map = template_substitutions_map; size_t substr_pos = 0; while (substr_pos != String::npos) { - substr_pos = query.find("{" + name + "}"); + substr_pos = query.find(substitution_mask); if (substr_pos != String::npos) - { - query.replace(substr_pos, 1 + name.length() + 1, value); - } + query.replace(substr_pos, substitution_mask.length(), value); } - substitutions_map[name] = value; - - /// If we've reached the end of substitution chain - if (substitutions_left == substitutions_right) - { - queries.push_back(query); - substitutions_maps.push_back(substitutions_map); - } - else - { - StringToVector::iterator next_it = substitutions_left; - ++next_it; - - runThroughAllOptionsAndPush(next_it, substitutions_right, query, queries, substitutions_map); - } + runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, query, queries); } } From 26a951ee5a5a7423acc7fcac2d076655a53613e8 Mon Sep 17 00:00:00 2001 From: ns-vasilev Date: Mon, 1 Oct 2018 21:04:55 +0300 Subject: [PATCH 41/52] Added "LOAD DATA LOCAL INFILE" arg --- libs/libmysqlxx/include/mysqlxx/Connection.h | 13 ++++++++++--- libs/libmysqlxx/include/mysqlxx/Pool.h | 9 ++++++--- libs/libmysqlxx/src/Connection.cpp | 14 ++++++++------ libs/libmysqlxx/src/Pool.cpp | 12 ++++++++++-- 4 files changed, 34 insertions(+), 14 deletions(-) diff --git a/libs/libmysqlxx/include/mysqlxx/Connection.h b/libs/libmysqlxx/include/mysqlxx/Connection.h index 93efb7d7002..797ebc5d28e 100644 --- a/libs/libmysqlxx/include/mysqlxx/Connection.h +++ b/libs/libmysqlxx/include/mysqlxx/Connection.h @@ -13,6 +13,9 @@ #define MYSQLXX_DEFAULT_TIMEOUT 60 #define MYSQLXX_DEFAULT_RW_TIMEOUT 1800 +/// Disable LOAD DATA LOCAL INFILE because it is insecure +#define MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE false + namespace mysqlxx { @@ -72,7 +75,8 @@ public: const char * ssl_cert = "", const char * ssl_key = "", unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, - unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT); + unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT, + bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); /// Creates connection. Can be used if Poco::Util::Application is using. /// All settings will be got from config_name section of configuration. @@ -91,7 +95,8 @@ public: const char* ssl_cert, const char* ssl_key, unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, - unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT); + unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT, + bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); void connect(const std::string & config_name) { @@ -106,6 +111,7 @@ public: std::string ssl_ca = cfg.getString(config_name + ".ssl_ca", ""); std::string ssl_cert = cfg.getString(config_name + ".ssl_cert", ""); std::string ssl_key = cfg.getString(config_name + ".ssl_key", ""); + bool enable_local_infile = cfg.getBool(config_name + ".enable_local_infile", MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); unsigned timeout = cfg.getInt(config_name + ".connect_timeout", @@ -128,7 +134,8 @@ public: ssl_cert.c_str(), ssl_key.c_str(), timeout, - rw_timeout); + rw_timeout, + enable_local_infile); } /// If MySQL connection was established. diff --git a/libs/libmysqlxx/include/mysqlxx/Pool.h b/libs/libmysqlxx/include/mysqlxx/Pool.h index 957d80418bf..bc7efcc06c3 100644 --- a/libs/libmysqlxx/include/mysqlxx/Pool.h +++ b/libs/libmysqlxx/include/mysqlxx/Pool.h @@ -162,10 +162,11 @@ public: unsigned connect_timeout_ = MYSQLXX_DEFAULT_TIMEOUT, unsigned rw_timeout_ = MYSQLXX_DEFAULT_RW_TIMEOUT, unsigned default_connections_ = MYSQLXX_POOL_DEFAULT_START_CONNECTIONS, - unsigned max_connections_ = MYSQLXX_POOL_DEFAULT_MAX_CONNECTIONS) + unsigned max_connections_ = MYSQLXX_POOL_DEFAULT_MAX_CONNECTIONS, + unsigned enable_local_infile_ = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE) : default_connections(default_connections_), max_connections(max_connections_), db(db_), server(server_), user(user_), password(password_), port(port_), socket(socket_), - connect_timeout(connect_timeout_), rw_timeout(rw_timeout_) {} + connect_timeout(connect_timeout_), rw_timeout(rw_timeout_), enable_local_infile(enable_local_infile_) {} Pool(const Pool & other) : default_connections{other.default_connections}, @@ -173,7 +174,8 @@ public: db{other.db}, server{other.server}, user{other.user}, password{other.password}, port{other.port}, socket{other.socket}, - connect_timeout{other.connect_timeout}, rw_timeout{other.rw_timeout} + connect_timeout{other.connect_timeout}, rw_timeout{other.rw_timeout}, + enable_local_infile{other.enable_local_infile} {} Pool & operator=(const Pool &) = delete; @@ -224,6 +226,7 @@ private: std::string ssl_ca; std::string ssl_cert; std::string ssl_key; + bool enable_local_infile; /// True if connection was established at least once. bool was_successful{false}; diff --git a/libs/libmysqlxx/src/Connection.cpp b/libs/libmysqlxx/src/Connection.cpp index 00eeed49616..64baf822d6c 100644 --- a/libs/libmysqlxx/src/Connection.cpp +++ b/libs/libmysqlxx/src/Connection.cpp @@ -43,11 +43,12 @@ Connection::Connection( const char* ssl_cert, const char* ssl_key, unsigned timeout, - unsigned rw_timeout) + unsigned rw_timeout, + bool enable_local_infile) : driver(std::make_unique()) { is_connected = false; - connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout); + connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout, enable_local_infile); } Connection::Connection(const std::string & config_name) @@ -73,7 +74,8 @@ void Connection::connect(const char* db, const char * ssl_cert, const char * ssl_key, unsigned timeout, - unsigned rw_timeout) + unsigned rw_timeout, + bool enable_local_infile) { if (is_connected) disconnect(); @@ -94,9 +96,9 @@ void Connection::connect(const char* db, if (mysql_options(driver.get(), MYSQL_OPT_WRITE_TIMEOUT, &rw_timeout)) throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); - /// Disable LOAD DATA LOCAL INFILE because it is insecure. - unsigned enable_local_infile = 0; - if (mysql_options(driver.get(), MYSQL_OPT_LOCAL_INFILE, &enable_local_infile)) + /// Disable LOAD DATA LOCAL INFILE because it is insecure if necessary. + unsigned enable_local_infile_arg = static_cast(enable_local_infile); + if (mysql_options(driver.get(), MYSQL_OPT_LOCAL_INFILE, &enable_local_infile_arg)) throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); /// Specifies particular ssl key and certificate if it needs diff --git a/libs/libmysqlxx/src/Pool.cpp b/libs/libmysqlxx/src/Pool.cpp index 33065df2bb5..682559778b1 100644 --- a/libs/libmysqlxx/src/Pool.cpp +++ b/libs/libmysqlxx/src/Pool.cpp @@ -65,6 +65,9 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co ssl_key = cfg.has(config_name + ".ssl_key") ? cfg.getString(config_name + ".ssl_key") : cfg.getString(parent_config_name + ".ssl_key", ""); + + enable_local_infile = cfg.getBool(config_name + ".enable_local_infile", + cfg.getBool(parent_config_name + ".enable_local_infile", MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE)); } else { @@ -80,6 +83,9 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co ssl_ca = cfg.getString(config_name + ".ssl_ca", ""); ssl_cert = cfg.getString(config_name + ".ssl_cert", ""); ssl_key = cfg.getString(config_name + ".ssl_key", ""); + + enable_local_infile = cfg.getBool( + config_name + ".enable_local_infile", MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); } connect_timeout = cfg.getInt(config_name + ".connect_timeout", @@ -188,7 +194,8 @@ void Pool::Entry::forceConnected() const pool->ssl_cert.c_str(), pool->ssl_key.c_str(), pool->connect_timeout, - pool->rw_timeout); + pool->rw_timeout, + pool->enable_local_infile); } while (!data->conn.ping()); } @@ -229,7 +236,8 @@ Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) ssl_cert.c_str(), ssl_key.c_str(), connect_timeout, - rw_timeout); + rw_timeout, + enable_local_infile); } catch (mysqlxx::ConnectionFailed & e) { From 3813cccd6e8dc9caae8b26f4a4ecc3e15a2d6361 Mon Sep 17 00:00:00 2001 From: ns-vasilev Date: Mon, 1 Oct 2018 21:19:59 +0300 Subject: [PATCH 42/52] resolved conflict --- libs/libmysqlxx/src/Connection.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/libs/libmysqlxx/src/Connection.cpp b/libs/libmysqlxx/src/Connection.cpp index 64baf822d6c..cbed56572f7 100644 --- a/libs/libmysqlxx/src/Connection.cpp +++ b/libs/libmysqlxx/src/Connection.cpp @@ -45,10 +45,9 @@ Connection::Connection( unsigned timeout, unsigned rw_timeout, bool enable_local_infile) - : driver(std::make_unique()) + : Connection() { - is_connected = false; - connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout, enable_local_infile); + connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout); } Connection::Connection(const std::string & config_name) From 6432cff455b94e1d22cb0da29834fab28cc10102 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Oct 2018 21:45:39 +0300 Subject: [PATCH 43/52] Update Macros.h --- dbms/src/Common/Macros.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/Macros.h b/dbms/src/Common/Macros.h index 4b28a9d2cd6..60a6bc18e4c 100644 --- a/dbms/src/Common/Macros.h +++ b/dbms/src/Common/Macros.h @@ -28,7 +28,7 @@ public: Macros(const Poco::Util::AbstractConfiguration & config, const String & key); /** Replace the substring of the form {macro_name} with the value for macro_name, obtained from the config file. - * If {database} and {table} macros aren`t defined explicitly, expand them as database_name and table_name respectively. + * If {database} and {table} macros aren`t defined explicitly, expand them as database_name and table_name respectively. * level - the level of recursion. */ String expand(const String & s, size_t level = 0, const String & database_name = "", const String & table_name = "") const; From 698be017e9a943e6321b9a192ee9d8ba67f5324e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Oct 2018 00:49:56 +0300 Subject: [PATCH 44/52] Changes after merge #3208 --- dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp | 8 ++++---- dbms/src/Interpreters/Settings.h | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index ed20e27bad4..b5178cbb17c 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -77,15 +77,15 @@ void PushingToViewsBlockOutputStream::write(const Block & block) return; // Insert data into materialized views only after successful insert into main table - bool allow_concurrent_view_processing = context.getSettingsRef().allow_concurrent_view_processing; - if (allow_concurrent_view_processing && views.size() > 1) + const Settings & settings = context.getSettingsRef(); + if (settings.parallel_view_processing && views.size() > 1) { // Push to views concurrently if enabled, and more than one view is attached - ThreadPool pool(std::min(getNumberOfPhysicalCPUCores(), views.size())); + ThreadPool pool(std::min(size_t(settings.max_threads), views.size())); for (size_t view_num = 0; view_num < views.size(); ++view_num) { auto thread_group = CurrentThread::getGroup(); - pool.schedule([=] () + pool.schedule([=] { setThreadName("PushingToViewsBlockOutputStream"); CurrentThread::attachToIfDetached(thread_group); diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 27eeb60af57..d8310edae3e 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -291,7 +291,7 @@ struct Settings M(SettingUInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).") \ M(SettingBool, calculate_text_stack_trace, 1, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.") \ M(SettingBool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.") \ - M(SettingBool, allow_concurrent_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.") \ + M(SettingBool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.") \ #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ From 7df73088abd2e8f6d01c580c4e577783e63dea1a Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 29 Sep 2018 22:14:29 +0800 Subject: [PATCH 45/52] Correct wcwidth computation for pretty outputs. --- dbms/CMakeLists.txt | 1 + dbms/src/Common/UTF8Helpers.cpp | 134 +++++ dbms/src/Common/UTF8Helpers.h | 5 + dbms/src/Formats/PrettyBlockOutputStream.cpp | 7 +- dbms/src/Formats/VerticalRowOutputStream.cpp | 7 +- .../00298_enum_width_and_cast.reference | 20 +- .../00730_unicode_terminal_format.reference | 101 ++++ .../00730_unicode_terminal_format.sql | 30 + libs/CMakeLists.txt | 1 + libs/libwidechar_width/CMakeLists.txt | 2 + libs/libwidechar_width/LICENSE | 4 + libs/libwidechar_width/README.md | 30 + libs/libwidechar_width/widechar_width.cpp | 1 + libs/libwidechar_width/widechar_width.h | 521 ++++++++++++++++++ 14 files changed, 850 insertions(+), 14 deletions(-) create mode 100644 dbms/src/Common/UTF8Helpers.cpp create mode 100644 dbms/tests/queries/0_stateless/00730_unicode_terminal_format.reference create mode 100644 dbms/tests/queries/0_stateless/00730_unicode_terminal_format.sql create mode 100644 libs/libwidechar_width/CMakeLists.txt create mode 100644 libs/libwidechar_width/LICENSE create mode 100644 libs/libwidechar_width/README.md create mode 100644 libs/libwidechar_width/widechar_width.cpp create mode 100644 libs/libwidechar_width/widechar_width.h diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 69fe7f46a4b..0cb5824e1fc 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -162,6 +162,7 @@ endif() target_link_libraries (clickhouse_common_io common string_utils + widechar_width ${LINK_LIBRARIES_ONLY_ON_X86_64} ${LZ4_LIBRARY} ${ZSTD_LIBRARY} diff --git a/dbms/src/Common/UTF8Helpers.cpp b/dbms/src/Common/UTF8Helpers.cpp new file mode 100644 index 00000000000..c18cce6df4e --- /dev/null +++ b/dbms/src/Common/UTF8Helpers.cpp @@ -0,0 +1,134 @@ +#include + +#include + +namespace DB +{ +namespace UTF8 +{ + +// based on https://bjoern.hoehrmann.de/utf-8/decoder/dfa/ +// Copyright (c) 2008-2009 Bjoern Hoehrmann +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: The above copyright +// notice and this permission notice shall be included in all copies or +// substantial portions of the Software. +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +// SOFTWARE. + +static const UInt8 TABLE[] = +{ + 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, // 00..1f + 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, // 20..3f + 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, // 40..5f + 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, // 60..7f + 1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,9,9,9,9,9,9,9,9,9,9,9,9,9,9,9,9, // 80..9f + 7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7, // a0..bf + 8,8,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2, // c0..df + 0xa,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x4,0x3,0x3, // e0..ef + 0xb,0x6,0x6,0x6,0x5,0x8,0x8,0x8,0x8,0x8,0x8,0x8,0x8,0x8,0x8,0x8, // f0..ff + 0x0,0x1,0x2,0x3,0x5,0x8,0x7,0x1,0x1,0x1,0x4,0x6,0x1,0x1,0x1,0x1, // s0..s0 + 1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,0,1,1,1,1,1,0,1,0,1,1,1,1,1,1, // s1..s2 + 1,2,1,1,1,1,1,2,1,2,1,1,1,1,1,1,1,1,1,1,1,1,1,2,1,1,1,1,1,1,1,1, // s3..s4 + 1,2,1,1,1,1,1,1,1,2,1,1,1,1,1,1,1,1,1,1,1,1,1,3,1,3,1,1,1,1,1,1, // s5..s6 + 1,3,1,1,1,1,1,3,1,3,1,1,1,1,1,1,1,3,1,1,1,1,1,1,1,1,1,1,1,1,1,1, // s7..s8 +}; + +struct UTF8Decoder +{ + enum + { + ACCEPT = 0, + REJECT = 1 + }; + + UInt32 decode(UInt8 byte) + { + UInt32 type = TABLE[byte]; + codepoint = (state != ACCEPT) ? (byte & 0x3fu) | (codepoint << 6) : (0xff >> type) & (byte); + state = TABLE[256 + state * 16 + type]; + return state; + } + + void reset() + { + state = ACCEPT; + codepoint = 0xfffdU; + } + + UInt8 state {ACCEPT}; + UInt32 codepoint {0}; +}; + +static int wcwidth(wchar_t wc) +{ + int width = widechar_wcwidth(wc); + switch (width) + { + case widechar_nonprint: + [[fallthrough]]; + case widechar_combining: + [[fallthrough]]; + case widechar_unassigned: + return 0; + case widechar_ambiguous: + [[fallthrough]]; + case widechar_private_use: + [[fallthrough]]; + case widechar_widened_in_9: + return 1; + default: + return width; + } +} + +size_t computeWidth(const UInt8 * data, size_t size, size_t prefix) noexcept +{ + UTF8Decoder decoder; + size_t width = 0; + size_t rollback = 0; + for (size_t i = 0; i < size; ++i) + { + switch (decoder.decode(data[i])) + { + case UTF8Decoder::REJECT: + decoder.reset(); + // invalid sequences seem to have zero width in modern terminals + // tested in libvte-based, alacritty, urxvt and xterm + i -= rollback; + rollback = 0; + break; + case UTF8Decoder::ACCEPT: + // there are special control characters that manipulate the terminal output. + // (`0x08`, `0x09`, `0x0a`, `0x0b`, `0x0c`, `0x0d`, `0x1b`) + // Since we don't touch the original column data, there is no easy way to escape them. + // TODO: escape control characters + // TODO: multiline support for '\n' + + // special treatment for '\t' + if (decoder.codepoint == '\t') + width += 8 - (prefix + width) % 8; + else + width += wcwidth(decoder.codepoint); + rollback = 0; + break; + // continue if we meet other values here + default: + ++rollback; + } + } + + // no need to handle trailing sequence as they have zero width + return width; +} +} +} diff --git a/dbms/src/Common/UTF8Helpers.h b/dbms/src/Common/UTF8Helpers.h index 5c32048bb7c..e02ce23b343 100644 --- a/dbms/src/Common/UTF8Helpers.h +++ b/dbms/src/Common/UTF8Helpers.h @@ -72,6 +72,11 @@ inline size_t countCodePoints(const UInt8 * data, size_t size) return res; } +/// returns UTF-8 wcswidth. Invalid sequence is treated as zero width character. +/// `prefix` is used to compute the `\t` width which extends the string before +/// and include `\t` to the nearest longer length with multiple of eight. +size_t computeWidth(const UInt8 * data, size_t size, size_t prefix = 0) noexcept; + } diff --git a/dbms/src/Formats/PrettyBlockOutputStream.cpp b/dbms/src/Formats/PrettyBlockOutputStream.cpp index 4f5c3441cc1..16df780993e 100644 --- a/dbms/src/Formats/PrettyBlockOutputStream.cpp +++ b/dbms/src/Formats/PrettyBlockOutputStream.cpp @@ -47,6 +47,7 @@ void PrettyBlockOutputStream::calculateWidths( /// Calculate widths of all values. String serialized_value; + size_t prefix = 2; // Tab character adjustment for (size_t i = 0; i < columns; ++i) { const ColumnWithTypeAndName & elem = block.getByPosition(i); @@ -61,16 +62,18 @@ void PrettyBlockOutputStream::calculateWidths( } widths[i][j] = std::min(format_settings.pretty.max_column_pad_width, - UTF8::countCodePoints(reinterpret_cast(serialized_value.data()), serialized_value.size())); + UTF8::computeWidth(reinterpret_cast(serialized_value.data()), serialized_value.size(), prefix)); max_widths[i] = std::max(max_widths[i], widths[i][j]); } /// And also calculate widths for names of columns. { + // name string doesn't contain Tab, no need to pass `prefix` name_widths[i] = std::min(format_settings.pretty.max_column_pad_width, - UTF8::countCodePoints(reinterpret_cast(elem.name.data()), elem.name.size())); + UTF8::computeWidth(reinterpret_cast(elem.name.data()), elem.name.size())); max_widths[i] = std::max(max_widths[i], name_widths[i]); } + prefix += max_widths[i] + 3; } } diff --git a/dbms/src/Formats/VerticalRowOutputStream.cpp b/dbms/src/Formats/VerticalRowOutputStream.cpp index 0b5bda81d5b..aafbadb8e59 100644 --- a/dbms/src/Formats/VerticalRowOutputStream.cpp +++ b/dbms/src/Formats/VerticalRowOutputStream.cpp @@ -28,7 +28,7 @@ VerticalRowOutputStream::VerticalRowOutputStream( /// Note that number of code points is just a rough approximation of visible string width. const String & name = sample.getByPosition(i).name; - name_widths[i] = UTF8::countCodePoints(reinterpret_cast(name.data()), name.size()); + name_widths[i] = UTF8::computeWidth(reinterpret_cast(name.data()), name.size()); if (name_widths[i] > max_name_width) max_name_width = name_widths[i]; @@ -43,7 +43,10 @@ VerticalRowOutputStream::VerticalRowOutputStream( } for (size_t i = 0; i < columns; ++i) - names_and_paddings[i].resize(max_name_width + strlen(": "), ' '); + { + size_t new_size = max_name_width - name_widths[i] + names_and_paddings[i].size(); + names_and_paddings[i].resize(new_size, ' '); + } } diff --git a/dbms/tests/queries/0_stateless/00298_enum_width_and_cast.reference b/dbms/tests/queries/0_stateless/00298_enum_width_and_cast.reference index 8ce3d6a62e2..617494b3101 100644 --- a/dbms/tests/queries/0_stateless/00298_enum_width_and_cast.reference +++ b/dbms/tests/queries/0_stateless/00298_enum_width_and_cast.reference @@ -5,13 +5,13 @@ │ Hello │ 0 │ │ \ │ 0 │ └───────┴───┘ -┌─x─────┬─y─┐ -│ Hello │ 0 │ -│ \ │ 0 │ -│ \t │ 0 │ -└───────┴───┘ -┌─x─────┬─y─┬─toInt8(x)─┬─s─────┬─casted─┐ -│ Hello │ 0 │ -100 │ Hello │ Hello │ -│ \ │ 0 │ 0 │ \ │ \ │ -│ \t │ 0 │ 111 │ \t │ \t │ -└───────┴───┴───────────┴───────┴────────┘ +┌─x────────┬─y─┐ +│ Hello │ 0 │ +│ \ │ 0 │ +│ \t │ 0 │ +└──────────┴───┘ +┌─x────────┬─y─┬─toInt8(x)─┬─s─────┬─casted─┐ +│ Hello │ 0 │ -100 │ Hello │ Hello │ +│ \ │ 0 │ 0 │ \ │ \ │ +│ \t │ 0 │ 111 │ \t │ \t │ +└──────────┴───┴───────────┴───────┴────────┘ diff --git a/dbms/tests/queries/0_stateless/00730_unicode_terminal_format.reference b/dbms/tests/queries/0_stateless/00730_unicode_terminal_format.reference new file mode 100644 index 00000000000..ff342111da9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00730_unicode_terminal_format.reference @@ -0,0 +1,101 @@ +┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Здравствуйте │ Этот код можно отредактировать и запустить! │ +└──────────────┴─────────────────────────────────────────────┘ +┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ 你好 │ 这段代码是可以编辑并且能够运行的! │ +└──────┴────────────────────────────────────┘ +┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Hola │ ¡Este código es editable y ejecutable! │ +└──────┴────────────────────────────────────────┘ +┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Bonjour │ Ce code est modifiable et exécutable ! │ +└─────────┴────────────────────────────────────────┘ +┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Ciao │ Questo codice è modificabile ed eseguibile! │ +└──────┴─────────────────────────────────────────────┘ +┏━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ こんにちは │ このコードは編集して実行出来ます! │ +└────────────┴────────────────────────────────────┘ +┏━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ 안녕하세요 │ 여기에서 코드를 수정하고 실행할 수 있습니다! │ +└────────────┴──────────────────────────────────────────────┘ +┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Cześć │ Ten kod można edytować oraz uruchomić! │ +└───────┴────────────────────────────────────────┘ +┏━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Olá │ Este código é editável e executável! │ +└─────┴──────────────────────────────────────┘ +┏━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Chào bạn │ Bạn có thể edit và run code trực tiếp! │ +└──────────┴────────────────────────────────────────┘ +┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Hallo │ Dieser Code kann bearbeitet und ausgeführt werden! │ +└───────┴────────────────────────────────────────────────────┘ +┏━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Hej │ Den här koden kan redigeras och köras! │ +└─────┴────────────────────────────────────────┘ +┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Ahoj │ Tento kód můžete upravit a spustit │ +└──────┴────────────────────────────────────┘ +┏━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Tabs Tabs │ Non-first Tabs │ +└─────────────┴───────────────────────┘ +┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Control characters  with zero width │ Invalid UTF-8 which eats pending characters , or invalid by itself with zero width │ +└─────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────┘ +┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━┓ +┃ c1 ┃ c2 ┃ +┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━┩ +│ Russian ё and ё │ Zero bytes in middle │ +└──────────────────┴────────────────────────┘ +┏━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┓ +┃ 'Tabs \t Tabs' ┃ 'Long\tTitle' ┃ +┡━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━┩ +│ Tabs Tabs │ Long Title │ +└────────────────┴───────────────┘ +Row 1: +────── +'你好': 你好 +'世界': 世界 +Row 1: +────── +'Tabs \t Tabs': Tabs Tabs +'Non-first \t Tabs': Non-first Tabs +Row 1: +────── +'Control characters  with zero width': Control characters  with zero width +'Invalid UTF-8 which eats pending characters , and invalid by itself with zero width': Invalid UTF-8 which eats pending characters , and invalid by itself with zero width +Row 1: +────── +'Russian ё and ё': Russian ё and ё +'Zero bytes \0 \0 in middle': Zero bytes in middle diff --git a/dbms/tests/queries/0_stateless/00730_unicode_terminal_format.sql b/dbms/tests/queries/0_stateless/00730_unicode_terminal_format.sql new file mode 100644 index 00000000000..e43c3279722 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00730_unicode_terminal_format.sql @@ -0,0 +1,30 @@ +CREATE DATABASE IF NOT EXISTS test; +DROP TABLE IF EXISTS test.unicode; + +CREATE TABLE test.unicode(c1 String, c2 String) ENGINE = Memory; +INSERT INTO test.unicode VALUES ('Здравствуйте', 'Этот код можно отредактировать и запустить!'), +INSERT INTO test.unicode VALUES ('你好', '这段代码是可以编辑并且能够运行的!'), +INSERT INTO test.unicode VALUES ('Hola', '¡Este código es editable y ejecutable!'), +INSERT INTO test.unicode VALUES ('Bonjour', 'Ce code est modifiable et exécutable !'), +INSERT INTO test.unicode VALUES ('Ciao', 'Questo codice è modificabile ed eseguibile!'), +INSERT INTO test.unicode VALUES ('こんにちは', 'このコードは編集して実行出来ます!'), +INSERT INTO test.unicode VALUES ('안녕하세요', '여기에서 코드를 수정하고 실행할 수 있습니다!'), +INSERT INTO test.unicode VALUES ('Cześć', 'Ten kod można edytować oraz uruchomić!'), +INSERT INTO test.unicode VALUES ('Olá', 'Este código é editável e executável!'), +INSERT INTO test.unicode VALUES ('Chào bạn', 'Bạn có thể edit và run code trực tiếp!'), +INSERT INTO test.unicode VALUES ('Hallo', 'Dieser Code kann bearbeitet und ausgeführt werden!'), +INSERT INTO test.unicode VALUES ('Hej', 'Den här koden kan redigeras och köras!'), +INSERT INTO test.unicode VALUES ('Ahoj', 'Tento kód můžete upravit a spustit'); +INSERT INTO test.unicode VALUES ('Tabs \t Tabs', 'Non-first \t Tabs'); +INSERT INTO test.unicode VALUES ('Control characters \x1f\x1f\x1f\x1f with zero width', 'Invalid UTF-8 which eats pending characters \xf0, or invalid by itself \x80 with zero width'); +INSERT INTO test.unicode VALUES ('Russian ё and ё ', 'Zero bytes \0 \0 in middle'); +SELECT * FROM test.unicode SETTINGS max_threads = 1 FORMAT PrettyNoEscapes; +SELECT 'Tabs \t Tabs', 'Long\tTitle' FORMAT PrettyNoEscapes; + +SELECT '你好', '世界' FORMAT Vertical; +SELECT 'Tabs \t Tabs', 'Non-first \t Tabs' FORMAT Vertical; +SELECT 'Control characters \x1f\x1f\x1f\x1f with zero width', 'Invalid UTF-8 which eats pending characters \xf0, and invalid by itself \x80 with zero width' FORMAT Vertical; +SELECT 'Russian ё and ё', 'Zero bytes \0 \0 in middle' FORMAT Vertical; + +DROP TABLE IF EXISTS test.unicode; +DROP DATABASE test; diff --git a/libs/CMakeLists.txt b/libs/CMakeLists.txt index a2fed5f8313..9b047c6948e 100644 --- a/libs/CMakeLists.txt +++ b/libs/CMakeLists.txt @@ -27,3 +27,4 @@ if (USE_INTERNAL_CONSISTENT_HASHING_LIBRARY) add_subdirectory (consistent-hashing) endif () add_subdirectory (consistent-hashing-sumbur) +add_subdirectory (libwidechar_width) diff --git a/libs/libwidechar_width/CMakeLists.txt b/libs/libwidechar_width/CMakeLists.txt new file mode 100644 index 00000000000..724498ac9c7 --- /dev/null +++ b/libs/libwidechar_width/CMakeLists.txt @@ -0,0 +1,2 @@ +add_library(widechar_width widechar_width.cpp) +target_include_directories(widechar_width PUBLIC ${CMAKE_CURRENT_SOURCE_DIR}) diff --git a/libs/libwidechar_width/LICENSE b/libs/libwidechar_width/LICENSE new file mode 100644 index 00000000000..d3b1dd767e2 --- /dev/null +++ b/libs/libwidechar_width/LICENSE @@ -0,0 +1,4 @@ +widecharwidth - wcwidth implementation +Written in 2018 by ridiculous_fish +To the extent possible under law, the author(s) have dedicated all copyright and related and neighboring rights to this software to the public domain worldwide. This software is distributed without any warranty. +You should have received a copy of the CC0 Public Domain Dedication along with this software. If not, see . diff --git a/libs/libwidechar_width/README.md b/libs/libwidechar_width/README.md new file mode 100644 index 00000000000..a9039987f98 --- /dev/null +++ b/libs/libwidechar_width/README.md @@ -0,0 +1,30 @@ +url: https://github.com/ridiculousfish/widecharwidth +commit: 62edda3d435392bf84c7db59f01f53370bbb1715 + +widecharwidth is a Python script that outputs a C++11 header-only implementation of `wcwidth()`, by downloading and parsing the latest `UnicodeData.txt`, `EastAsianWidth.txt`, and `emoji-data.txt`. + +## Usage + +You may directly copy and use the included `widechar_width.h`. + +This header contains a single public function `widechar_wcwidth()`. This returns either a positive width value (1 or 2), or a negative value such as `widechar_private_use`. Note that there are several possible negative return values, to distinguish among different scenarios. + +If you aren't sure how to handle negative return values, try this table: + + | return value | width | + |-------------------------|---| + | `widechar_nonprint` | 0 | + | `widechar_combining` | 0 | + | `widechar_ambiguous` | 1 | + | `widechar_private_use` | 1 | + | `widechar_unassigned` | 0 | + | `widechar_widened_in_9` | 2 (or maybe 1, renderer dependent) | + +## Regenerating the header + +To regenerate the header, run `make`. This will download and parse the relevant files, and run tests. + +## License + +widecharwidth and its output headers are released into the public domain. They may be used for any purpose without requiring attribution, or under the CC0 license if public domain is not available. See included LICENSE. + diff --git a/libs/libwidechar_width/widechar_width.cpp b/libs/libwidechar_width/widechar_width.cpp new file mode 100644 index 00000000000..9825f7af920 --- /dev/null +++ b/libs/libwidechar_width/widechar_width.cpp @@ -0,0 +1 @@ +#include "widechar_width.h" diff --git a/libs/libwidechar_width/widechar_width.h b/libs/libwidechar_width/widechar_width.h new file mode 100644 index 00000000000..9465fdc5f97 --- /dev/null +++ b/libs/libwidechar_width/widechar_width.h @@ -0,0 +1,521 @@ +/** + * widechar_width.h, generated on 2018-07-09. + * See https://github.com/ridiculousfish/widecharwidth/ + * + * SHA1 file hashes: + * UnicodeData.txt: 0e060fafb08d6722fbec56d9f9ebe8509f01d0ee + * EastAsianWidth.txt: 240b5b2c235671d330860742615f798e3d334c4c + * emoji-data.txt: 11fd60a01e17df80035c459728350073cd9ed37b + */ + +#ifndef WIDECHAR_WIDTH_H +#define WIDECHAR_WIDTH_H + +#include +#include +#include +#include + +/* Special width values */ +enum { + widechar_nonprint = -1, // The character is not printable. + widechar_combining = -2, // The character is a zero-width combiner. + widechar_ambiguous = -3, // The character is East-Asian ambiguous width. + widechar_private_use = -4, // The character is for private use. + widechar_unassigned = -5, // The character is unassigned. + widechar_widened_in_9 = -6 // Width is 1 in Unicode 8, 2 in Unicode 9+. +}; + +/* An inclusive range of characters. */ +struct widechar_range { + int32_t lo; + int32_t hi; +}; + +/* Private usage range. */ +static const struct widechar_range widechar_private_table[] = { + {0x0E000, 0x0F8FF}, {0xF0000, 0xFFFFD}, {0x100000, 0x10FFFD} +}; + +/* Nonprinting characters. */ +static const struct widechar_range widechar_nonprint_table[] = { + {0x00000, 0x0001F}, {0x0007F, 0x0009F}, {0x000AD, 0x000AD}, + {0x00600, 0x00605}, {0x0061C, 0x0061C}, {0x006DD, 0x006DD}, + {0x0070F, 0x0070F}, {0x008E2, 0x008E2}, {0x0180E, 0x0180E}, + {0x0200B, 0x0200F}, {0x02028, 0x0202E}, {0x02060, 0x02064}, + {0x02066, 0x0206F}, {0x0D800, 0x0DFFF}, {0x0FEFF, 0x0FEFF}, + {0x0FFF9, 0x0FFFB}, {0x110BD, 0x110BD}, {0x110CD, 0x110CD}, + {0x1BCA0, 0x1BCA3}, {0x1D173, 0x1D17A}, {0xE0001, 0xE0001}, + {0xE0020, 0xE007F} +}; + +/* Width 0 combining marks. */ +static const struct widechar_range widechar_combining_table[] = { + {0x00300, 0x0036F}, {0x00483, 0x00489}, {0x00591, 0x005BD}, + {0x005BF, 0x005BF}, {0x005C1, 0x005C2}, {0x005C4, 0x005C5}, + {0x005C7, 0x005C7}, {0x00610, 0x0061A}, {0x0064B, 0x0065F}, + {0x00670, 0x00670}, {0x006D6, 0x006DC}, {0x006DF, 0x006E4}, + {0x006E7, 0x006E8}, {0x006EA, 0x006ED}, {0x00711, 0x00711}, + {0x00730, 0x0074A}, {0x007A6, 0x007B0}, {0x007EB, 0x007F3}, + {0x007FD, 0x007FD}, {0x00816, 0x00819}, {0x0081B, 0x00823}, + {0x00825, 0x00827}, {0x00829, 0x0082D}, {0x00859, 0x0085B}, + {0x008D3, 0x008E1}, {0x008E3, 0x00903}, {0x0093A, 0x0093C}, + {0x0093E, 0x0094F}, {0x00951, 0x00957}, {0x00962, 0x00963}, + {0x00981, 0x00983}, {0x009BC, 0x009BC}, {0x009BE, 0x009C4}, + {0x009C7, 0x009C8}, {0x009CB, 0x009CD}, {0x009D7, 0x009D7}, + {0x009E2, 0x009E3}, {0x009FE, 0x009FE}, {0x00A01, 0x00A03}, + {0x00A3C, 0x00A3C}, {0x00A3E, 0x00A42}, {0x00A47, 0x00A48}, + {0x00A4B, 0x00A4D}, {0x00A51, 0x00A51}, {0x00A70, 0x00A71}, + {0x00A75, 0x00A75}, {0x00A81, 0x00A83}, {0x00ABC, 0x00ABC}, + {0x00ABE, 0x00AC5}, {0x00AC7, 0x00AC9}, {0x00ACB, 0x00ACD}, + {0x00AE2, 0x00AE3}, {0x00AFA, 0x00AFF}, {0x00B01, 0x00B03}, + {0x00B3C, 0x00B3C}, {0x00B3E, 0x00B44}, {0x00B47, 0x00B48}, + {0x00B4B, 0x00B4D}, {0x00B56, 0x00B57}, {0x00B62, 0x00B63}, + {0x00B82, 0x00B82}, {0x00BBE, 0x00BC2}, {0x00BC6, 0x00BC8}, + {0x00BCA, 0x00BCD}, {0x00BD7, 0x00BD7}, {0x00C00, 0x00C04}, + {0x00C3E, 0x00C44}, {0x00C46, 0x00C48}, {0x00C4A, 0x00C4D}, + {0x00C55, 0x00C56}, {0x00C62, 0x00C63}, {0x00C81, 0x00C83}, + {0x00CBC, 0x00CBC}, {0x00CBE, 0x00CC4}, {0x00CC6, 0x00CC8}, + {0x00CCA, 0x00CCD}, {0x00CD5, 0x00CD6}, {0x00CE2, 0x00CE3}, + {0x00D00, 0x00D03}, {0x00D3B, 0x00D3C}, {0x00D3E, 0x00D44}, + {0x00D46, 0x00D48}, {0x00D4A, 0x00D4D}, {0x00D57, 0x00D57}, + {0x00D62, 0x00D63}, {0x00D82, 0x00D83}, {0x00DCA, 0x00DCA}, + {0x00DCF, 0x00DD4}, {0x00DD6, 0x00DD6}, {0x00DD8, 0x00DDF}, + {0x00DF2, 0x00DF3}, {0x00E31, 0x00E31}, {0x00E34, 0x00E3A}, + {0x00E47, 0x00E4E}, {0x00EB1, 0x00EB1}, {0x00EB4, 0x00EB9}, + {0x00EBB, 0x00EBC}, {0x00EC8, 0x00ECD}, {0x00F18, 0x00F19}, + {0x00F35, 0x00F35}, {0x00F37, 0x00F37}, {0x00F39, 0x00F39}, + {0x00F3E, 0x00F3F}, {0x00F71, 0x00F84}, {0x00F86, 0x00F87}, + {0x00F8D, 0x00F97}, {0x00F99, 0x00FBC}, {0x00FC6, 0x00FC6}, + {0x0102B, 0x0103E}, {0x01056, 0x01059}, {0x0105E, 0x01060}, + {0x01062, 0x01064}, {0x01067, 0x0106D}, {0x01071, 0x01074}, + {0x01082, 0x0108D}, {0x0108F, 0x0108F}, {0x0109A, 0x0109D}, + {0x0135D, 0x0135F}, {0x01712, 0x01714}, {0x01732, 0x01734}, + {0x01752, 0x01753}, {0x01772, 0x01773}, {0x017B4, 0x017D3}, + {0x017DD, 0x017DD}, {0x0180B, 0x0180D}, {0x01885, 0x01886}, + {0x018A9, 0x018A9}, {0x01920, 0x0192B}, {0x01930, 0x0193B}, + {0x01A17, 0x01A1B}, {0x01A55, 0x01A5E}, {0x01A60, 0x01A7C}, + {0x01A7F, 0x01A7F}, {0x01AB0, 0x01ABE}, {0x01B00, 0x01B04}, + {0x01B34, 0x01B44}, {0x01B6B, 0x01B73}, {0x01B80, 0x01B82}, + {0x01BA1, 0x01BAD}, {0x01BE6, 0x01BF3}, {0x01C24, 0x01C37}, + {0x01CD0, 0x01CD2}, {0x01CD4, 0x01CE8}, {0x01CED, 0x01CED}, + {0x01CF2, 0x01CF4}, {0x01CF7, 0x01CF9}, {0x01DC0, 0x01DF9}, + {0x01DFB, 0x01DFF}, {0x020D0, 0x020F0}, {0x02CEF, 0x02CF1}, + {0x02D7F, 0x02D7F}, {0x02DE0, 0x02DFF}, {0x0302A, 0x0302F}, + {0x03099, 0x0309A}, {0x0A66F, 0x0A672}, {0x0A674, 0x0A67D}, + {0x0A69E, 0x0A69F}, {0x0A6F0, 0x0A6F1}, {0x0A802, 0x0A802}, + {0x0A806, 0x0A806}, {0x0A80B, 0x0A80B}, {0x0A823, 0x0A827}, + {0x0A880, 0x0A881}, {0x0A8B4, 0x0A8C5}, {0x0A8E0, 0x0A8F1}, + {0x0A8FF, 0x0A8FF}, {0x0A926, 0x0A92D}, {0x0A947, 0x0A953}, + {0x0A980, 0x0A983}, {0x0A9B3, 0x0A9C0}, {0x0A9E5, 0x0A9E5}, + {0x0AA29, 0x0AA36}, {0x0AA43, 0x0AA43}, {0x0AA4C, 0x0AA4D}, + {0x0AA7B, 0x0AA7D}, {0x0AAB0, 0x0AAB0}, {0x0AAB2, 0x0AAB4}, + {0x0AAB7, 0x0AAB8}, {0x0AABE, 0x0AABF}, {0x0AAC1, 0x0AAC1}, + {0x0AAEB, 0x0AAEF}, {0x0AAF5, 0x0AAF6}, {0x0ABE3, 0x0ABEA}, + {0x0ABEC, 0x0ABED}, {0x0FB1E, 0x0FB1E}, {0x0FE00, 0x0FE0F}, + {0x0FE20, 0x0FE2F}, {0x101FD, 0x101FD}, {0x102E0, 0x102E0}, + {0x10376, 0x1037A}, {0x10A01, 0x10A03}, {0x10A05, 0x10A06}, + {0x10A0C, 0x10A0F}, {0x10A38, 0x10A3A}, {0x10A3F, 0x10A3F}, + {0x10AE5, 0x10AE6}, {0x10D24, 0x10D27}, {0x10F46, 0x10F50}, + {0x11000, 0x11002}, {0x11038, 0x11046}, {0x1107F, 0x11082}, + {0x110B0, 0x110BA}, {0x11100, 0x11102}, {0x11127, 0x11134}, + {0x11145, 0x11146}, {0x11173, 0x11173}, {0x11180, 0x11182}, + {0x111B3, 0x111C0}, {0x111C9, 0x111CC}, {0x1122C, 0x11237}, + {0x1123E, 0x1123E}, {0x112DF, 0x112EA}, {0x11300, 0x11303}, + {0x1133B, 0x1133C}, {0x1133E, 0x11344}, {0x11347, 0x11348}, + {0x1134B, 0x1134D}, {0x11357, 0x11357}, {0x11362, 0x11363}, + {0x11366, 0x1136C}, {0x11370, 0x11374}, {0x11435, 0x11446}, + {0x1145E, 0x1145E}, {0x114B0, 0x114C3}, {0x115AF, 0x115B5}, + {0x115B8, 0x115C0}, {0x115DC, 0x115DD}, {0x11630, 0x11640}, + {0x116AB, 0x116B7}, {0x1171D, 0x1172B}, {0x1182C, 0x1183A}, + {0x11A01, 0x11A0A}, {0x11A33, 0x11A39}, {0x11A3B, 0x11A3E}, + {0x11A47, 0x11A47}, {0x11A51, 0x11A5B}, {0x11A8A, 0x11A99}, + {0x11C2F, 0x11C36}, {0x11C38, 0x11C3F}, {0x11C92, 0x11CA7}, + {0x11CA9, 0x11CB6}, {0x11D31, 0x11D36}, {0x11D3A, 0x11D3A}, + {0x11D3C, 0x11D3D}, {0x11D3F, 0x11D45}, {0x11D47, 0x11D47}, + {0x11D8A, 0x11D8E}, {0x11D90, 0x11D91}, {0x11D93, 0x11D97}, + {0x11EF3, 0x11EF6}, {0x16AF0, 0x16AF4}, {0x16B30, 0x16B36}, + {0x16F51, 0x16F7E}, {0x16F8F, 0x16F92}, {0x1BC9D, 0x1BC9E}, + {0x1D165, 0x1D169}, {0x1D16D, 0x1D172}, {0x1D17B, 0x1D182}, + {0x1D185, 0x1D18B}, {0x1D1AA, 0x1D1AD}, {0x1D242, 0x1D244}, + {0x1DA00, 0x1DA36}, {0x1DA3B, 0x1DA6C}, {0x1DA75, 0x1DA75}, + {0x1DA84, 0x1DA84}, {0x1DA9B, 0x1DA9F}, {0x1DAA1, 0x1DAAF}, + {0x1E000, 0x1E006}, {0x1E008, 0x1E018}, {0x1E01B, 0x1E021}, + {0x1E023, 0x1E024}, {0x1E026, 0x1E02A}, {0x1E8D0, 0x1E8D6}, + {0x1E944, 0x1E94A}, {0xE0100, 0xE01EF} +}; + +/* Width.2 characters. */ +static const struct widechar_range widechar_doublewide_table[] = { + {0x01100, 0x0115F}, {0x0231A, 0x0231B}, {0x02329, 0x0232A}, + {0x023E9, 0x023EC}, {0x023F0, 0x023F0}, {0x023F3, 0x023F3}, + {0x025FD, 0x025FE}, {0x02614, 0x02615}, {0x02648, 0x02653}, + {0x0267F, 0x0267F}, {0x02693, 0x02693}, {0x026A1, 0x026A1}, + {0x026AA, 0x026AB}, {0x026BD, 0x026BE}, {0x026C4, 0x026C5}, + {0x026CE, 0x026CE}, {0x026D4, 0x026D4}, {0x026EA, 0x026EA}, + {0x026F2, 0x026F3}, {0x026F5, 0x026F5}, {0x026FA, 0x026FA}, + {0x026FD, 0x026FD}, {0x02705, 0x02705}, {0x0270A, 0x0270B}, + {0x02728, 0x02728}, {0x0274C, 0x0274C}, {0x0274E, 0x0274E}, + {0x02753, 0x02755}, {0x02757, 0x02757}, {0x02795, 0x02797}, + {0x027B0, 0x027B0}, {0x027BF, 0x027BF}, {0x02B1B, 0x02B1C}, + {0x02B50, 0x02B50}, {0x02B55, 0x02B55}, {0x02E80, 0x02E99}, + {0x02E9B, 0x02EF3}, {0x02F00, 0x02FD5}, {0x02FF0, 0x02FFB}, + {0x03000, 0x0303E}, {0x03041, 0x03096}, {0x03099, 0x030FF}, + {0x03105, 0x0312F}, {0x03131, 0x0318E}, {0x03190, 0x031BA}, + {0x031C0, 0x031E3}, {0x031F0, 0x0321E}, {0x03220, 0x03247}, + {0x03250, 0x032FE}, {0x03300, 0x04DBF}, {0x04E00, 0x0A48C}, + {0x0A490, 0x0A4C6}, {0x0A960, 0x0A97C}, {0x0AC00, 0x0D7A3}, + {0x0F900, 0x0FAFF}, {0x0FE10, 0x0FE19}, {0x0FE30, 0x0FE52}, + {0x0FE54, 0x0FE66}, {0x0FE68, 0x0FE6B}, {0x0FF01, 0x0FF60}, + {0x0FFE0, 0x0FFE6}, {0x16FE0, 0x16FE1}, {0x17000, 0x187F1}, + {0x18800, 0x18AF2}, {0x1B000, 0x1B11E}, {0x1B170, 0x1B2FB}, + {0x1F200, 0x1F200}, {0x1F210, 0x1F219}, {0x1F21B, 0x1F22E}, + {0x1F230, 0x1F231}, {0x1F23B, 0x1F23B}, {0x1F240, 0x1F248}, + {0x1F260, 0x1F265}, {0x1F57A, 0x1F57A}, {0x1F5A4, 0x1F5A4}, + {0x1F6D1, 0x1F6D2}, {0x1F6F4, 0x1F6F9}, {0x1F919, 0x1F93E}, + {0x1F940, 0x1F970}, {0x1F973, 0x1F976}, {0x1F97A, 0x1F97A}, + {0x1F97C, 0x1F97F}, {0x1F985, 0x1F9A2}, {0x1F9B0, 0x1F9B9}, + {0x1F9C1, 0x1F9C2}, {0x1F9D0, 0x1F9FF}, {0x20000, 0x2FFFD}, + {0x30000, 0x3FFFD} +}; + +/* Ambiguous-width characters. */ +static const struct widechar_range widechar_ambiguous_table[] = { + {0x000A1, 0x000A1}, {0x000A4, 0x000A4}, {0x000A7, 0x000A8}, + {0x000AA, 0x000AA}, {0x000AD, 0x000AE}, {0x000B0, 0x000B4}, + {0x000B6, 0x000BA}, {0x000BC, 0x000BF}, {0x000C6, 0x000C6}, + {0x000D0, 0x000D0}, {0x000D7, 0x000D8}, {0x000DE, 0x000E1}, + {0x000E6, 0x000E6}, {0x000E8, 0x000EA}, {0x000EC, 0x000ED}, + {0x000F0, 0x000F0}, {0x000F2, 0x000F3}, {0x000F7, 0x000FA}, + {0x000FC, 0x000FC}, {0x000FE, 0x000FE}, {0x00101, 0x00101}, + {0x00111, 0x00111}, {0x00113, 0x00113}, {0x0011B, 0x0011B}, + {0x00126, 0x00127}, {0x0012B, 0x0012B}, {0x00131, 0x00133}, + {0x00138, 0x00138}, {0x0013F, 0x00142}, {0x00144, 0x00144}, + {0x00148, 0x0014B}, {0x0014D, 0x0014D}, {0x00152, 0x00153}, + {0x00166, 0x00167}, {0x0016B, 0x0016B}, {0x001CE, 0x001CE}, + {0x001D0, 0x001D0}, {0x001D2, 0x001D2}, {0x001D4, 0x001D4}, + {0x001D6, 0x001D6}, {0x001D8, 0x001D8}, {0x001DA, 0x001DA}, + {0x001DC, 0x001DC}, {0x00251, 0x00251}, {0x00261, 0x00261}, + {0x002C4, 0x002C4}, {0x002C7, 0x002C7}, {0x002C9, 0x002CB}, + {0x002CD, 0x002CD}, {0x002D0, 0x002D0}, {0x002D8, 0x002DB}, + {0x002DD, 0x002DD}, {0x002DF, 0x002DF}, {0x00300, 0x0036F}, + {0x00391, 0x003A1}, {0x003A3, 0x003A9}, {0x003B1, 0x003C1}, + {0x003C3, 0x003C9}, {0x00401, 0x00401}, {0x00410, 0x0044F}, + {0x00451, 0x00451}, {0x02010, 0x02010}, {0x02013, 0x02016}, + {0x02018, 0x02019}, {0x0201C, 0x0201D}, {0x02020, 0x02022}, + {0x02024, 0x02027}, {0x02030, 0x02030}, {0x02032, 0x02033}, + {0x02035, 0x02035}, {0x0203B, 0x0203B}, {0x0203E, 0x0203E}, + {0x02074, 0x02074}, {0x0207F, 0x0207F}, {0x02081, 0x02084}, + {0x020AC, 0x020AC}, {0x02103, 0x02103}, {0x02105, 0x02105}, + {0x02109, 0x02109}, {0x02113, 0x02113}, {0x02116, 0x02116}, + {0x02121, 0x02122}, {0x02126, 0x02126}, {0x0212B, 0x0212B}, + {0x02153, 0x02154}, {0x0215B, 0x0215E}, {0x02160, 0x0216B}, + {0x02170, 0x02179}, {0x02189, 0x02189}, {0x02190, 0x02199}, + {0x021B8, 0x021B9}, {0x021D2, 0x021D2}, {0x021D4, 0x021D4}, + {0x021E7, 0x021E7}, {0x02200, 0x02200}, {0x02202, 0x02203}, + {0x02207, 0x02208}, {0x0220B, 0x0220B}, {0x0220F, 0x0220F}, + {0x02211, 0x02211}, {0x02215, 0x02215}, {0x0221A, 0x0221A}, + {0x0221D, 0x02220}, {0x02223, 0x02223}, {0x02225, 0x02225}, + {0x02227, 0x0222C}, {0x0222E, 0x0222E}, {0x02234, 0x02237}, + {0x0223C, 0x0223D}, {0x02248, 0x02248}, {0x0224C, 0x0224C}, + {0x02252, 0x02252}, {0x02260, 0x02261}, {0x02264, 0x02267}, + {0x0226A, 0x0226B}, {0x0226E, 0x0226F}, {0x02282, 0x02283}, + {0x02286, 0x02287}, {0x02295, 0x02295}, {0x02299, 0x02299}, + {0x022A5, 0x022A5}, {0x022BF, 0x022BF}, {0x02312, 0x02312}, + {0x02460, 0x024E9}, {0x024EB, 0x0254B}, {0x02550, 0x02573}, + {0x02580, 0x0258F}, {0x02592, 0x02595}, {0x025A0, 0x025A1}, + {0x025A3, 0x025A9}, {0x025B2, 0x025B3}, {0x025B6, 0x025B7}, + {0x025BC, 0x025BD}, {0x025C0, 0x025C1}, {0x025C6, 0x025C8}, + {0x025CB, 0x025CB}, {0x025CE, 0x025D1}, {0x025E2, 0x025E5}, + {0x025EF, 0x025EF}, {0x02605, 0x02606}, {0x02609, 0x02609}, + {0x0260E, 0x0260F}, {0x0261C, 0x0261C}, {0x0261E, 0x0261E}, + {0x02640, 0x02640}, {0x02642, 0x02642}, {0x02660, 0x02661}, + {0x02663, 0x02665}, {0x02667, 0x0266A}, {0x0266C, 0x0266D}, + {0x0266F, 0x0266F}, {0x0269E, 0x0269F}, {0x026BF, 0x026BF}, + {0x026C6, 0x026CD}, {0x026CF, 0x026D3}, {0x026D5, 0x026E1}, + {0x026E3, 0x026E3}, {0x026E8, 0x026E9}, {0x026EB, 0x026F1}, + {0x026F4, 0x026F4}, {0x026F6, 0x026F9}, {0x026FB, 0x026FC}, + {0x026FE, 0x026FF}, {0x0273D, 0x0273D}, {0x02776, 0x0277F}, + {0x02B56, 0x02B59}, {0x03248, 0x0324F}, {0x0E000, 0x0F8FF}, + {0x0FE00, 0x0FE0F}, {0x0FFFD, 0x0FFFD}, {0x1F100, 0x1F10A}, + {0x1F110, 0x1F12D}, {0x1F130, 0x1F169}, {0x1F172, 0x1F17D}, + {0x1F180, 0x1F18D}, {0x1F18F, 0x1F190}, {0x1F19B, 0x1F1AC}, + {0xE0100, 0xE01EF}, {0xF0000, 0xFFFFD}, {0x100000, 0x10FFFD} +}; + + +/* Unassigned characters. */ +static const struct widechar_range widechar_unassigned_table[] = { + {0x00378, 0x00379}, {0x00380, 0x00383}, {0x0038B, 0x0038B}, + {0x0038D, 0x0038D}, {0x003A2, 0x003A2}, {0x00530, 0x00530}, + {0x00557, 0x00558}, {0x0058B, 0x0058C}, {0x00590, 0x00590}, + {0x005C8, 0x005CF}, {0x005EB, 0x005EE}, {0x005F5, 0x005FF}, + {0x0061D, 0x0061D}, {0x0070E, 0x0070E}, {0x0074B, 0x0074C}, + {0x007B2, 0x007BF}, {0x007FB, 0x007FC}, {0x0082E, 0x0082F}, + {0x0083F, 0x0083F}, {0x0085C, 0x0085D}, {0x0085F, 0x0085F}, + {0x0086B, 0x0089F}, {0x008B5, 0x008B5}, {0x008BE, 0x008D2}, + {0x00984, 0x00984}, {0x0098D, 0x0098E}, {0x00991, 0x00992}, + {0x009A9, 0x009A9}, {0x009B1, 0x009B1}, {0x009B3, 0x009B5}, + {0x009BA, 0x009BB}, {0x009C5, 0x009C6}, {0x009C9, 0x009CA}, + {0x009CF, 0x009D6}, {0x009D8, 0x009DB}, {0x009DE, 0x009DE}, + {0x009E4, 0x009E5}, {0x009FF, 0x00A00}, {0x00A04, 0x00A04}, + {0x00A0B, 0x00A0E}, {0x00A11, 0x00A12}, {0x00A29, 0x00A29}, + {0x00A31, 0x00A31}, {0x00A34, 0x00A34}, {0x00A37, 0x00A37}, + {0x00A3A, 0x00A3B}, {0x00A3D, 0x00A3D}, {0x00A43, 0x00A46}, + {0x00A49, 0x00A4A}, {0x00A4E, 0x00A50}, {0x00A52, 0x00A58}, + {0x00A5D, 0x00A5D}, {0x00A5F, 0x00A65}, {0x00A77, 0x00A80}, + {0x00A84, 0x00A84}, {0x00A8E, 0x00A8E}, {0x00A92, 0x00A92}, + {0x00AA9, 0x00AA9}, {0x00AB1, 0x00AB1}, {0x00AB4, 0x00AB4}, + {0x00ABA, 0x00ABB}, {0x00AC6, 0x00AC6}, {0x00ACA, 0x00ACA}, + {0x00ACE, 0x00ACF}, {0x00AD1, 0x00ADF}, {0x00AE4, 0x00AE5}, + {0x00AF2, 0x00AF8}, {0x00B00, 0x00B00}, {0x00B04, 0x00B04}, + {0x00B0D, 0x00B0E}, {0x00B11, 0x00B12}, {0x00B29, 0x00B29}, + {0x00B31, 0x00B31}, {0x00B34, 0x00B34}, {0x00B3A, 0x00B3B}, + {0x00B45, 0x00B46}, {0x00B49, 0x00B4A}, {0x00B4E, 0x00B55}, + {0x00B58, 0x00B5B}, {0x00B5E, 0x00B5E}, {0x00B64, 0x00B65}, + {0x00B78, 0x00B81}, {0x00B84, 0x00B84}, {0x00B8B, 0x00B8D}, + {0x00B91, 0x00B91}, {0x00B96, 0x00B98}, {0x00B9B, 0x00B9B}, + {0x00B9D, 0x00B9D}, {0x00BA0, 0x00BA2}, {0x00BA5, 0x00BA7}, + {0x00BAB, 0x00BAD}, {0x00BBA, 0x00BBD}, {0x00BC3, 0x00BC5}, + {0x00BC9, 0x00BC9}, {0x00BCE, 0x00BCF}, {0x00BD1, 0x00BD6}, + {0x00BD8, 0x00BE5}, {0x00BFB, 0x00BFF}, {0x00C0D, 0x00C0D}, + {0x00C11, 0x00C11}, {0x00C29, 0x00C29}, {0x00C3A, 0x00C3C}, + {0x00C45, 0x00C45}, {0x00C49, 0x00C49}, {0x00C4E, 0x00C54}, + {0x00C57, 0x00C57}, {0x00C5B, 0x00C5F}, {0x00C64, 0x00C65}, + {0x00C70, 0x00C77}, {0x00C8D, 0x00C8D}, {0x00C91, 0x00C91}, + {0x00CA9, 0x00CA9}, {0x00CB4, 0x00CB4}, {0x00CBA, 0x00CBB}, + {0x00CC5, 0x00CC5}, {0x00CC9, 0x00CC9}, {0x00CCE, 0x00CD4}, + {0x00CD7, 0x00CDD}, {0x00CDF, 0x00CDF}, {0x00CE4, 0x00CE5}, + {0x00CF0, 0x00CF0}, {0x00CF3, 0x00CFF}, {0x00D04, 0x00D04}, + {0x00D0D, 0x00D0D}, {0x00D11, 0x00D11}, {0x00D45, 0x00D45}, + {0x00D49, 0x00D49}, {0x00D50, 0x00D53}, {0x00D64, 0x00D65}, + {0x00D80, 0x00D81}, {0x00D84, 0x00D84}, {0x00D97, 0x00D99}, + {0x00DB2, 0x00DB2}, {0x00DBC, 0x00DBC}, {0x00DBE, 0x00DBF}, + {0x00DC7, 0x00DC9}, {0x00DCB, 0x00DCE}, {0x00DD5, 0x00DD5}, + {0x00DD7, 0x00DD7}, {0x00DE0, 0x00DE5}, {0x00DF0, 0x00DF1}, + {0x00DF5, 0x00E00}, {0x00E3B, 0x00E3E}, {0x00E5C, 0x00E80}, + {0x00E83, 0x00E83}, {0x00E85, 0x00E86}, {0x00E89, 0x00E89}, + {0x00E8B, 0x00E8C}, {0x00E8E, 0x00E93}, {0x00E98, 0x00E98}, + {0x00EA0, 0x00EA0}, {0x00EA4, 0x00EA4}, {0x00EA6, 0x00EA6}, + {0x00EA8, 0x00EA9}, {0x00EAC, 0x00EAC}, {0x00EBA, 0x00EBA}, + {0x00EBE, 0x00EBF}, {0x00EC5, 0x00EC5}, {0x00EC7, 0x00EC7}, + {0x00ECE, 0x00ECF}, {0x00EDA, 0x00EDB}, {0x00EE0, 0x00EFF}, + {0x00F48, 0x00F48}, {0x00F6D, 0x00F70}, {0x00F98, 0x00F98}, + {0x00FBD, 0x00FBD}, {0x00FCD, 0x00FCD}, {0x00FDB, 0x00FFF}, + {0x010C6, 0x010C6}, {0x010C8, 0x010CC}, {0x010CE, 0x010CF}, + {0x01249, 0x01249}, {0x0124E, 0x0124F}, {0x01257, 0x01257}, + {0x01259, 0x01259}, {0x0125E, 0x0125F}, {0x01289, 0x01289}, + {0x0128E, 0x0128F}, {0x012B1, 0x012B1}, {0x012B6, 0x012B7}, + {0x012BF, 0x012BF}, {0x012C1, 0x012C1}, {0x012C6, 0x012C7}, + {0x012D7, 0x012D7}, {0x01311, 0x01311}, {0x01316, 0x01317}, + {0x0135B, 0x0135C}, {0x0137D, 0x0137F}, {0x0139A, 0x0139F}, + {0x013F6, 0x013F7}, {0x013FE, 0x013FF}, {0x0169D, 0x0169F}, + {0x016F9, 0x016FF}, {0x0170D, 0x0170D}, {0x01715, 0x0171F}, + {0x01737, 0x0173F}, {0x01754, 0x0175F}, {0x0176D, 0x0176D}, + {0x01771, 0x01771}, {0x01774, 0x0177F}, {0x017DE, 0x017DF}, + {0x017EA, 0x017EF}, {0x017FA, 0x017FF}, {0x0180F, 0x0180F}, + {0x0181A, 0x0181F}, {0x01879, 0x0187F}, {0x018AB, 0x018AF}, + {0x018F6, 0x018FF}, {0x0191F, 0x0191F}, {0x0192C, 0x0192F}, + {0x0193C, 0x0193F}, {0x01941, 0x01943}, {0x0196E, 0x0196F}, + {0x01975, 0x0197F}, {0x019AC, 0x019AF}, {0x019CA, 0x019CF}, + {0x019DB, 0x019DD}, {0x01A1C, 0x01A1D}, {0x01A5F, 0x01A5F}, + {0x01A7D, 0x01A7E}, {0x01A8A, 0x01A8F}, {0x01A9A, 0x01A9F}, + {0x01AAE, 0x01AAF}, {0x01ABF, 0x01AFF}, {0x01B4C, 0x01B4F}, + {0x01B7D, 0x01B7F}, {0x01BF4, 0x01BFB}, {0x01C38, 0x01C3A}, + {0x01C4A, 0x01C4C}, {0x01C89, 0x01C8F}, {0x01CBB, 0x01CBC}, + {0x01CC8, 0x01CCF}, {0x01CFA, 0x01CFF}, {0x01DFA, 0x01DFA}, + {0x01F16, 0x01F17}, {0x01F1E, 0x01F1F}, {0x01F46, 0x01F47}, + {0x01F4E, 0x01F4F}, {0x01F58, 0x01F58}, {0x01F5A, 0x01F5A}, + {0x01F5C, 0x01F5C}, {0x01F5E, 0x01F5E}, {0x01F7E, 0x01F7F}, + {0x01FB5, 0x01FB5}, {0x01FC5, 0x01FC5}, {0x01FD4, 0x01FD5}, + {0x01FDC, 0x01FDC}, {0x01FF0, 0x01FF1}, {0x01FF5, 0x01FF5}, + {0x01FFF, 0x01FFF}, {0x02065, 0x02065}, {0x02072, 0x02073}, + {0x0208F, 0x0208F}, {0x0209D, 0x0209F}, {0x020C0, 0x020CF}, + {0x020F1, 0x020FF}, {0x0218C, 0x0218F}, {0x02427, 0x0243F}, + {0x0244B, 0x0245F}, {0x02B74, 0x02B75}, {0x02B96, 0x02B97}, + {0x02BC9, 0x02BC9}, {0x02BFF, 0x02BFF}, {0x02C2F, 0x02C2F}, + {0x02C5F, 0x02C5F}, {0x02CF4, 0x02CF8}, {0x02D26, 0x02D26}, + {0x02D28, 0x02D2C}, {0x02D2E, 0x02D2F}, {0x02D68, 0x02D6E}, + {0x02D71, 0x02D7E}, {0x02D97, 0x02D9F}, {0x02DA7, 0x02DA7}, + {0x02DAF, 0x02DAF}, {0x02DB7, 0x02DB7}, {0x02DBF, 0x02DBF}, + {0x02DC7, 0x02DC7}, {0x02DCF, 0x02DCF}, {0x02DD7, 0x02DD7}, + {0x02DDF, 0x02DDF}, {0x02E4F, 0x02E7F}, {0x02E9A, 0x02E9A}, + {0x02EF4, 0x02EFF}, {0x02FD6, 0x02FEF}, {0x02FFC, 0x02FFF}, + {0x03040, 0x03040}, {0x03097, 0x03098}, {0x03100, 0x03104}, + {0x03130, 0x03130}, {0x0318F, 0x0318F}, {0x031BB, 0x031BF}, + {0x031E4, 0x031EF}, {0x0321F, 0x0321F}, {0x032FF, 0x032FF}, + {0x03401, 0x04DB4}, {0x04DB6, 0x04DBF}, {0x04E01, 0x09FEE}, + {0x09FF0, 0x09FFF}, {0x0A48D, 0x0A48F}, {0x0A4C7, 0x0A4CF}, + {0x0A62C, 0x0A63F}, {0x0A6F8, 0x0A6FF}, {0x0A7BA, 0x0A7F6}, + {0x0A82C, 0x0A82F}, {0x0A83A, 0x0A83F}, {0x0A878, 0x0A87F}, + {0x0A8C6, 0x0A8CD}, {0x0A8DA, 0x0A8DF}, {0x0A954, 0x0A95E}, + {0x0A97D, 0x0A97F}, {0x0A9CE, 0x0A9CE}, {0x0A9DA, 0x0A9DD}, + {0x0A9FF, 0x0A9FF}, {0x0AA37, 0x0AA3F}, {0x0AA4E, 0x0AA4F}, + {0x0AA5A, 0x0AA5B}, {0x0AAC3, 0x0AADA}, {0x0AAF7, 0x0AB00}, + {0x0AB07, 0x0AB08}, {0x0AB0F, 0x0AB10}, {0x0AB17, 0x0AB1F}, + {0x0AB27, 0x0AB27}, {0x0AB2F, 0x0AB2F}, {0x0AB66, 0x0AB6F}, + {0x0ABEE, 0x0ABEF}, {0x0ABFA, 0x0ABFF}, {0x0AC01, 0x0D7A2}, + {0x0D7A4, 0x0D7AF}, {0x0D7C7, 0x0D7CA}, {0x0D7FC, 0x0D7FF}, + {0x0FA6E, 0x0FA6F}, {0x0FADA, 0x0FAFF}, {0x0FB07, 0x0FB12}, + {0x0FB18, 0x0FB1C}, {0x0FB37, 0x0FB37}, {0x0FB3D, 0x0FB3D}, + {0x0FB3F, 0x0FB3F}, {0x0FB42, 0x0FB42}, {0x0FB45, 0x0FB45}, + {0x0FBC2, 0x0FBD2}, {0x0FD40, 0x0FD4F}, {0x0FD90, 0x0FD91}, + {0x0FDC8, 0x0FDEF}, {0x0FDFE, 0x0FDFF}, {0x0FE1A, 0x0FE1F}, + {0x0FE53, 0x0FE53}, {0x0FE67, 0x0FE67}, {0x0FE6C, 0x0FE6F}, + {0x0FE75, 0x0FE75}, {0x0FEFD, 0x0FEFE}, {0x0FF00, 0x0FF00}, + {0x0FFBF, 0x0FFC1}, {0x0FFC8, 0x0FFC9}, {0x0FFD0, 0x0FFD1}, + {0x0FFD8, 0x0FFD9}, {0x0FFDD, 0x0FFDF}, {0x0FFE7, 0x0FFE7}, + {0x0FFEF, 0x0FFF8}, {0x0FFFE, 0x0FFFF}, {0x1000C, 0x1000C}, + {0x10027, 0x10027}, {0x1003B, 0x1003B}, {0x1003E, 0x1003E}, + {0x1004E, 0x1004F}, {0x1005E, 0x1007F}, {0x100FB, 0x100FF}, + {0x10103, 0x10106}, {0x10134, 0x10136}, {0x1018F, 0x1018F}, + {0x1019C, 0x1019F}, {0x101A1, 0x101CF}, {0x101FE, 0x1027F}, + {0x1029D, 0x1029F}, {0x102D1, 0x102DF}, {0x102FC, 0x102FF}, + {0x10324, 0x1032C}, {0x1034B, 0x1034F}, {0x1037B, 0x1037F}, + {0x1039E, 0x1039E}, {0x103C4, 0x103C7}, {0x103D6, 0x103FF}, + {0x1049E, 0x1049F}, {0x104AA, 0x104AF}, {0x104D4, 0x104D7}, + {0x104FC, 0x104FF}, {0x10528, 0x1052F}, {0x10564, 0x1056E}, + {0x10570, 0x105FF}, {0x10737, 0x1073F}, {0x10756, 0x1075F}, + {0x10768, 0x107FF}, {0x10806, 0x10807}, {0x10809, 0x10809}, + {0x10836, 0x10836}, {0x10839, 0x1083B}, {0x1083D, 0x1083E}, + {0x10856, 0x10856}, {0x1089F, 0x108A6}, {0x108B0, 0x108DF}, + {0x108F3, 0x108F3}, {0x108F6, 0x108FA}, {0x1091C, 0x1091E}, + {0x1093A, 0x1093E}, {0x10940, 0x1097F}, {0x109B8, 0x109BB}, + {0x109D0, 0x109D1}, {0x10A04, 0x10A04}, {0x10A07, 0x10A0B}, + {0x10A14, 0x10A14}, {0x10A18, 0x10A18}, {0x10A36, 0x10A37}, + {0x10A3B, 0x10A3E}, {0x10A49, 0x10A4F}, {0x10A59, 0x10A5F}, + {0x10AA0, 0x10ABF}, {0x10AE7, 0x10AEA}, {0x10AF7, 0x10AFF}, + {0x10B36, 0x10B38}, {0x10B56, 0x10B57}, {0x10B73, 0x10B77}, + {0x10B92, 0x10B98}, {0x10B9D, 0x10BA8}, {0x10BB0, 0x10BFF}, + {0x10C49, 0x10C7F}, {0x10CB3, 0x10CBF}, {0x10CF3, 0x10CF9}, + {0x10D28, 0x10D2F}, {0x10D3A, 0x10E5F}, {0x10E7F, 0x10EFF}, + {0x10F28, 0x10F2F}, {0x10F5A, 0x10FFF}, {0x1104E, 0x11051}, + {0x11070, 0x1107E}, {0x110C2, 0x110CC}, {0x110CE, 0x110CF}, + {0x110E9, 0x110EF}, {0x110FA, 0x110FF}, {0x11135, 0x11135}, + {0x11147, 0x1114F}, {0x11177, 0x1117F}, {0x111CE, 0x111CF}, + {0x111E0, 0x111E0}, {0x111F5, 0x111FF}, {0x11212, 0x11212}, + {0x1123F, 0x1127F}, {0x11287, 0x11287}, {0x11289, 0x11289}, + {0x1128E, 0x1128E}, {0x1129E, 0x1129E}, {0x112AA, 0x112AF}, + {0x112EB, 0x112EF}, {0x112FA, 0x112FF}, {0x11304, 0x11304}, + {0x1130D, 0x1130E}, {0x11311, 0x11312}, {0x11329, 0x11329}, + {0x11331, 0x11331}, {0x11334, 0x11334}, {0x1133A, 0x1133A}, + {0x11345, 0x11346}, {0x11349, 0x1134A}, {0x1134E, 0x1134F}, + {0x11351, 0x11356}, {0x11358, 0x1135C}, {0x11364, 0x11365}, + {0x1136D, 0x1136F}, {0x11375, 0x113FF}, {0x1145A, 0x1145A}, + {0x1145C, 0x1145C}, {0x1145F, 0x1147F}, {0x114C8, 0x114CF}, + {0x114DA, 0x1157F}, {0x115B6, 0x115B7}, {0x115DE, 0x115FF}, + {0x11645, 0x1164F}, {0x1165A, 0x1165F}, {0x1166D, 0x1167F}, + {0x116B8, 0x116BF}, {0x116CA, 0x116FF}, {0x1171B, 0x1171C}, + {0x1172C, 0x1172F}, {0x11740, 0x117FF}, {0x1183C, 0x1189F}, + {0x118F3, 0x118FE}, {0x11900, 0x119FF}, {0x11A48, 0x11A4F}, + {0x11A84, 0x11A85}, {0x11AA3, 0x11ABF}, {0x11AF9, 0x11BFF}, + {0x11C09, 0x11C09}, {0x11C37, 0x11C37}, {0x11C46, 0x11C4F}, + {0x11C6D, 0x11C6F}, {0x11C90, 0x11C91}, {0x11CA8, 0x11CA8}, + {0x11CB7, 0x11CFF}, {0x11D07, 0x11D07}, {0x11D0A, 0x11D0A}, + {0x11D37, 0x11D39}, {0x11D3B, 0x11D3B}, {0x11D3E, 0x11D3E}, + {0x11D48, 0x11D4F}, {0x11D5A, 0x11D5F}, {0x11D66, 0x11D66}, + {0x11D69, 0x11D69}, {0x11D8F, 0x11D8F}, {0x11D92, 0x11D92}, + {0x11D99, 0x11D9F}, {0x11DAA, 0x11EDF}, {0x11EF9, 0x11FFF}, + {0x1239A, 0x123FF}, {0x1246F, 0x1246F}, {0x12475, 0x1247F}, + {0x12544, 0x12FFF}, {0x1342F, 0x143FF}, {0x14647, 0x167FF}, + {0x16A39, 0x16A3F}, {0x16A5F, 0x16A5F}, {0x16A6A, 0x16A6D}, + {0x16A70, 0x16ACF}, {0x16AEE, 0x16AEF}, {0x16AF6, 0x16AFF}, + {0x16B46, 0x16B4F}, {0x16B5A, 0x16B5A}, {0x16B62, 0x16B62}, + {0x16B78, 0x16B7C}, {0x16B90, 0x16E3F}, {0x16E9B, 0x16EFF}, + {0x16F45, 0x16F4F}, {0x16F7F, 0x16F8E}, {0x16FA0, 0x16FDF}, + {0x16FE2, 0x16FFF}, {0x17001, 0x187F0}, {0x187F2, 0x187FF}, + {0x18AF3, 0x1AFFF}, {0x1B11F, 0x1B16F}, {0x1B2FC, 0x1BBFF}, + {0x1BC6B, 0x1BC6F}, {0x1BC7D, 0x1BC7F}, {0x1BC89, 0x1BC8F}, + {0x1BC9A, 0x1BC9B}, {0x1BCA4, 0x1CFFF}, {0x1D0F6, 0x1D0FF}, + {0x1D127, 0x1D128}, {0x1D1E9, 0x1D1FF}, {0x1D246, 0x1D2DF}, + {0x1D2F4, 0x1D2FF}, {0x1D357, 0x1D35F}, {0x1D379, 0x1D3FF}, + {0x1D455, 0x1D455}, {0x1D49D, 0x1D49D}, {0x1D4A0, 0x1D4A1}, + {0x1D4A3, 0x1D4A4}, {0x1D4A7, 0x1D4A8}, {0x1D4AD, 0x1D4AD}, + {0x1D4BA, 0x1D4BA}, {0x1D4BC, 0x1D4BC}, {0x1D4C4, 0x1D4C4}, + {0x1D506, 0x1D506}, {0x1D50B, 0x1D50C}, {0x1D515, 0x1D515}, + {0x1D51D, 0x1D51D}, {0x1D53A, 0x1D53A}, {0x1D53F, 0x1D53F}, + {0x1D545, 0x1D545}, {0x1D547, 0x1D549}, {0x1D551, 0x1D551}, + {0x1D6A6, 0x1D6A7}, {0x1D7CC, 0x1D7CD}, {0x1DA8C, 0x1DA9A}, + {0x1DAA0, 0x1DAA0}, {0x1DAB0, 0x1DFFF}, {0x1E007, 0x1E007}, + {0x1E019, 0x1E01A}, {0x1E022, 0x1E022}, {0x1E025, 0x1E025}, + {0x1E02B, 0x1E7FF}, {0x1E8C5, 0x1E8C6}, {0x1E8D7, 0x1E8FF}, + {0x1E94B, 0x1E94F}, {0x1E95A, 0x1E95D}, {0x1E960, 0x1EC70}, + {0x1ECB5, 0x1EDFF}, {0x1EE04, 0x1EE04}, {0x1EE20, 0x1EE20}, + {0x1EE23, 0x1EE23}, {0x1EE25, 0x1EE26}, {0x1EE28, 0x1EE28}, + {0x1EE33, 0x1EE33}, {0x1EE38, 0x1EE38}, {0x1EE3A, 0x1EE3A}, + {0x1EE3C, 0x1EE41}, {0x1EE43, 0x1EE46}, {0x1EE48, 0x1EE48}, + {0x1EE4A, 0x1EE4A}, {0x1EE4C, 0x1EE4C}, {0x1EE50, 0x1EE50}, + {0x1EE53, 0x1EE53}, {0x1EE55, 0x1EE56}, {0x1EE58, 0x1EE58}, + {0x1EE5A, 0x1EE5A}, {0x1EE5C, 0x1EE5C}, {0x1EE5E, 0x1EE5E}, + {0x1EE60, 0x1EE60}, {0x1EE63, 0x1EE63}, {0x1EE65, 0x1EE66}, + {0x1EE6B, 0x1EE6B}, {0x1EE73, 0x1EE73}, {0x1EE78, 0x1EE78}, + {0x1EE7D, 0x1EE7D}, {0x1EE7F, 0x1EE7F}, {0x1EE8A, 0x1EE8A}, + {0x1EE9C, 0x1EEA0}, {0x1EEA4, 0x1EEA4}, {0x1EEAA, 0x1EEAA}, + {0x1EEBC, 0x1EEEF}, {0x1EEF2, 0x1EFFF}, {0x1F02C, 0x1F02F}, + {0x1F094, 0x1F09F}, {0x1F0AF, 0x1F0B0}, {0x1F0C0, 0x1F0C0}, + {0x1F0D0, 0x1F0D0}, {0x1F0F6, 0x1F0FF}, {0x1F10D, 0x1F10F}, + {0x1F16C, 0x1F16F}, {0x1F1AD, 0x1F1E5}, {0x1F203, 0x1F20F}, + {0x1F23C, 0x1F23F}, {0x1F249, 0x1F24F}, {0x1F252, 0x1F25F}, + {0x1F266, 0x1F2FF}, {0x1F6D5, 0x1F6DF}, {0x1F6ED, 0x1F6EF}, + {0x1F6FA, 0x1F6FF}, {0x1F774, 0x1F77F}, {0x1F7D9, 0x1F7FF}, + {0x1F80C, 0x1F80F}, {0x1F848, 0x1F84F}, {0x1F85A, 0x1F85F}, + {0x1F888, 0x1F88F}, {0x1F8AE, 0x1F8FF}, {0x1F90C, 0x1F90F}, + {0x1F93F, 0x1F93F}, {0x1F971, 0x1F972}, {0x1F977, 0x1F979}, + {0x1F97B, 0x1F97B}, {0x1F9A3, 0x1F9AF}, {0x1F9BA, 0x1F9BF}, + {0x1F9C3, 0x1F9CF}, {0x1FA00, 0x1FA5F}, {0x1FA6E, 0x1FFFF}, + {0x20001, 0x2A6D5}, {0x2A6D7, 0x2A6FF}, {0x2A701, 0x2B733}, + {0x2B735, 0x2B73F}, {0x2B741, 0x2B81C}, {0x2B81E, 0x2B81F}, + {0x2B821, 0x2CEA0}, {0x2CEA2, 0x2CEAF}, {0x2CEB1, 0x2EBDF}, + {0x2EBE1, 0x2F7FF}, {0x2FA1E, 0xE0000}, {0xE0002, 0xE001F}, + {0xE0080, 0xE00FF}, {0xE01F0, 0xEFFFF}, {0xFFFFE, 0xFFFFF}, + {0x10FFFE, 0x110000} +}; + +/* Characters that were widened from with 1 to 2 in Unicode 9. */ +static const struct widechar_range widechar_widened_table[] = { + {0x1F004, 0x1F004}, {0x1F0CF, 0x1F0CF}, {0x1F170, 0x1F171}, + {0x1F17E, 0x1F17F}, {0x1F18E, 0x1F18E}, {0x1F191, 0x1F19A}, + {0x1F1E6, 0x1F1FF}, {0x1F201, 0x1F202}, {0x1F21A, 0x1F21A}, + {0x1F22F, 0x1F22F}, {0x1F232, 0x1F23A}, {0x1F250, 0x1F251}, + {0x1F300, 0x1F321}, {0x1F324, 0x1F393}, {0x1F396, 0x1F397}, + {0x1F399, 0x1F39B}, {0x1F39E, 0x1F3F0}, {0x1F3F3, 0x1F3F5}, + {0x1F3F7, 0x1F4FD}, {0x1F4FF, 0x1F53D}, {0x1F549, 0x1F54E}, + {0x1F550, 0x1F567}, {0x1F56F, 0x1F570}, {0x1F573, 0x1F579}, + {0x1F587, 0x1F587}, {0x1F58A, 0x1F58D}, {0x1F590, 0x1F590}, + {0x1F595, 0x1F596}, {0x1F5A5, 0x1F5A5}, {0x1F5A8, 0x1F5A8}, + {0x1F5B1, 0x1F5B2}, {0x1F5BC, 0x1F5BC}, {0x1F5C2, 0x1F5C4}, + {0x1F5D1, 0x1F5D3}, {0x1F5DC, 0x1F5DE}, {0x1F5E1, 0x1F5E1}, + {0x1F5E3, 0x1F5E3}, {0x1F5E8, 0x1F5E8}, {0x1F5EF, 0x1F5EF}, + {0x1F5F3, 0x1F5F3}, {0x1F5FA, 0x1F64F}, {0x1F680, 0x1F6C5}, + {0x1F6CB, 0x1F6D0}, {0x1F6E0, 0x1F6E5}, {0x1F6E9, 0x1F6E9}, + {0x1F6EB, 0x1F6EC}, {0x1F6F0, 0x1F6F0}, {0x1F6F3, 0x1F6F3}, + {0x1F910, 0x1F918}, {0x1F980, 0x1F984}, {0x1F9C0, 0x1F9C0} +}; + +template +bool widechar_in_table(const Collection &arr, int32_t c) { + auto where = std::lower_bound(std::begin(arr), std::end(arr), c, + [](widechar_range p, wchar_t c) { return p.hi < c; }); + return where != std::end(arr) && where->lo <= c; +} + +/* Return the width of character c, or a special negative value. */ +int widechar_wcwidth(wchar_t c) { + if (widechar_in_table(widechar_private_table, c)) + return widechar_private_use; + if (widechar_in_table(widechar_nonprint_table, c)) + return widechar_nonprint; + if (widechar_in_table(widechar_combining_table, c)) + return widechar_combining; + if (widechar_in_table(widechar_doublewide_table, c)) + return 2; + if (widechar_in_table(widechar_ambiguous_table, c)) + return widechar_ambiguous; + if (widechar_in_table(widechar_unassigned_table, c)) + return widechar_unassigned; + if (widechar_in_table(widechar_widened_table, c)) + return widechar_widened_in_9; + return 1; +} + +#endif // WIDECHAR_WIDTH_H From e1817e481496d8ed2c74f78daaeba20879ad9b6c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Oct 2018 16:25:47 +0300 Subject: [PATCH 46/52] Auto version update to [18.14.2] [54409] --- dbms/cmake/version.cmake | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 7f720afa804..26565b8ebec 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54409 CACHE STRING "") set(VERSION_MAJOR 18 CACHE STRING "") set(VERSION_MINOR 14 CACHE STRING "") -set(VERSION_PATCH 1 CACHE STRING "") -set(VERSION_GITHASH 4452a1f49bf9ae93838da62694b7f1b1523470e3 CACHE STRING "") -set(VERSION_DESCRIBE v18.14.1-testing CACHE STRING "") -set(VERSION_STRING 18.14.1 CACHE STRING "") +set(VERSION_PATCH 2 CACHE STRING "") +set(VERSION_GITHASH 147a2a13c256c72b7155e864e6a95024ca9ba31a CACHE STRING "") +set(VERSION_DESCRIBE v18.14.2-testing CACHE STRING "") +set(VERSION_STRING 18.14.2 CACHE STRING "") # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/debian/changelog b/debian/changelog index b2654d91dbd..e96f15c324d 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (18.14.1) unstable; urgency=low +clickhouse (18.14.2) unstable; urgency=low * Modified source code - -- Fri, 28 Sep 2018 14:58:24 +0300 + -- Tue, 02 Oct 2018 16:25:47 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index e2209f79b82..e28420c693e 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.1 +ARG version=18.14.2 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 7bed030ee7c..7863bfc1399 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.1 +ARG version=18.14.2 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index febec335824..81e1cfa4758 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.1 +ARG version=18.14.2 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ From c9880ad9c01f69582ff5ec9c4e78d3d66a85351c Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Oct 2018 16:38:28 +0300 Subject: [PATCH 47/52] Fix join of unjoinable thread --- dbms/src/Interpreters/ExternalLoader.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 55c4e45bc82..caf37df43a5 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -78,7 +78,9 @@ void ExternalLoader::init(bool throw_on_error) ExternalLoader::~ExternalLoader() { destroy.set(); - reloading_thread.join(); + /// It can be partially initialized + if (reloading_thread.joinable()) + reloading_thread.join(); } From de92ecfe3237996267cd5d03891a0eda40808bdf Mon Sep 17 00:00:00 2001 From: Alexander Marshalov <_@marshalov.org> Date: Wed, 3 Oct 2018 06:04:06 +0700 Subject: [PATCH 48/52] Fix broken link in doc --- docs/ru/query_language/dicts/external_dicts.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/query_language/dicts/external_dicts.md b/docs/ru/query_language/dicts/external_dicts.md index 9399e316248..6fdd4d2d266 100644 --- a/docs/ru/query_language/dicts/external_dicts.md +++ b/docs/ru/query_language/dicts/external_dicts.md @@ -44,7 +44,7 @@ ClickHouse: - [Настройка внешнего словаря](external_dicts_dict.md#dicts-external_dicts_dict) - [Хранение словарей в памяти](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) -- [Обновление словарей](external_dicts_dict_lifetime#dicts-external_dicts_dict_lifetime) +- [Обновление словарей](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime) - [Источники внешних словарей](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources) - [Ключ и поля словаря](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure) - [Функции для работы с внешними словарями](../functions/ext_dict_functions.md#ext_dict_functions) From 038169e0a732b1418619ed22f259c17fe94d97b5 Mon Sep 17 00:00:00 2001 From: Andrey Date: Wed, 3 Oct 2018 10:10:28 +0300 Subject: [PATCH 49/52] Update nested.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Change 'как в запроса' on 'как у запроса' --- docs/ru/data_types/nested_data_structures/nested.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/data_types/nested_data_structures/nested.md b/docs/ru/data_types/nested_data_structures/nested.md index b513bc39938..6db4e9e83ab 100644 --- a/docs/ru/data_types/nested_data_structures/nested.md +++ b/docs/ru/data_types/nested_data_structures/nested.md @@ -1,6 +1,6 @@ # Nested(Name1 Type1, Name2 Type2, ...) -Вложенная структура данных - это как будто вложенная таблица. Параметры вложенной структуры данных - имена и типы столбцов, указываются так же, как в запроса CREATE. Каждой строке таблицы может соответствовать произвольное количество строк вложенной структуры данных. +Вложенная структура данных - это как будто вложенная таблица. Параметры вложенной структуры данных - имена и типы столбцов, указываются так же, как у запроса CREATE. Каждой строке таблицы может соответствовать произвольное количество строк вложенной структуры данных. Пример: From 1815c101a0ceba4c32b95793244658abf37ac18c Mon Sep 17 00:00:00 2001 From: Andrey Date: Wed, 3 Oct 2018 12:46:55 +0300 Subject: [PATCH 50/52] Update select.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Change 'Если качестве правой части оператора' on 'Если в качестве правой части оператора' --- docs/ru/query_language/select.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/query_language/select.md b/docs/ru/query_language/select.md index d3e8b8a985b..81537d1573b 100644 --- a/docs/ru/query_language/select.md +++ b/docs/ru/query_language/select.md @@ -793,7 +793,7 @@ SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... Если в качестве правой части оператора указано имя таблицы (например, `UserID IN users`), то это эквивалентно подзапросу `UserID IN (SELECT * FROM users)`. Это используется при работе с внешними данными, отправляемым вместе с запросом. Например, вместе с запросом может быть отправлено множество идентификаторов посетителей, загруженное во временную таблицу users, по которому следует выполнить фильтрацию. -Если качестве правой части оператора, указано имя таблицы, имеющий движок Set (подготовленное множество, постоянно находящееся в оперативке), то множество не будет создаваться заново при каждом запросе. +Если в качестве правой части оператора, указано имя таблицы, имеющий движок Set (подготовленное множество, постоянно находящееся в оперативке), то множество не будет создаваться заново при каждом запросе. В подзапросе может быть указано более одного столбца для фильтрации кортежей. Пример: From 0cd6d095e9c2a411a0029562eddeca1c24952449 Mon Sep 17 00:00:00 2001 From: Andrey Date: Thu, 4 Oct 2018 15:26:49 +0300 Subject: [PATCH 51/52] Update access_rights.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Change 'нет доступа к отдельным ДБ' to 'нет доступа к отдельным БД' --- docs/ru/operations/access_rights.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/access_rights.md b/docs/ru/operations/access_rights.md index 027b2fb9e8a..7f09a917c7a 100644 --- a/docs/ru/operations/access_rights.md +++ b/docs/ru/operations/access_rights.md @@ -95,6 +95,6 @@ Доступ к БД `system` всегда считается разрешённым (так как эта БД используется для выполнения запросов). -Пользователь может получить список всех БД и таблиц в них с помощью запросов `SHOW` или системных таблиц, даже если у него нет доступа к отдельным ДБ. +Пользователь может получить список всех БД и таблиц в них с помощью запросов `SHOW` или системных таблиц, даже если у него нет доступа к отдельным БД. Доступ к БД не связан с настройкой [readonly](settings/query_complexity.md#query_complexity_readonly). Невозможно дать полный доступ к одной БД и `readonly` к другой. From b131c8dd668ced9ea3a0f38a4f579a6111db2d4b Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 4 Oct 2018 18:51:52 +0300 Subject: [PATCH 52/52] Remove Paris link --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index b555abdcc1b..dec43532bc1 100644 --- a/README.md +++ b/README.md @@ -11,5 +11,4 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Meetups -* [Paris on October 2](http://bit.ly/clickhouse-paris-2-october-2018) * [Beijing on October 28](http://www.clickhouse.com.cn/topic/5ba0e3f99d28dfde2ddc62a1)