Merge branch 'postgresql-protocol-with-row-policy' of github.com:kssenii/ClickHouse into postgresql-protocol-with-row-policy

This commit is contained in:
kssenii 2021-04-12 18:42:15 +00:00
commit 1251b3260c
62 changed files with 854 additions and 363 deletions

View File

@ -191,8 +191,9 @@ dynamicConfigFile=/etc/zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }}/conf/zoo.
Java version:
``` text
Java(TM) SE Runtime Environment (build 1.8.0_25-b17)
Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode)
openjdk 11.0.5-shenandoah 2019-10-15
OpenJDK Runtime Environment (build 11.0.5-shenandoah+10-adhoc.heretic.src)
OpenJDK 64-Bit Server VM (build 11.0.5-shenandoah+10-adhoc.heretic.src, mixed mode)
```
JVM parameters:
@ -204,7 +205,7 @@ ZOOCFGDIR=/etc/$NAME/conf
# TODO this is really ugly
# How to find out, which jars are needed?
# seems, that log4j requires the log4j.properties file to be in the classpath
CLASSPATH="$ZOOCFGDIR:/usr/build/classes:/usr/build/lib/*.jar:/usr/share/zookeeper/zookeeper-3.5.1-metrika.jar:/usr/share/zookeeper/slf4j-log4j12-1.7.5.jar:/usr/share/zookeeper/slf4j-api-1.7.5.jar:/usr/share/zookeeper/servlet-api-2.5-20081211.jar:/usr/share/zookeeper/netty-3.7.0.Final.jar:/usr/share/zookeeper/log4j-1.2.16.jar:/usr/share/zookeeper/jline-2.11.jar:/usr/share/zookeeper/jetty-util-6.1.26.jar:/usr/share/zookeeper/jetty-6.1.26.jar:/usr/share/zookeeper/javacc.jar:/usr/share/zookeeper/jackson-mapper-asl-1.9.11.jar:/usr/share/zookeeper/jackson-core-asl-1.9.11.jar:/usr/share/zookeeper/commons-cli-1.2.jar:/usr/src/java/lib/*.jar:/usr/etc/zookeeper"
CLASSPATH="$ZOOCFGDIR:/usr/build/classes:/usr/build/lib/*.jar:/usr/share/zookeeper-3.6.2/lib/audience-annotations-0.5.0.jar:/usr/share/zookeeper-3.6.2/lib/commons-cli-1.2.jar:/usr/share/zookeeper-3.6.2/lib/commons-lang-2.6.jar:/usr/share/zookeeper-3.6.2/lib/jackson-annotations-2.10.3.jar:/usr/share/zookeeper-3.6.2/lib/jackson-core-2.10.3.jar:/usr/share/zookeeper-3.6.2/lib/jackson-databind-2.10.3.jar:/usr/share/zookeeper-3.6.2/lib/javax.servlet-api-3.1.0.jar:/usr/share/zookeeper-3.6.2/lib/jetty-http-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-io-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-security-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-server-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-servlet-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jetty-util-9.4.24.v20191120.jar:/usr/share/zookeeper-3.6.2/lib/jline-2.14.6.jar:/usr/share/zookeeper-3.6.2/lib/json-simple-1.1.1.jar:/usr/share/zookeeper-3.6.2/lib/log4j-1.2.17.jar:/usr/share/zookeeper-3.6.2/lib/metrics-core-3.2.5.jar:/usr/share/zookeeper-3.6.2/lib/netty-buffer-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-codec-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-common-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-handler-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-resolver-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-transport-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-transport-native-epoll-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/netty-transport-native-unix-common-4.1.50.Final.jar:/usr/share/zookeeper-3.6.2/lib/simpleclient-0.6.0.jar:/usr/share/zookeeper-3.6.2/lib/simpleclient_common-0.6.0.jar:/usr/share/zookeeper-3.6.2/lib/simpleclient_hotspot-0.6.0.jar:/usr/share/zookeeper-3.6.2/lib/simpleclient_servlet-0.6.0.jar:/usr/share/zookeeper-3.6.2/lib/slf4j-api-1.7.25.jar:/usr/share/zookeeper-3.6.2/lib/slf4j-log4j12-1.7.25.jar:/usr/share/zookeeper-3.6.2/lib/snappy-java-1.1.7.jar:/usr/share/zookeeper-3.6.2/lib/zookeeper-3.6.2.jar:/usr/share/zookeeper-3.6.2/lib/zookeeper-jute-3.6.2.jar:/usr/share/zookeeper-3.6.2/lib/zookeeper-prometheus-metrics-3.6.2.jar:/usr/share/zookeeper-3.6.2/etc"
ZOOCFG="$ZOOCFGDIR/zoo.cfg"
ZOO_LOG_DIR=/var/log/$NAME
@ -213,27 +214,17 @@ GROUP=zookeeper
PIDDIR=/var/run/$NAME
PIDFILE=$PIDDIR/$NAME.pid
SCRIPTNAME=/etc/init.d/$NAME
JAVA=/usr/bin/java
JAVA=/usr/local/jdk-11/bin/java
ZOOMAIN="org.apache.zookeeper.server.quorum.QuorumPeerMain"
ZOO_LOG4J_PROP="INFO,ROLLINGFILE"
JMXLOCALONLY=false
JAVA_OPTS="-Xms{{ '{{' }} cluster.get('xms','128M') {{ '}}' }} \
-Xmx{{ '{{' }} cluster.get('xmx','1G') {{ '}}' }} \
-Xloggc:/var/log/$NAME/zookeeper-gc.log \
-XX:+UseGCLogFileRotation \
-XX:NumberOfGCLogFiles=16 \
-XX:GCLogFileSize=16M \
-Xlog:safepoint,gc*=info,age*=debug:file=/var/log/$NAME/zookeeper-gc.log:time,level,tags:filecount=16,filesize=16M
-verbose:gc \
-XX:+PrintGCTimeStamps \
-XX:+PrintGCDateStamps \
-XX:+PrintGCDetails
-XX:+PrintTenuringDistribution \
-XX:+PrintGCApplicationStoppedTime \
-XX:+PrintGCApplicationConcurrentTime \
-XX:+PrintSafepointStatistics \
-XX:+UseParNewGC \
-XX:+UseConcMarkSweepGC \
-XX:+CMSParallelRemarkEnabled"
-XX:+UseG1GC \
-Djute.maxbuffer=8388608 \
-XX:MaxGCPauseMillis=50"
```
Salt init:

View File

@ -147,6 +147,9 @@ Result:
└────────────────┘
```
!!! attention "Attention"
The return type `toStartOf*` functions described below is `Date` or `DateTime`. Though these functions can take `DateTime64` as an argument, passing them a `DateTime64` that is out of normal range (years 1970 - 2105) will give incorrect result.
## toStartOfYear {#tostartofyear}
Rounds down a date or date with time to the first day of the year.
@ -388,13 +391,13 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d
Truncates date and time data to the specified part of date.
**Syntax**
**Syntax**
``` sql
date_trunc(unit, value[, timezone])
```
Alias: `dateTrunc`.
Alias: `dateTrunc`.
**Arguments**
@ -457,13 +460,13 @@ Result:
Adds the time interval or date interval to the provided date or date with time.
**Syntax**
**Syntax**
``` sql
date_add(unit, value, date)
```
Aliases: `dateAdd`, `DATE_ADD`.
Aliases: `dateAdd`, `DATE_ADD`.
**Arguments**
@ -478,7 +481,7 @@ Aliases: `dateAdd`, `DATE_ADD`.
- `month`
- `quarter`
- `year`
- `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md).
- `date` — The date or date with time to which `value` is added. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).
@ -583,7 +586,7 @@ Aliases: `dateSub`, `DATE_SUB`.
- `month`
- `quarter`
- `year`
- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md).
- `date` — The date or date with time from which `value` is subtracted. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).
@ -613,16 +616,16 @@ Result:
Adds the specified time value with the provided date or date time value.
**Syntax**
**Syntax**
``` sql
timestamp_add(date, INTERVAL value unit)
```
Aliases: `timeStampAdd`, `TIMESTAMP_ADD`.
Aliases: `timeStampAdd`, `TIMESTAMP_ADD`.
**Arguments**
- `date` — Date or date with time. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).
- `value` — Value of interval to add. [Int](../../sql-reference/data-types/int-uint.md).
- `unit` — The type of interval to add. [String](../../sql-reference/data-types/string.md).
@ -642,7 +645,7 @@ Aliases: `timeStampAdd`, `TIMESTAMP_ADD`.
Date or date with time with the specified `value` expressed in `unit` added to `date`.
Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).
**Example**
Query:
@ -663,13 +666,13 @@ Result:
Subtracts the time interval from the provided date or date with time.
**Syntax**
**Syntax**
``` sql
timestamp_sub(unit, value, date)
```
Aliases: `timeStampSub`, `TIMESTAMP_SUB`.
Aliases: `timeStampSub`, `TIMESTAMP_SUB`.
**Arguments**
@ -684,7 +687,7 @@ Aliases: `timeStampSub`, `TIMESTAMP_SUB`.
- `month`
- `quarter`
- `year`
- `value` — Value of interval to subtract. [Int](../../sql-reference/data-types/int-uint.md).
- `date` — Date or date with time. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md).
@ -709,12 +712,12 @@ Result:
│ 2018-07-18 01:02:03 │
└──────────────────────────────────────────────────────────────┘
```
## now {#now}
Returns the current date and time.
Returns the current date and time.
**Syntax**
**Syntax**
``` sql
now([timezone])
@ -1069,4 +1072,3 @@ Result:
│ 2020-01-01 │
└────────────────────────────────────┘
```

View File

@ -70,7 +70,7 @@ CacheDictionary<dictionary_key_type>::CacheDictionary(
, rnd_engine(randomSeed())
{
if (!source_ptr->supportsSelectiveLoad())
throw Exception{full_name + ": source cannot be used with CacheDictionary", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: source cannot be used with CacheDictionary", full_name);
}
template <DictionaryKeyType dictionary_key_type>

View File

@ -77,7 +77,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
else
throw Exception("Method fetchColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for complex key storage");
}
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
@ -85,7 +85,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception("Method insertColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
}
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
@ -93,7 +93,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
insertDefaultKeysImpl(keys);
else
throw Exception("Method insertDefaultKeysImpl is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
}
PaddedPODArray<UInt64> getCachedSimpleKeys() const override
@ -101,7 +101,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
return getCachedKeysImpl();
else
throw Exception("Method getCachedSimpleKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage");
}
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; }
@ -113,7 +113,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, column_fetch_requests);
else
throw Exception("Method fetchColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage");
}
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
@ -121,7 +121,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception("Method insertColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage");
}
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
@ -129,7 +129,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
insertDefaultKeysImpl(keys);
else
throw Exception("Method insertDefaultKeysImpl is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
}
PaddedPODArray<StringRef> getCachedComplexKeys() const override
@ -137,7 +137,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
return getCachedKeysImpl();
else
throw Exception("Method getCachedComplexKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage");
}
size_t getSize() const override { return size; }
@ -601,6 +601,7 @@ private:
PaddedPODArray<Decimal32>,
PaddedPODArray<Decimal64>,
PaddedPODArray<Decimal128>,
PaddedPODArray<Decimal256>,
PaddedPODArray<Float32>,
PaddedPODArray<Float64>,
PaddedPODArray<StringRef>,

View File

@ -49,11 +49,10 @@ template <DictionaryKeyType dictionary_key_type>
void CacheDictionaryUpdateQueue<dictionary_key_type>::tryPushToUpdateQueueOrThrow(CacheDictionaryUpdateUnitPtr<dictionary_key_type> & update_unit_ptr)
{
if (finished)
throw Exception{"CacheDictionaryUpdateQueue finished", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "CacheDictionaryUpdateQueue finished");
if (!update_queue.tryPush(update_unit_ptr, configuration.update_queue_push_timeout_milliseconds))
throw DB::Exception(
ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL,
throw DB::Exception(ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL,
"Cannot push to internal update queue in dictionary {}. "
"Timelimit of {} ms. exceeded. Current queue size is {}",
dictionary_name_for_logs,
@ -65,7 +64,7 @@ template <DictionaryKeyType dictionary_key_type>
void CacheDictionaryUpdateQueue<dictionary_key_type>::waitForCurrentUpdateFinish(CacheDictionaryUpdateUnitPtr<dictionary_key_type> & update_unit_ptr) const
{
if (finished)
throw Exception{"CacheDictionaryUpdateQueue finished", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "CacheDictionaryUpdateQueue finished");
std::unique_lock<std::mutex> update_lock(update_mutex);

View File

@ -142,7 +142,7 @@ void CassandraBlockInputStream::insertValue(IColumn & column, ValueType type, co
break;
}
default:
throw Exception("Unknown type : " + std::to_string(static_cast<int>(type)), ErrorCodes::UNKNOWN_TYPE);
throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type : {}", std::to_string(static_cast<int>(type)));
}
}
@ -256,7 +256,7 @@ void CassandraBlockInputStream::assertTypes(const CassResultPtr & result)
expected_text = "uuid";
break;
default:
throw Exception("Unknown type : " + std::to_string(static_cast<int>(description.types[i].first)), ErrorCodes::UNKNOWN_TYPE);
throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type : {}", std::to_string(static_cast<int>(description.types[i].first)));
}
CassValueType got = cass_result_column_type(result, i);
@ -267,8 +267,10 @@ void CassandraBlockInputStream::assertTypes(const CassResultPtr & result)
continue;
const auto & column_name = description.sample_block.getColumnsWithTypeAndName()[i].name;
throw Exception("Type mismatch for column " + column_name + ": expected Cassandra type " + expected_text,
ErrorCodes::TYPE_MISMATCH);
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Type mismatch for column {} : expected Cassandra type {}",
column_name,
expected_text);
}
}

View File

@ -25,8 +25,8 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory)
setupCassandraDriverLibraryLogging(CASS_LOG_INFO);
return std::make_unique<CassandraDictionarySource>(dict_struct, config, config_prefix + ".cassandra", sample_block);
#else
throw Exception{"Dictionary source of type `cassandra` is disabled because ClickHouse was built without cassandra support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `cassandra` is disabled because ClickHouse was built without cassandra support.");
#endif
};
factory.registerSource("cassandra", create_table_source);
@ -90,7 +90,7 @@ void CassandraSettings::setConsistency(const String & config_str)
else if (config_str == "LocalSerial")
consistency = CASS_CONSISTENCY_LOCAL_SERIAL;
else /// CASS_CONSISTENCY_ANY is only valid for writes
throw Exception("Unsupported consistency level: " + config_str, ErrorCodes::INVALID_CONFIG_PARAMETER);
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Unsupported consistency level: {}", config_str);
}
static const size_t max_block_size = 8192;
@ -156,7 +156,7 @@ BlockInputStreamPtr CassandraDictionarySource::loadIds(const std::vector<UInt64>
BlockInputStreamPtr CassandraDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
if (requested_rows.empty())
throw Exception("No rows requested", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "No rows requested");
/// TODO is there a better way to load data by complex keys?
std::unordered_map<UInt64, std::vector<size_t>> partitions;
@ -185,7 +185,7 @@ BlockInputStreamPtr CassandraDictionarySource::loadKeys(const Columns & key_colu
BlockInputStreamPtr CassandraDictionarySource::loadUpdatedAll()
{
throw Exception("Method loadUpdatedAll is unsupported for CassandraDictionarySource", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for CassandraDictionarySource");
}
CassSessionShared CassandraDictionarySource::getSession()

View File

@ -16,8 +16,10 @@ extern const int CASSANDRA_INTERNAL_ERROR;
void cassandraCheck(CassError code)
{
if (code != CASS_OK)
throw Exception("Cassandra driver error " + std::to_string(code) + ": " + cass_error_desc(code),
ErrorCodes::CASSANDRA_INTERNAL_ERROR);
throw Exception(ErrorCodes::CASSANDRA_INTERNAL_ERROR,
"Cassandra driver error {}: {}",
std::to_string(code),
cass_error_desc(code));
}
@ -31,8 +33,12 @@ void cassandraWaitAndCheck(CassFuturePtr & future)
const char * message;
size_t message_len;
cass_future_error_message(future, &message, & message_len);
std::string full_message = "Cassandra driver error " + std::to_string(code) + ": " + cass_error_desc(code) + ": " + message;
throw Exception(full_message, ErrorCodes::CASSANDRA_INTERNAL_ERROR);
throw Exception(ErrorCodes::CASSANDRA_INTERNAL_ERROR,
"Cassandra driver error {}: {}: {}",
std::to_string(code),
cass_error_desc(code),
message);
}
static std::once_flag setup_logging_flag;

View File

@ -20,7 +20,7 @@ namespace ErrorCodes
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex)
{
if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second)
throw Exception("DictionaryFactory: the layout name '" + layout_type + "' is not unique", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "DictionaryFactory: the layout name '{}' is not unique", layout_type);
layout_complexity[layout_type] = is_complex;
@ -38,8 +38,9 @@ DictionaryPtr DictionaryFactory::create(
const auto & layout_prefix = config_prefix + ".layout";
config.keys(layout_prefix, keys);
if (keys.size() != 1)
throw Exception{name + ": element dictionary.layout should have exactly one child element",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG};
throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG,
"{}: element dictionary.layout should have exactly one child element",
name);
const DictionaryStructure dict_struct{config, config_prefix};
@ -61,7 +62,10 @@ DictionaryPtr DictionaryFactory::create(
}
}
throw Exception{name + ": unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"{}: unknown dictionary layout type: {}",
name,
layout_type);
}
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr context) const
@ -77,7 +81,9 @@ bool DictionaryFactory::isComplex(const std::string & layout_type) const
if (found != layout_complexity.end())
return found->second;
throw Exception{"Unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"Unknown dictionary layout type: {}",
layout_type);
}

View File

@ -251,7 +251,7 @@ public:
else if constexpr (IsNumber<DictionaryAttributeType>)
return ColumnType::create(size);
else
throw Exception{"Unsupported attribute type.", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Unsupported attribute type.");
}
};
@ -288,7 +288,7 @@ public:
use_default_value_from_column = false;
}
else
throw Exception{"Type of default column is not the same as dictionary attribute type.", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Type of default column is not the same as dictionary attribute type.");
}
}
@ -548,10 +548,10 @@ static const PaddedPODArray<T> & getColumnVectorData(
if (!vector_col)
{
throw Exception{ErrorCodes::TYPE_MISMATCH,
throw Exception(ErrorCodes::TYPE_MISMATCH,
"{}: type mismatch: column has wrong type expected {}",
dictionary->getDictionaryID().getNameForLogs(),
TypeName<T>::get()};
TypeName<T>::get());
}
if (is_const_column)

View File

@ -71,7 +71,7 @@ DictionarySourceFactory::DictionarySourceFactory() : log(&Poco::Logger::get("Dic
void DictionarySourceFactory::registerSource(const std::string & source_type, Creator create_source)
{
if (!registered_sources.emplace(source_type, std::move(create_source)).second)
throw Exception("DictionarySourceFactory: the source name '" + source_type + "' is not unique", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "DictionarySourceFactory: the source name '{}' is not unique", source_type);
}
DictionarySourcePtr DictionarySourceFactory::create(
@ -87,8 +87,9 @@ DictionarySourcePtr DictionarySourceFactory::create(
config.keys(config_prefix, keys);
if (keys.empty() || keys.size() > 2)
throw Exception{name + ": element dictionary.source should have one or two child elements",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG};
throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG,
"{}: element dictionary.source should have one or two child elements",
name);
const std::string & source_type = keys.front() == "settings" ? keys.back() : keys.front();
@ -100,7 +101,10 @@ DictionarySourcePtr DictionarySourceFactory::create(
return create_source(dict_struct, config, config_prefix, sample_block, context, default_database, check_config);
}
throw Exception{name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"{}: unknown dictionary source type: {}",
name,
source_type);
}
DictionarySourceFactory & DictionarySourceFactory::instance()

View File

@ -125,9 +125,8 @@ Block BlockInputStreamWithAdditionalColumns::readImpl()
auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, block_rows);
if (cut_block.rows() != block_rows)
throw Exception(
"Number of rows in block to add after cut must equal to number of rows in block from inner stream",
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH,
"Number of rows in block to add after cut must equal to number of rows in block from inner stream");
for (Int64 i = static_cast<Int64>(cut_block.columns() - 1); i >= 0; --i)
block.insert(0, cut_block.getByPosition(i));

View File

@ -33,7 +33,7 @@ namespace
const auto expression = config.getString(config_prefix + ".expression", "");
if (name.empty() && !expression.empty())
throw Exception{"Element " + config_prefix + ".name is empty", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty");
const auto type_name = config.getString(config_prefix + ".type", default_type);
return DictionaryTypedSpecialAttribute{std::move(name), std::move(expression), DataTypeFactory::instance().get(type_name)};
@ -65,6 +65,7 @@ AttributeUnderlyingType getAttributeUnderlyingType(const DataTypePtr & type)
case TypeIndex::Decimal32: return AttributeUnderlyingType::utDecimal32;
case TypeIndex::Decimal64: return AttributeUnderlyingType::utDecimal64;
case TypeIndex::Decimal128: return AttributeUnderlyingType::utDecimal128;
case TypeIndex::Decimal256: return AttributeUnderlyingType::utDecimal256;
case TypeIndex::Date: return AttributeUnderlyingType::utUInt16;
case TypeIndex::DateTime: return AttributeUnderlyingType::utUInt32;
@ -81,11 +82,11 @@ AttributeUnderlyingType getAttributeUnderlyingType(const DataTypePtr & type)
default: break;
}
throw Exception{"Unknown type for dictionary" + type->getName(), ErrorCodes::UNKNOWN_TYPE};
throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type {} for dictionary attribute", type->getName());
}
std::string toString(const AttributeUnderlyingType type)
std::string toString(AttributeUnderlyingType type)
{
switch (type)
{
@ -117,11 +118,13 @@ std::string toString(const AttributeUnderlyingType type)
return "Decimal64";
case AttributeUnderlyingType::utDecimal128:
return "Decimal128";
case AttributeUnderlyingType::utDecimal256:
return "Decimal256";
case AttributeUnderlyingType::utString:
return "String";
}
throw Exception{"Unknown attribute_type " + toString(static_cast<int>(type)), ErrorCodes::ARGUMENT_OUT_OF_BOUND};
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Unknown dictionary attribute type {}", toString(static_cast<int>(type)));
}
@ -129,7 +132,7 @@ DictionarySpecialAttribute::DictionarySpecialAttribute(const Poco::Util::Abstrac
: name{config.getString(config_prefix + ".name", "")}, expression{config.getString(config_prefix + ".expression", "")}
{
if (name.empty() && !expression.empty())
throw Exception{"Element " + config_prefix + ".name is empty", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty", config_prefix);
}
@ -141,7 +144,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
const auto has_key = config.has(structure_prefix + ".key");
if (has_key && has_id)
throw Exception{"Only one of 'id' and 'key' should be specified", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only one of 'id' and 'key' should be specified");
if (has_id)
id.emplace(config, structure_prefix + ".id");
@ -149,15 +152,15 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
{
key.emplace(getAttributes(config, structure_prefix + ".key", true));
if (key->empty())
throw Exception{"Empty 'key' supplied", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty 'key' supplied");
}
else
throw Exception{"Dictionary structure should specify either 'id' or 'key'", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary structure should specify either 'id' or 'key'");
if (id)
{
if (id->name.empty())
throw Exception{"'id' cannot be empty", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "'id' cannot be empty");
const char * range_default_type = "Date";
if (config.has(structure_prefix + ".range_min"))
@ -168,28 +171,27 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
if (range_min.has_value() != range_max.has_value())
{
throw Exception{"Dictionary structure should have both 'range_min' and 'range_max' either specified or not.",
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Dictionary structure should have both 'range_min' and 'range_max' either specified or not.");
}
if (range_min && range_max && !range_min->type->equals(*range_max->type))
{
throw Exception{"Dictionary structure 'range_min' and 'range_max' should have same type, "
"'range_min' type: "
+ range_min->type->getName()
+ ", "
"'range_max' type: "
+ range_max->type->getName(),
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Dictionary structure 'range_min' and 'range_max' should have same type, "
"'range_min' type: {},"
"'range_max' type: {}",
range_min->type->getName(),
range_max->type->getName());
}
if (range_min)
{
if (!range_min->type->isValueRepresentedByInteger())
throw Exception{"Dictionary structure type of 'range_min' and 'range_max' should be an integer, Date, DateTime, or Enum."
" Actual 'range_min' and 'range_max' type is "
+ range_min->type->getName(),
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Dictionary structure type of 'range_min' and 'range_max' should be an integer, Date, DateTime, or Enum."
" Actual 'range_min' and 'range_max' type is {}",
range_min->type->getName());
}
if (!id->expression.empty() || (range_min && !range_min->expression.empty()) || (range_max && !range_max->expression.empty()))
@ -208,8 +210,9 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
{
if (id && attribute.underlying_type != AttributeUnderlyingType::utUInt64)
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Hierarchical attribute type for dictionary with simple key must be UInt64. Actual ({})",
"Hierarchical attribute type for dictionary with simple key must be UInt64. Actual {}",
toString(attribute.underlying_type));
else if (key)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary with complex key does not support hierarchy");
@ -218,7 +221,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
}
if (attributes.empty())
throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary has no attributes defined");
if (config.getBool(config_prefix + ".layout.ip_trie.access_to_key_from_attributes", false))
access_to_key_from_attributes = true;
@ -228,7 +231,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const
{
if (key_types.size() != key->size())
throw Exception{"Key structure does not match, expected " + getKeyDescription(), ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Key structure does not match, expected {}", getKeyDescription());
for (const auto i : ext::range(0, key_types.size()))
{
@ -236,9 +239,11 @@ void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const
const auto & actual_type = key_types[i];
if (!areTypesEqual(expected_type, actual_type))
throw Exception{"Key type at position " + std::to_string(i) + " does not match, expected " + expected_type->getName() + ", found "
+ actual_type->getName(),
ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Key type at position {} does not match, expected {}, found {}",
std::to_string(i),
expected_type->getName(),
actual_type->getName());
}
}
@ -249,13 +254,13 @@ const DictionaryAttribute & DictionaryStructure::getAttribute(const std::string
if (it == attribute_name_to_index.end())
{
if (!access_to_key_from_attributes)
throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such attribute '{}'", attribute_name);
for (const auto & key_attribute : *key)
if (key_attribute.name == attribute_name)
return key_attribute;
throw Exception{"No such attribute '" + attribute_name + "' in keys", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such attribute '{}' in keys", attribute_name);
}
size_t attribute_index = it->second;
@ -267,8 +272,10 @@ const DictionaryAttribute & DictionaryStructure::getAttribute(const std::string
const auto & attribute = getAttribute(attribute_name);
if (!areTypesEqual(attribute.type, type))
throw Exception{"Attribute type does not match, expected " + attribute.type->getName() + ", found " + type->getName(),
ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Attribute type does not match, expected {}, found {}",
attribute.type->getName(),
type->getName());
return attribute;
}
@ -343,11 +350,10 @@ static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & k
for (const auto & key : keys)
{
if (valid_keys.find(key) == valid_keys.end())
throw Exception{"Unknown key '" + key + "' inside attribute section", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown key '{}' inside attribute section", key);
}
}
std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
@ -388,8 +394,7 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
bool inserted = insert_result.second;
if (!inserted)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Dictionary attributes names must be unique. Attribute name ({}) is not unique",
name);
@ -442,13 +447,13 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
const auto injective = config.getBool(prefix + "injective", false);
const auto is_object_id = config.getBool(prefix + "is_object_id", false);
if (name.empty())
throw Exception{"Properties 'name' and 'type' of an attribute cannot be empty", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Properties 'name' and 'type' of an attribute cannot be empty");
if (has_hierarchy && !hierarchy_allowed)
throw Exception{"Hierarchy not allowed in '" + prefix, ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Hierarchy not allowed in '{}'", prefix);
if (has_hierarchy && hierarchical)
throw Exception{"Only one hierarchical attribute supported", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Only one hierarchical attribute supported");
has_hierarchy = has_hierarchy || hierarchical;

View File

@ -32,13 +32,14 @@ enum class AttributeUnderlyingType
utDecimal32,
utDecimal64,
utDecimal128,
utDecimal256,
utString
};
AttributeUnderlyingType getAttributeUnderlyingType(const std::string & type);
std::string toString(const AttributeUnderlyingType type);
std::string toString(AttributeUnderlyingType type);
/// Min and max lifetimes for a dictionary or it's entry
using DictionaryLifetime = ExternalLoadableLifetime;
@ -125,6 +126,9 @@ void callOnDictionaryAttributeType(AttributeUnderlyingType type, F&& func)
case AttributeUnderlyingType::utDecimal128:
func(DictionaryAttributeType<Decimal128>());
break;
case AttributeUnderlyingType::utDecimal256:
func(DictionaryAttributeType<Decimal256>());
break;
}
};

View File

@ -27,7 +27,7 @@ DirectDictionary<dictionary_key_type>::DirectDictionary(
, source_ptr{std::move(source_ptr_)}
{
if (!source_ptr->supportsSelectiveLoad())
throw Exception{full_name + ": source cannot be used with DirectDictionary", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: source cannot be used with DirectDictionary", full_name);
}
template <DictionaryKeyType dictionary_key_type>
@ -290,20 +290,20 @@ namespace
{
if (dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"'key' is not supported for dictionary of layout '({})'",
"'key' is not supported for dictionary of layout '{}'",
layout_name);
}
else
{
if (dict_struct.id)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"'id' is not supported for dictionary of layout '({})'",
"'id' is not supported for dictionary of layout '{}'",
layout_name);
}
if (dict_struct.range_min || dict_struct.range_max)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): elements .structure.range_min and .structure.range_max should be defined only " \
"{}: elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
full_name);
@ -311,7 +311,7 @@ namespace
if (config.has(config_prefix + ".lifetime.min") || config.has(config_prefix + ".lifetime.max"))
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"'lifetime' parameter is redundant for the dictionary' of layout '({})'",
"'lifetime' parameter is redundant for the dictionary' of layout '{}'",
layout_name);
return std::make_unique<DirectDictionary<dictionary_key_type>>(dict_id, dict_struct, std::move(source_ptr));

View File

@ -104,7 +104,7 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar
BlockInputStreamPtr ExecutableDictionarySource::loadAll()
{
if (implicit_key)
throw Exception("ExecutableDictionarySource with implicit_key does not support loadAll method", ErrorCodes::UNSUPPORTED_METHOD);
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutableDictionarySource with implicit_key does not support loadAll method");
LOG_TRACE(log, "loadAll {}", toString());
auto process = ShellCommand::execute(command);
@ -115,7 +115,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadAll()
BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll()
{
if (implicit_key)
throw Exception("ExecutableDictionarySource with implicit_key does not support loadUpdatedAll method", ErrorCodes::UNSUPPORTED_METHOD);
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutableDictionarySource with implicit_key does not support loadUpdatedAll method");
time_t new_update_time = time(nullptr);
SCOPE_EXIT(update_time = new_update_time);

View File

@ -70,12 +70,12 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(const ExecutableP
BlockInputStreamPtr ExecutablePoolDictionarySource::loadAll()
{
throw Exception("ExecutablePoolDictionarySource with implicit_key does not support loadAll method", ErrorCodes::UNSUPPORTED_METHOD);
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutablePoolDictionarySource with implicit_key does not support loadAll method");
}
BlockInputStreamPtr ExecutablePoolDictionarySource::loadUpdatedAll()
{
throw Exception("ExecutablePoolDictionarySource with implicit_key does not support loadAll method", ErrorCodes::UNSUPPORTED_METHOD);
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutablePoolDictionarySource with implicit_key does not support loadAll method");
}
namespace

View File

@ -173,7 +173,7 @@ std::string ExternalQueryBuilder::composeUpdateQuery(const std::string & update_
std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector<UInt64> & ids)
{
if (!dict_struct.id)
throw Exception{"Simple key required for method", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Simple key required for method");
WriteBufferFromOwnString out;
writeString("SELECT ", out);
@ -244,10 +244,10 @@ std::string ExternalQueryBuilder::composeLoadKeysQuery(
const Columns & key_columns, const std::vector<size_t> & requested_rows, LoadKeysMethod method, size_t partition_key_prefix)
{
if (!dict_struct.key)
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Composite key required for method");
if (key_columns.size() != dict_struct.key->size())
throw Exception{"The size of key_columns does not equal to the size of dictionary key", ErrorCodes::LOGICAL_ERROR};
throw Exception(ErrorCodes::LOGICAL_ERROR, "The size of key_columns does not equal to the size of dictionary key");
WriteBufferFromOwnString out;
writeString("SELECT ", out);
@ -386,7 +386,7 @@ void ExternalQueryBuilder::composeInWithTuples(const Columns & key_columns, cons
void ExternalQueryBuilder::composeKeyTupleDefinition(WriteBuffer & out, size_t beg, size_t end) const
{
if (!dict_struct.key)
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Composite key required for method");
writeChar('(', out);

View File

@ -94,7 +94,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
bool check_config) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `file` does not support attribute expressions");
const auto filepath = config.getString(config_prefix + ".file.path");
const auto format = config.getString(config_prefix + ".file.format");

View File

@ -25,17 +25,17 @@ public:
BlockInputStreamPtr loadUpdatedAll() override
{
throw Exception{"Method loadUpdatedAll is unsupported for FileDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for FileDictionarySource");
}
BlockInputStreamPtr loadIds(const std::vector<UInt64> & /*ids*/) override
{
throw Exception{"Method loadIds is unsupported for FileDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadIds is unsupported for FileDictionarySource");
}
BlockInputStreamPtr loadKeys(const Columns & /*key_columns*/, const std::vector<size_t> & /*requested_rows*/) override
{
throw Exception{"Method loadKeys is unsupported for FileDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadKeys is unsupported for FileDictionarySource");
}
bool isModified() const override

View File

@ -322,7 +322,7 @@ void FlatDictionary::loadData()
updateData();
if (configuration.require_nonempty && 0 == element_count)
throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY};
throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, "{}: dictionary source is empty and 'require_nonempty' property is set.", full_name);
}
void FlatDictionary::calculateBytesAllocated()
@ -407,7 +407,7 @@ void FlatDictionary::resize(Attribute & attribute, UInt64 key)
{
if (key >= configuration.max_array_size)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"({}): identifier should be less than ({})",
"{}: identifier should be less than {}",
full_name,
toString(configuration.max_array_size));
@ -489,7 +489,7 @@ void registerDictionaryFlat(DictionaryFactory & factory)
if (dict_struct.range_min || dict_struct.range_max)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): elements .structure.range_min and .structure.range_max should be defined only "
"{}: elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
full_name);

View File

@ -120,6 +120,7 @@ private:
Decimal32,
Decimal64,
Decimal128,
Decimal256,
Float32,
Float64,
StringRef>
@ -137,6 +138,7 @@ private:
ContainerType<Decimal32>,
ContainerType<Decimal64>,
ContainerType<Decimal128>,
ContainerType<Decimal256>,
ContainerType<Float32>,
ContainerType<Float64>,
ContainerType<StringRef>>

View File

@ -206,7 +206,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
bool check_config) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `http` does not support attribute expressions");
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config);

View File

@ -123,7 +123,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
[&](const size_t row, const auto value) { return out[row] = value; },
[&](const size_t row)
{
out[row] = 0;
out[row] = ValueType();
(*vec_null_map_to)[row] = true;
},
default_value_extractor);
@ -547,7 +547,7 @@ void HashedDictionary<dictionary_key_type, sparse>::loadData()
if (require_nonempty && 0 == element_count)
throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY,
"({}): dictionary source is empty and 'require_nonempty' property is set.",
"{}: dictionary source is empty and 'require_nonempty' property is set.",
full_name);
}
@ -676,10 +676,10 @@ void registerDictionaryHashed(DictionaryFactory & factory)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed dictionary");
if (dict_struct.range_min || dict_struct.range_max)
throw Exception{full_name
+ ": elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{}: elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
full_name);
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};

View File

@ -152,6 +152,7 @@ private:
Decimal32,
Decimal64,
Decimal128,
Decimal256,
Float32,
Float64,
StringRef>
@ -170,13 +171,13 @@ private:
CollectionType<Decimal32>,
CollectionType<Decimal64>,
CollectionType<Decimal128>,
CollectionType<Decimal256>,
CollectionType<Float32>,
CollectionType<Float64>,
CollectionType<StringRef>>
container;
std::unique_ptr<Arena> string_arena;
};
void createAttributes();

View File

@ -52,7 +52,7 @@ namespace
const auto * parent_key_column_typed = checkAndGetColumn<ColumnVector<UInt64>>(*parent_key_column);
if (!parent_key_column_typed)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Parent key column should be UInt64. Actual ({})",
"Parent key column should be UInt64. Actual {}",
hierarchical_attribute.type->getName());
const auto & parent_keys = parent_key_column_typed->getData();

View File

@ -124,20 +124,20 @@ static size_t formatIPWithPrefix(const unsigned char * src, UInt8 prefix_len, bo
static void validateKeyTypes(const DataTypes & key_types)
{
if (key_types.empty() || key_types.size() > 2)
throw Exception{"Expected a single IP address or IP with mask", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a single IP address or IP with mask");
const auto * key_ipv4type = typeid_cast<const DataTypeUInt32 *>(key_types[0].get());
const auto * key_ipv6type = typeid_cast<const DataTypeFixedString *>(key_types[0].get());
if (key_ipv4type == nullptr && (key_ipv6type == nullptr || key_ipv6type->getN() != 16))
throw Exception{"Key does not match, expected either `IPv4` (`UInt32`) or `IPv6` (`FixedString(16)`)",
ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Key does not match, expected either `IPv4` (`UInt32`) or `IPv6` (`FixedString(16)`)");
if (key_types.size() > 1)
{
const auto * mask_col_type = typeid_cast<const DataTypeUInt8 *>(key_types[1].get());
if (mask_col_type == nullptr)
throw Exception{"Mask do not match, expected UInt8", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Mask do not match, expected UInt8");
}
}
@ -293,7 +293,7 @@ ColumnUInt8::Ptr IPAddressDictionary::hasKeys(const Columns & key_columns, const
{
auto addr = first_column->getDataAt(i);
if (unlikely(addr.size != IPV6_BINARY_LENGTH))
throw Exception("Expected key to be FixedString(16)", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected key to be FixedString(16)");
auto found = tryLookupIPv6(reinterpret_cast<const uint8_t *>(addr.data));
out[i] = (found != ipNotFound());
@ -316,8 +316,10 @@ void IPAddressDictionary::createAttributes()
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
throw Exception{full_name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(),
ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH,
"{}: hierarchical attributes not supported for dictionary of type {}",
full_name,
getTypeName());
}
};
@ -492,7 +494,7 @@ void IPAddressDictionary::loadData()
LOG_TRACE(logger, "{} ip records are read", ip_records.size());
if (require_nonempty && 0 == element_count)
throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY};
throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, "{}: dictionary source is empty and 'require_nonempty' property is set.", full_name);
}
template <typename T>
@ -591,7 +593,7 @@ void IPAddressDictionary::getItemsByTwoKeyColumnsImpl(
{
const auto * key_ip_column_ptr = typeid_cast<const ColumnVector<UInt32> *>(&*key_columns.front());
if (key_ip_column_ptr == nullptr)
throw Exception{"Expected a UInt32 IP column", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a UInt32 IP column");
const auto & key_mask_column = assert_cast<const ColumnVector<UInt8> &>(*key_columns.back());
@ -625,7 +627,7 @@ void IPAddressDictionary::getItemsByTwoKeyColumnsImpl(
const auto * key_ip_column_ptr = typeid_cast<const ColumnFixedString *>(&*key_columns.front());
if (key_ip_column_ptr == nullptr || key_ip_column_ptr->getN() != IPV6_BINARY_LENGTH)
throw Exception{"Expected a FixedString(16) IP column", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a FixedString(16) IP column");
const auto & key_mask_column = assert_cast<const ColumnVector<UInt8> &>(*key_columns.back());
@ -698,7 +700,7 @@ void IPAddressDictionary::getItemsImpl(
{
auto addr = first_column->getDataAt(i);
if (addr.size != IPV6_BINARY_LENGTH)
throw Exception("Expected key to be FixedString(16)", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected key to be FixedString(16)");
auto found = tryLookupIPv6(reinterpret_cast<const uint8_t *>(addr.data));
if (found != ipNotFound())
@ -744,7 +746,7 @@ const IPAddressDictionary::Attribute & IPAddressDictionary::getAttribute(const s
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: no such attribute '{}'", full_name, attribute_name);
return attributes[it->second];
}
@ -921,7 +923,7 @@ void registerDictionaryTrie(DictionaryFactory & factory)
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (!dict_struct.key || dict_struct.key->size() != 1)
throw Exception{"Dictionary of layout 'ip_trie' has to have one 'key'", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary of layout 'ip_trie' has to have one 'key'");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};

View File

@ -101,6 +101,7 @@ private:
Decimal32,
Decimal64,
Decimal128,
Decimal256,
Float32,
Float64,
String>
@ -118,6 +119,7 @@ private:
ContainerType<Decimal32>,
ContainerType<Decimal64>,
ContainerType<Decimal128>,
ContainerType<Decimal256>,
ContainerType<Float32>,
ContainerType<Float64>,
ContainerType<StringRef>>

View File

@ -51,7 +51,7 @@ public:
BlockInputStreamPtr loadUpdatedAll() override
{
throw Exception{"Method loadUpdatedAll is unsupported for LibraryDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for LibraryDictionarySource");
}
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;

View File

@ -126,7 +126,7 @@ MongoDBDictionarySource::MongoDBDictionarySource(
#if POCO_VERSION >= 0x01070800
Poco::MongoDB::Database poco_db(db);
if (!poco_db.authenticate(*connection, user, password, method.empty() ? Poco::MongoDB::Database::AUTH_SCRAM_SHA1 : method))
throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
throw Exception(ErrorCodes::MONGODB_CANNOT_AUTHENTICATE, "Cannot authenticate in MongoDB, incorrect user or password");
#else
authenticate(*connection, db, user, password);
#endif
@ -151,7 +151,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadAll()
BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
if (!dict_struct.id)
throw Exception{"'id' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is required for selective loading");
auto cursor = createCursor(db, collection, sample_block);
@ -172,7 +172,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector<UInt64> &
BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
if (!dict_struct.key)
throw Exception{"'key' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is required for selective loading");
auto cursor = createCursor(db, collection, sample_block);
@ -198,6 +198,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_column
case AttributeUnderlyingType::utDecimal32:
case AttributeUnderlyingType::utDecimal64:
case AttributeUnderlyingType::utDecimal128:
case AttributeUnderlyingType::utDecimal256:
key.add(attr.second.name, Int32(key_columns[attr.first]->get64(row_idx)));
break;

View File

@ -50,7 +50,7 @@ public:
BlockInputStreamPtr loadUpdatedAll() override
{
throw Exception{"Method loadUpdatedAll is unsupported for MongoDBDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for MongoDBDictionarySource");
}
bool supportsSelectiveLoad() const override { return true; }

View File

@ -28,8 +28,8 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
(void)config;
(void)config_prefix;
(void)sample_block;
throw Exception{"Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support.");
#endif
};
factory.registerSource("mysql", create_table_source);

View File

@ -151,7 +151,7 @@ BlockInputStreamPtr IPolygonDictionary::getBlockInputStream(const Names &, size_
{
// TODO: In order for this to work one would first have to support retrieving arrays from dictionaries.
// I believe this is a separate task done by some other people.
throw Exception{"Reading the dictionary is not allowed", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Reading the dictionary is not allowed");
}
void IPolygonDictionary::setup()
@ -164,9 +164,9 @@ void IPolygonDictionary::setup()
attributes.emplace_back(std::move(column));
if (attribute.hierarchical)
throw Exception{ErrorCodes::TYPE_MISMATCH,
throw Exception(ErrorCodes::TYPE_MISMATCH,
"{}: hierarchical attributes not supported for dictionary of polygonal type",
getDictionaryID().getNameForLogs()};
getDictionaryID().getNameForLogs());
}
}
@ -248,13 +248,13 @@ void IPolygonDictionary::calculateBytesAllocated()
std::vector<IPolygonDictionary::Point> IPolygonDictionary::extractPoints(const Columns & key_columns)
{
if (key_columns.size() != 2)
throw Exception{"Expected two columns of coordinates with type Float64", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected two columns of coordinates with type Float64");
const auto * column_x = typeid_cast<const ColumnVector<Float64>*>(key_columns[0].get());
const auto * column_y = typeid_cast<const ColumnVector<Float64>*>(key_columns[1].get());
if (!column_x || !column_y)
throw Exception{"Expected columns of Float64", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected columns of Float64");
const auto rows = key_columns.front()->size();
@ -392,17 +392,17 @@ const IColumn * unrollMultiPolygons(const ColumnPtr & column, Offset & offset)
{
const auto * ptr_multi_polygons = typeid_cast<const ColumnArray*>(column.get());
if (!ptr_multi_polygons)
throw Exception{"Expected a column containing arrays of polygons", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of polygons");
offset.multi_polygon_offsets.assign(ptr_multi_polygons->getOffsets());
const auto * ptr_polygons = typeid_cast<const ColumnArray*>(&ptr_multi_polygons->getData());
if (!ptr_polygons)
throw Exception{"Expected a column containing arrays of rings when reading polygons", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of rings when reading polygons");
offset.polygon_offsets.assign(ptr_polygons->getOffsets());
const auto * ptr_rings = typeid_cast<const ColumnArray*>(&ptr_polygons->getData());
if (!ptr_rings)
throw Exception{"Expected a column containing arrays of points when reading rings", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of points when reading rings");
offset.ring_offsets.assign(ptr_rings->getOffsets());
return ptr_rings->getDataPtr().get();
@ -412,7 +412,7 @@ const IColumn * unrollSimplePolygons(const ColumnPtr & column, Offset & offset)
{
const auto * ptr_polygons = typeid_cast<const ColumnArray*>(column.get());
if (!ptr_polygons)
throw Exception{"Expected a column containing arrays of points", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of points");
offset.ring_offsets.assign(ptr_polygons->getOffsets());
std::iota(offset.polygon_offsets.begin(), offset.polygon_offsets.end(), 1);
offset.multi_polygon_offsets.assign(offset.polygon_offsets);
@ -425,13 +425,13 @@ void handlePointsReprByArrays(const IColumn * column, Data & data, Offset & offs
const auto * ptr_points = typeid_cast<const ColumnArray*>(column);
const auto * ptr_coord = typeid_cast<const ColumnVector<Float64>*>(&ptr_points->getData());
if (!ptr_coord)
throw Exception{"Expected coordinates to be of type Float64", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected coordinates to be of type Float64");
const auto & offsets = ptr_points->getOffsets();
IColumn::Offset prev_offset = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
if (offsets[i] - prev_offset != 2)
throw Exception{"All points should be two-dimensional", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "All points should be two-dimensional");
prev_offset = offsets[i];
addNewPoint(ptr_coord->getElement(2 * i), ptr_coord->getElement(2 * i + 1), data, offset);
}
@ -441,13 +441,13 @@ void handlePointsReprByTuples(const IColumn * column, Data & data, Offset & offs
{
const auto * ptr_points = typeid_cast<const ColumnTuple*>(column);
if (!ptr_points)
throw Exception{"Expected a column of tuples representing points", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column of tuples representing points");
if (ptr_points->tupleSize() != 2)
throw Exception{"Points should be two-dimensional", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Points should be two-dimensional");
const auto * column_x = typeid_cast<const ColumnVector<Float64>*>(&ptr_points->getColumn(0));
const auto * column_y = typeid_cast<const ColumnVector<Float64>*>(&ptr_points->getColumn(1));
if (!column_x || !column_y)
throw Exception{"Expected coordinates to be of type Float64", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected coordinates to be of type Float64");
for (size_t i = 0; i < column_x->size(); ++i)
{
addNewPoint(column_x->getElement(i), column_y->getElement(i), data, offset);
@ -473,8 +473,8 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column)
}
if (!offset.allRingsHaveAPositiveArea())
throw Exception{"Every ring included in a polygon or excluded from it should contain at least 3 points",
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Every ring included in a polygon or excluded from it should contain at least 3 points");
/** Adding the first empty polygon */
data.addPolygon(true);

View File

@ -172,10 +172,10 @@ DictionaryPtr createLayout(const std::string & ,
const String name = config.getString(config_prefix + ".name");
if (!dict_struct.key)
throw Exception{"'key' is required for a polygon dictionary", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "'key' is required for a polygon dictionary");
if (dict_struct.key->size() != 1)
throw Exception{"The 'key' should consist of a single attribute for a polygon dictionary",
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"The 'key' should consist of a single attribute for a polygon dictionary");
IPolygonDictionary::InputType input_type;
IPolygonDictionary::PointType point_type;
@ -206,19 +206,19 @@ DictionaryPtr createLayout(const std::string & ,
point_type = IPolygonDictionary::PointType::Tuple;
}
else
throw Exception{"The key type " + key_type->getName() +
" is not one of the following allowed types for a polygon dictionary: " +
multi_polygon_array.getName() + " " +
multi_polygon_tuple.getName() + " " +
simple_polygon_array.getName() + " " +
simple_polygon_tuple.getName() + " ",
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"The key type {} is not one of the following allowed types for a polygon dictionary: {} {} {} {} ",
key_type->getName(),
multi_polygon_array.getName(),
multi_polygon_tuple.getName(),
simple_polygon_array.getName(),
simple_polygon_tuple.getName());
if (dict_struct.range_min || dict_struct.range_max)
throw Exception{name
+ ": elements range_min and range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{}: elements range_min and range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
name);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};

View File

@ -180,8 +180,8 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
(void)config;
(void)root_config_prefix;
(void)sample_block;
throw Exception{"Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support.");
#endif
};
factory.registerSource("postgresql", create_table_source);

View File

@ -260,8 +260,8 @@ void RangeHashedDictionary::createAttributes()
attributes.push_back(createAttribute(attribute, attribute.null_value));
if (attribute.hierarchical)
throw Exception{ErrorCodes::BAD_ARGUMENTS, "Hierarchical attributes not supported by {} dictionary.",
getDictionaryID().getNameForLogs()};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Hierarchical attributes not supported by {} dictionary.",
getDictionaryID().getNameForLogs());
}
}
@ -311,8 +311,8 @@ void RangeHashedDictionary::loadData()
stream->readSuffix();
if (require_nonempty && 0 == element_count)
throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.",
ErrorCodes::DICTIONARY_IS_EMPTY};
throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY,
"{}: dictionary source is empty and 'require_nonempty' property is set.");
}
template <typename T>
@ -497,7 +497,7 @@ const RangeHashedDictionary::Attribute & RangeHashedDictionary::getAttribute(con
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}: no such attribute '{}'", full_name, attribute_name);
return attributes[it->second];
}
@ -507,8 +507,9 @@ RangeHashedDictionary::getAttributeWithType(const std::string & attribute_name,
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != type)
throw Exception{attribute_name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "attribute {} has type {}",
attribute_name,
toString(attribute.type));
return attribute;
}
@ -613,8 +614,9 @@ BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & col
ListType::forEach(callable);
if (!callable.stream)
throw Exception(
"Unexpected range type for RangeHashed dictionary: " + dict_struct.range_min->type->getName(), ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unexpected range type for RangeHashed dictionary: {}",
dict_struct.range_min->type->getName());
return callable.stream;
}
@ -629,11 +631,12 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout 'range_hashed'", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout 'range_hashed'");
if (!dict_struct.range_min || !dict_struct.range_max)
throw Exception{full_name + ": dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max",
ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{}: dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max",
full_name);
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};

View File

@ -112,6 +112,7 @@ private:
Decimal32,
Decimal64,
Decimal128,
Decimal256,
Float32,
Float64,
StringRef>
@ -129,6 +130,7 @@ private:
Ptr<Decimal32>,
Ptr<Decimal64>,
Ptr<Decimal128>,
Ptr<Decimal256>,
Ptr<Float32>,
Ptr<Float64>,
Ptr<StringRef>>

View File

@ -57,7 +57,7 @@ namespace DB
void insertValue(IColumn & column, const ValueType type, const Poco::Redis::BulkString & bulk_string)
{
if (bulk_string.isNull())
throw Exception{"Type mismatch, expected not Null String", ErrorCodes::TYPE_MISMATCH};
throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected not Null String");
const String & string_value = bulk_string.value();
switch (type)
@ -112,7 +112,9 @@ namespace DB
assert_cast<ColumnUInt128 &>(column).insertValue(parse<UUID>(string_value));
break;
default:
throw Exception("Value of unsupported type:" + column.getName(), ErrorCodes::UNKNOWN_TYPE);
throw Exception(ErrorCodes::UNKNOWN_TYPE,
"Value of unsupported type: {}",
column.getName());
}
}
}
@ -152,8 +154,9 @@ namespace DB
const auto & keys_array = keys.get<RedisArray>(cursor);
if (keys_array.size() < 2)
{
throw Exception{"Too low keys in request to source: " + DB::toString(keys_array.size())
+ ", expected 2 or more", ErrorCodes::LOGICAL_ERROR};
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Too low keys in request to source: {}, expected 2 or more",
DB::toString(keys_array.size()));
}
if (num_rows + keys_array.size() - 1 > max_block_size)
@ -166,8 +169,8 @@ namespace DB
auto values = client->execute<RedisArray>(command_for_values);
if (keys_array.size() != values.size() + 1) // 'HMGET' primary_key secondary_keys
throw Exception{"Inconsistent sizes of keys and values in Redis request",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH,
"Inconsistent sizes of keys and values in Redis request");
const auto & primary_key = keys_array.get<RedisBulkString>(0);
for (size_t i = 0; i < values.size(); ++i)
@ -196,7 +199,8 @@ namespace DB
auto values = client->execute<RedisArray>(command_for_values);
if (values.size() != need_values)
throw Exception{"Inconsistent sizes of keys and values in Redis request", ErrorCodes::INTERNAL_REDIS_ERROR};
throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR,
"Inconsistent sizes of keys and values in Redis request");
for (size_t i = 0; i < values.size(); ++i)
{

View File

@ -66,25 +66,27 @@ namespace DB
, client{std::make_shared<Poco::Redis::Client>(host, port)}
{
if (dict_struct.attributes.size() != 1)
throw Exception{"Invalid number of non key columns for Redis source: " +
DB::toString(dict_struct.attributes.size()) + ", expected 1",
ErrorCodes::INVALID_CONFIG_PARAMETER};
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
"Invalid number of non key columns for Redis source: {}, expected 1",
DB::toString(dict_struct.attributes.size()));
if (storage_type == RedisStorageType::HASH_MAP)
{
if (!dict_struct.key)
throw Exception{"Redis source with storage type \'hash_map\' must have key",
ErrorCodes::INVALID_CONFIG_PARAMETER};
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
"Redis source with storage type \'hash_map\' must have key");
if (dict_struct.key->size() != 2)
throw Exception{"Redis source with storage type \'hash_map\' requires 2 keys",
ErrorCodes::INVALID_CONFIG_PARAMETER};
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
"Redis source with storage type \'hash_map\' requires 2 keys");
// suppose key[0] is primary key, key[1] is secondary key
for (const auto & key : *dict_struct.key)
if (!isInteger(key.type) && !isString(key.type))
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
"Redis source supports only integer or string key, but key '{}' of type {} given", key.name, key.type->getName());
"Redis source supports only integer or string key, but key '{}' of type {} given",
key.name,
key.type->getName());
}
if (!password.empty())
@ -93,8 +95,9 @@ namespace DB
command << password;
String reply = client->execute<String>(command);
if (reply != "OK")
throw Exception{"Authentication failed with reason "
+ reply, ErrorCodes::INTERNAL_REDIS_ERROR};
throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR,
"Authentication failed with reason {}",
reply);
}
if (db_index != 0)
@ -103,8 +106,10 @@ namespace DB
command << std::to_string(db_index);
String reply = client->execute<String>(command);
if (reply != "OK")
throw Exception{"Selecting database with index " + DB::toString(db_index)
+ " failed with reason " + reply, ErrorCodes::INTERNAL_REDIS_ERROR};
throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR,
"Selecting database with index {} failed with reason {}",
DB::toString(db_index),
reply);
}
}
@ -215,10 +220,10 @@ namespace DB
client->connect(host, port);
if (storage_type == RedisStorageType::HASH_MAP)
throw Exception{"Cannot use loadIds with 'hash_map' storage type", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use loadIds with 'hash_map' storage type");
if (!dict_struct.id)
throw Exception{"'id' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is required for selective loading");
RedisArray keys;
@ -234,7 +239,7 @@ namespace DB
client->connect(host, port);
if (key_columns.size() != dict_struct.key->size())
throw Exception{"The size of key_columns does not equal to the size of dictionary key", ErrorCodes::LOGICAL_ERROR};
throw Exception(ErrorCodes::LOGICAL_ERROR, "The size of key_columns does not equal to the size of dictionary key");
RedisArray keys;
for (auto row : requested_rows)
@ -268,7 +273,7 @@ namespace DB
if (storage_type_str == "hash_map")
return RedisStorageType::HASH_MAP;
else if (!storage_type_str.empty() && storage_type_str != "simple")
throw Exception("Unknown storage type " + storage_type_str + " for Redis dictionary", ErrorCodes::INVALID_CONFIG_PARAMETER);
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Unknown storage type {} for Redis dictionary", storage_type_str);
return RedisStorageType::SIMPLE;
}

View File

@ -63,7 +63,7 @@ namespace ErrorCodes
BlockInputStreamPtr loadUpdatedAll() override
{
throw Exception{"Method loadUpdatedAll is unsupported for RedisDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for RedisDictionarySource");
}
bool supportsSelectiveLoad() const override { return true; }

View File

@ -465,7 +465,7 @@ public:
/// If cache file is in directory that does not exists create it
if (!std::filesystem::exists(parent_path_directory))
if (!std::filesystem::create_directories(parent_path_directory))
throw Exception{"Failed to create directories.", ErrorCodes::CANNOT_CREATE_DIRECTORY};
throw Exception(ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories.");
ProfileEvents::increment(ProfileEvents::FileOpen);
@ -514,7 +514,7 @@ public:
while (io_submit(aio_context.ctx, 1, &write_request_ptr) < 0)
{
if (errno != EINTR)
throw Exception("Cannot submit request for asynchronous IO on file " + file_path, ErrorCodes::CANNOT_IO_SUBMIT);
throw Exception(ErrorCodes::CANNOT_IO_SUBMIT, "Cannot submit request for asynchronous IO on file {}", file_path);
}
// CurrentMetrics::Increment metric_increment_write{CurrentMetrics::Write};
@ -524,7 +524,7 @@ public:
while (io_getevents(aio_context.ctx, 1, 1, &event, nullptr) < 0)
{
if (errno != EINTR)
throw Exception("Failed to wait for asynchronous IO completion on file " + file_path, ErrorCodes::CANNOT_IO_GETEVENTS);
throw Exception(ErrorCodes::CANNOT_IO_GETEVENTS, "Failed to wait for asynchronous IO completion on file {}", file_path);
}
// Unpoison the memory returned from an uninstrumented system function.
@ -536,7 +536,10 @@ public:
ProfileEvents::increment(ProfileEvents::WriteBufferAIOWriteBytes, bytes_written);
if (bytes_written != static_cast<decltype(bytes_written)>(block_size * buffer_size_in_blocks))
throw Exception("Not all data was written for asynchronous IO on file " + file_path + ". returned: " + std::to_string(bytes_written), ErrorCodes::AIO_WRITE_ERROR);
throw Exception(ErrorCodes::AIO_WRITE_ERROR,
"Not all data was written for asynchronous IO on file {}. returned: {}",
file_path,
std::to_string(bytes_written));
if (::fsync(file.fd) < 0)
throwFromErrnoWithPath("Cannot fsync " + file_path, file_path, ErrorCodes::CANNOT_FSYNC);
@ -593,7 +596,10 @@ public:
if (read_bytes != static_cast<ssize_t>(buffer_size_in_bytes))
throw Exception(ErrorCodes::AIO_READ_ERROR,
"GC: AIO failed to read file ({}). Expected bytes ({}). Actual bytes ({})", file_path, buffer_size_in_bytes, read_bytes);
"GC: AIO failed to read file {}. Expected bytes {}. Actual bytes {}",
file_path,
buffer_size_in_bytes,
read_bytes);
SSDCacheBlock block(block_size);
@ -694,7 +700,10 @@ public:
{
std::string calculated_check_sum = std::to_string(block.calculateCheckSum());
std::string check_sum = std::to_string(block.getCheckSum());
throw Exception("Cache data corrupted. Checksum validation failed. Calculated " + calculated_check_sum + " in block " + check_sum, ErrorCodes::CORRUPTED_DATA);
throw Exception(ErrorCodes::CORRUPTED_DATA,
"Cache data corrupted. Checksum validation failed. Calculated {} in block {}",
calculated_check_sum,
check_sum);
}
std::forward<FetchBlockFunc>(func)(blocks_to_fetch[block_to_fetch_index], block.getBlockData());
@ -835,7 +844,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
else
throw Exception("Method insertColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
}
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
@ -843,7 +852,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception("Method insertColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
}
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
@ -851,7 +860,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
insertDefaultKeysImpl(keys);
else
throw Exception("Method insertDefaultKeysImpl is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
}
PaddedPODArray<UInt64> getCachedSimpleKeys() const override
@ -859,7 +868,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
return getCachedKeysImpl();
else
throw Exception("Method getCachedSimpleKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage");
}
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; }
@ -871,7 +880,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, fetch_request);
else
throw Exception("Method fetchColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage");
}
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
@ -879,7 +888,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception("Method insertColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage");
}
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
@ -887,7 +896,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
insertDefaultKeysImpl(keys);
else
throw Exception("Method insertDefaultKeysImpl is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
}
PaddedPODArray<StringRef> getCachedComplexKeys() const override
@ -895,7 +904,7 @@ public:
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
return getCachedKeysImpl();
else
throw Exception("Method getCachedSimpleKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for simple key storage");
}
size_t getSize() const override { return index.size(); }
@ -1108,7 +1117,7 @@ private:
SSDCacheKeyType ssd_cache_key { key, allocated_size_for_columns, block_start };
if (!SSDCacheBlock::canBeWrittenInEmptyBlock(ssd_cache_key, configuration.block_size))
throw Exception("Serialized columns size is greater than allowed block size and metadata", ErrorCodes::UNSUPPORTED_METHOD);
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Serialized columns size is greater than allowed block size and metadata");
/// We cannot reuse place that is already allocated in file or memory cache so we erase key from index
eraseKeyFromIndex(key);

View File

@ -88,9 +88,9 @@ namespace
else
{
if (!schema.empty())
throw Exception{"Dictionary source of type " + bridge_.getName() + " specifies a schema but schema is not supported by "
+ bridge_.getName() + "-driver",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Dictionary source of type {0} specifies a schema but schema is not supported by {0}-driver",
bridge_.getName());
}
return {dict_struct_, db_, schema, table, where_, bridge_.getIdentifierQuotingStyle()};
@ -287,8 +287,8 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
(void)config_prefix;
(void)sample_block;
(void)context;
throw Exception{"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.");
#endif
};
factory.registerSource("odbc", create_table_source);
@ -303,8 +303,8 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
ContextPtr /* context */,
const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr {
throw Exception{"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.",
ErrorCodes::SUPPORT_IS_DISABLED};
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.");
// BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string"));
// return std::make_unique<XDBCDictionarySource>(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge);
};

View File

@ -299,8 +299,8 @@ void buildPrimaryKeyConfiguration(
if (!complex)
{
if (key_names.size() != 1)
throw Exception("Primary key for simple dictionary must contain exactly one element",
ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION,
"Primary key for simple dictionary must contain exactly one element");
AutoPtr<Element> id_element(doc->createElement("id"));
root->appendChild(id_element);
@ -317,8 +317,8 @@ void buildPrimaryKeyConfiguration(
else
{
if (children.size() < key_names.size())
throw Exception(
"Primary key fields count is more, than dictionary attributes count.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION,
"Primary key fields count is more, than dictionary attributes count.");
AutoPtr<Element> key_element(doc->createElement("key"));
root->appendChild(key_element);
@ -336,8 +336,9 @@ void buildPrimaryKeyConfiguration(
}
}
if (!found)
throw Exception(
"Primary key field '" + key_name + "' not found among attributes.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION,
"Primary key field '{}' not found among attributes.",
key_name);
}
}
}
@ -358,7 +359,7 @@ NamesToTypeNames buildDictionaryAttributesConfiguration(
{
const ASTDictionaryAttributeDeclaration * dict_attr = child->as<const ASTDictionaryAttributeDeclaration>();
if (!dict_attr->type)
throw Exception("Dictionary attribute must has type", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Dictionary attribute must has type");
attributes_names_and_types.emplace(dict_attr->name, queryToString(dict_attr->type));
if (std::find(key_columns.begin(), key_columns.end(), dict_attr->name) == key_columns.end())
@ -416,9 +417,8 @@ void buildConfigurationFromFunctionWithKeyValueArguments(
}
else
{
throw Exception(
"Incorrect ASTPair contains wrong value, should be literal, identifier or list",
ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION,
"Incorrect ASTPair contains wrong value, should be literal, identifier or list");
}
}
}
@ -470,24 +470,24 @@ void buildSourceConfiguration(
void checkAST(const ASTCreateQuery & query)
{
if (!query.is_dictionary || query.dictionary == nullptr)
throw Exception("Cannot convert dictionary to configuration from non-dictionary AST.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot convert dictionary to configuration from non-dictionary AST.");
if (query.dictionary_attributes_list == nullptr || query.dictionary_attributes_list->children.empty())
throw Exception("Cannot create dictionary with empty attributes list", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty attributes list");
if (query.dictionary->layout == nullptr)
throw Exception("Cannot create dictionary with empty layout", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty layout");
const auto is_direct_layout = !strcasecmp(query.dictionary->layout->layout_type.data(), "direct") ||
!strcasecmp(query.dictionary->layout->layout_type.data(), "complex_key_direct");
if (query.dictionary->lifetime == nullptr && !is_direct_layout)
throw Exception("Cannot create dictionary with empty lifetime", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty lifetime");
if (query.dictionary->primary_key == nullptr)
throw Exception("Cannot create dictionary without primary key", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary without primary key");
if (query.dictionary->source == nullptr)
throw Exception("Cannot create dictionary with empty source", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty source");
/// Range can be empty
}
@ -496,7 +496,7 @@ void checkPrimaryKey(const NamesToTypeNames & all_attrs, const Names & key_attrs
{
for (const auto & key_attr : key_attrs)
if (all_attrs.count(key_attr) == 0)
throw Exception("Unknown key attribute '" + key_attr + "'", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Unknown key attribute '{}'", key_attr);
}
}

View File

@ -20,17 +20,17 @@ std::string readInvalidateQuery(IBlockInputStream & block_input_stream)
Block block = block_input_stream.read();
if (!block)
throw Exception("Empty response", ErrorCodes::RECEIVED_EMPTY_DATA);
throw Exception(ErrorCodes::RECEIVED_EMPTY_DATA, "Empty response");
auto columns = block.columns();
if (columns > 1)
throw Exception("Expected single column in resultset, got " + std::to_string(columns), ErrorCodes::TOO_MANY_COLUMNS);
throw Exception(ErrorCodes::TOO_MANY_COLUMNS, "Expected single column in resultset, got {}", std::to_string(columns));
auto rows = block.rows();
if (rows == 0)
throw Exception("Expected single row in resultset, got 0", ErrorCodes::RECEIVED_EMPTY_DATA);
throw Exception(ErrorCodes::RECEIVED_EMPTY_DATA, "Expected single row in resultset, got 0");
if (rows > 1)
throw Exception("Expected single row in resultset, got at least " + std::to_string(rows), ErrorCodes::TOO_MANY_ROWS);
throw Exception(ErrorCodes::TOO_MANY_ROWS, "Expected single row in resultset, got at least {}", std::to_string(rows));
WriteBufferFromOwnString out;
auto & column_type = block.getByPosition(0);
@ -38,7 +38,7 @@ std::string readInvalidateQuery(IBlockInputStream & block_input_stream)
while ((block = block_input_stream.read()))
if (block.rows() > 0)
throw Exception("Expected single row in resultset, got at least " + std::to_string(rows + 1), ErrorCodes::TOO_MANY_ROWS);
throw Exception(ErrorCodes::TOO_MANY_ROWS, "Expected single row in resultset, got at least {}", std::to_string(rows + 1));
block_input_stream.readSuffix();
return out.str();

View File

@ -26,7 +26,7 @@ CacheDictionaryStorageConfiguration parseCacheStorageConfiguration(
const size_t size = config.getUInt64(dictionary_configuration_prefix + "size_in_cells");
if (size == 0)
throw Exception(ErrorCodes::TOO_SMALL_BUFFER_SIZE,
"({}): cache dictionary cannot have 0 cells",
"{}: cache dictionary cannot have 0 cells",
full_name);
size_t dict_lifetime_seconds = static_cast<size_t>(dict_lifetime.max_sec);
@ -68,26 +68,26 @@ SSDCacheDictionaryStorageConfiguration parseSSDCacheStorageConfiguration(
const size_t file_size = config.getInt64(dictionary_configuration_prefix + "file_size", DEFAULT_FILE_SIZE_BYTES);
if (file_size % block_size != 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): file_size must be a multiple of block_size",
"{}: file_size must be a multiple of block_size",
full_name);
const size_t read_buffer_size = config.getInt64(dictionary_configuration_prefix + "read_buffer_size", DEFAULT_READ_BUFFER_SIZE_BYTES);
if (read_buffer_size % block_size != 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): read_buffer_size must be a multiple of block_size",
"{}: read_buffer_size must be a multiple of block_size",
full_name);
const size_t write_buffer_size
= config.getInt64(dictionary_configuration_prefix + "write_buffer_size", DEFAULT_WRITE_BUFFER_SIZE_BYTES);
if (write_buffer_size % block_size != 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): write_buffer_size must be a multiple of block_size",
"{}: write_buffer_size must be a multiple of block_size",
full_name);
auto directory_path = config.getString(dictionary_configuration_prefix + "path");
if (directory_path.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): ssd cache dictionary cannot have empty path",
"{}: ssd cache dictionary cannot have empty path",
full_name);
if (directory_path.at(0) != '/')
@ -119,7 +119,7 @@ CacheDictionaryUpdateQueueConfiguration parseCacheDictionaryUpdateQueueConfigura
const size_t max_update_queue_size = config.getUInt64(layout_prefix + ".cache.max_update_queue_size", 100000);
if (max_update_queue_size == 0)
throw Exception(ErrorCodes::TOO_SMALL_BUFFER_SIZE,
"({}): dictionary of layout '({})' cannot have empty update queue of size 0",
"{}: dictionary of layout '{}' cannot have empty update queue of size 0",
full_name,
layout_type);
@ -127,7 +127,7 @@ CacheDictionaryUpdateQueueConfiguration parseCacheDictionaryUpdateQueueConfigura
= config.getUInt64(layout_prefix + ".cache.update_queue_push_timeout_milliseconds", 10);
if (update_queue_push_timeout_milliseconds < 10)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): dictionary of layout '({})' have too little update_queue_push_timeout",
"{}: dictionary of layout '{}' have too little update_queue_push_timeout",
full_name,
layout_type);
@ -136,7 +136,7 @@ CacheDictionaryUpdateQueueConfiguration parseCacheDictionaryUpdateQueueConfigura
const size_t max_threads_for_updates = config.getUInt64(layout_prefix + ".max_threads_for_updates", 4);
if (max_threads_for_updates == 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): dictionary of layout) '({})' cannot have zero threads for updates",
"{}: dictionary of layout) '{}' cannot have zero threads for updates",
full_name,
layout_type);
@ -169,14 +169,14 @@ DictionaryPtr createCacheDictionaryLayout(
if (dict_struct.range_min || dict_struct.range_max)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
"{}: elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
full_name);
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
if (require_nonempty)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): cache dictionary of layout cannot have 'require_nonempty' attribute set",
"{}: cache dictionary of layout cannot have 'require_nonempty' attribute set",
full_name);
const auto & layout_prefix = config_prefix + ".layout";
@ -222,14 +222,14 @@ DictionaryPtr createSSDCacheDictionaryLayout(
if (dict_struct.range_min || dict_struct.range_max)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
"{}: elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
full_name);
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
if (require_nonempty)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): cache dictionary of layout cannot have 'require_nonempty' attribute set",
"{}: cache dictionary of layout cannot have 'require_nonempty' attribute set",
full_name);
const auto & layout_prefix = config_prefix + ".layout";

View File

@ -3,12 +3,14 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/TransformDateTime64.h>
#include <IO/WriteHelpers.h>
@ -42,6 +44,7 @@ namespace
*/
class FunctionDateDiff : public IFunction
{
using ColumnDateTime64 = ColumnDecimal<DateTime64>;
public:
static constexpr auto name = "dateDiff";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionDateDiff>(); }
@ -133,17 +136,21 @@ private:
dispatchForSecondColumn<Transform>(*x_vec_16, y, timezone_x, timezone_y, result);
else if (const auto * x_vec_32 = checkAndGetColumn<ColumnUInt32>(&x))
dispatchForSecondColumn<Transform>(*x_vec_32, y, timezone_x, timezone_y, result);
else if (const auto * x_vec_64 = checkAndGetColumn<ColumnDateTime64>(&x))
dispatchForSecondColumn<Transform>(*x_vec_64, y, timezone_x, timezone_y, result);
else if (const auto * x_const_16 = checkAndGetColumnConst<ColumnUInt16>(&x))
dispatchConstForSecondColumn<Transform>(x_const_16->getValue<UInt16>(), y, timezone_x, timezone_y, result);
else if (const auto * x_const_32 = checkAndGetColumnConst<ColumnUInt32>(&x))
dispatchConstForSecondColumn<Transform>(x_const_32->getValue<UInt32>(), y, timezone_x, timezone_y, result);
else if (const auto * x_const_64 = checkAndGetColumnConst<ColumnDateTime64>(&x))
dispatchConstForSecondColumn<Transform>(x_const_64->getValue<DecimalField<DateTime64>>(), y, timezone_x, timezone_y, result);
else
throw Exception("Illegal column for first argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN);
throw Exception("Illegal column for first argument of function " + getName() + ", must be Date, DateTime or DateTime64", ErrorCodes::ILLEGAL_COLUMN);
}
template <typename Transform, typename T1>
template <typename Transform, typename LeftColumnType>
void dispatchForSecondColumn(
const ColumnVector<T1> & x, const IColumn & y,
const LeftColumnType & x, const IColumn & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result) const
{
@ -151,10 +158,14 @@ private:
vectorVector<Transform>(x, *y_vec_16, timezone_x, timezone_y, result);
else if (const auto * y_vec_32 = checkAndGetColumn<ColumnUInt32>(&y))
vectorVector<Transform>(x, *y_vec_32, timezone_x, timezone_y, result);
else if (const auto * y_vec_64 = checkAndGetColumn<ColumnDateTime64>(&y))
vectorVector<Transform>(x, *y_vec_64, timezone_x, timezone_y, result);
else if (const auto * y_const_16 = checkAndGetColumnConst<ColumnUInt16>(&y))
vectorConstant<Transform>(x, y_const_16->getValue<UInt16>(), timezone_x, timezone_y, result);
else if (const auto * y_const_32 = checkAndGetColumnConst<ColumnUInt32>(&y))
vectorConstant<Transform>(x, y_const_32->getValue<UInt32>(), timezone_x, timezone_y, result);
else if (const auto * y_const_64 = checkAndGetColumnConst<ColumnDateTime64>(&y))
vectorConstant<Transform>(x, y_const_64->getValue<DecimalField<DateTime64>>(), timezone_x, timezone_y, result);
else
throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN);
}
@ -169,49 +180,81 @@ private:
constantVector<Transform>(x, *y_vec_16, timezone_x, timezone_y, result);
else if (const auto * y_vec_32 = checkAndGetColumn<ColumnUInt32>(&y))
constantVector<Transform>(x, *y_vec_32, timezone_x, timezone_y, result);
else if (const auto * y_vec_64 = checkAndGetColumn<ColumnDateTime64>(&y))
constantVector<Transform>(x, *y_vec_64, timezone_x, timezone_y, result);
else
throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN);
}
template <typename Transform, typename T1, typename T2>
template <typename Transform, typename LeftColumnType, typename RightColumnType>
void vectorVector(
const ColumnVector<T1> & x, const ColumnVector<T2> & y,
const LeftColumnType & x, const RightColumnType & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result) const
{
const auto & x_data = x.getData();
const auto & y_data = y.getData();
const auto transform_x = TransformDateTime64<Transform>(getScale(x));
const auto transform_y = TransformDateTime64<Transform>(getScale(y));
for (size_t i = 0, size = x.size(); i < size; ++i)
result[i] = calculate<Transform>(x_data[i], y_data[i], timezone_x, timezone_y);
result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y);
}
template <typename Transform, typename T1, typename T2>
template <typename Transform, typename LeftColumnType, typename T2>
void vectorConstant(
const ColumnVector<T1> & x, T2 y,
const LeftColumnType & x, T2 y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result) const
{
const auto & x_data = x.getData();
const auto transform_x = TransformDateTime64<Transform>(getScale(x));
const auto transform_y = TransformDateTime64<Transform>(getScale(y));
const auto y_value = stripDecimalFieldValue(y);
for (size_t i = 0, size = x.size(); i < size; ++i)
result[i] = calculate<Transform>(x_data[i], y, timezone_x, timezone_y);
result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y);
}
template <typename Transform, typename T1, typename T2>
template <typename Transform, typename T1, typename RightColumnType>
void constantVector(
T1 x, const ColumnVector<T2> & y,
T1 x, const RightColumnType & y,
const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y,
ColumnInt64::Container & result) const
{
const auto & y_data = y.getData();
const auto transform_x = TransformDateTime64<Transform>(getScale(x));
const auto transform_y = TransformDateTime64<Transform>(getScale(y));
const auto x_value = stripDecimalFieldValue(x);
for (size_t i = 0, size = y.size(); i < size; ++i)
result[i] = calculate<Transform>(x, y_data[i], timezone_x, timezone_y);
result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y);
}
template <typename Transform, typename T1, typename T2>
Int64 calculate(T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const
template <typename TransformX, typename TransformY, typename T1, typename T2>
Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const
{
return Int64(Transform::execute(y, timezone_y))
- Int64(Transform::execute(x, timezone_x));
return Int64(transform_y.execute(y, timezone_y))
- Int64(transform_x.execute(x, timezone_x));
}
template <typename T>
static UInt32 getScale(const T & v)
{
if constexpr (std::is_same_v<T, ColumnDateTime64>)
return v.getScale();
else if constexpr (std::is_same_v<T, DecimalField<DateTime64>>)
return v.getScale();
return 0;
}
template <typename T>
static auto stripDecimalFieldValue(T && v)
{
if constexpr (std::is_same_v<std::decay_t<T>, DecimalField<DateTime64>>)
return v.getValue();
else
return v;
}
};

View File

@ -56,6 +56,8 @@
#include <boost/range/adaptor/filtered.hpp>
#include <boost/algorithm/string/join.hpp>
#include <ext/scope_guard_safe.h>
#include <algorithm>
#include <iomanip>
#include <optional>
@ -1211,14 +1213,21 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re
{
/// Parallel parts removal.
size_t num_threads = std::min(size_t(settings->max_part_removal_threads), parts_to_remove.size());
size_t num_threads = std::min<size_t>(settings->max_part_removal_threads, parts_to_remove.size());
ThreadPool pool(num_threads);
/// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool.
for (const DataPartPtr & part : parts_to_remove)
{
pool.scheduleOrThrowOnError([&]
pool.scheduleOrThrowOnError([&, thread_group = CurrentThread::getGroup()]
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);
if (thread_group)
CurrentThread::attachTo(thread_group);
LOG_DEBUG(log, "Removing part from filesystem {}", part->name);
part->remove();
});

View File

@ -5701,6 +5701,56 @@ CancellationCode StorageReplicatedMergeTree::killMutation(const String & mutatio
return CancellationCode::CancelSent;
}
void StorageReplicatedMergeTree::removePartsFromFilesystem(const DataPartsVector & parts)
{
auto remove_part = [&](const auto & part)
{
LOG_DEBUG(log, "Removing part from filesystem {}", part.name);
try
{
bool keep_s3 = !this->unlockSharedData(part);
part.remove(keep_s3);
}
catch (...)
{
tryLogCurrentException(log, "There is a problem with deleting part " + part.name + " from filesystem");
}
};
const auto settings = getSettings();
if (settings->max_part_removal_threads > 1 && parts.size() > settings->concurrent_part_removal_threshold)
{
/// Parallel parts removal.
size_t num_threads = std::min<size_t>(settings->max_part_removal_threads, parts.size());
ThreadPool pool(num_threads);
/// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool.
for (const DataPartPtr & part : parts)
{
pool.scheduleOrThrowOnError([&, thread_group = CurrentThread::getGroup()]
{
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);
if (thread_group)
CurrentThread::attachTo(thread_group);
remove_part(*part);
});
}
pool.wait();
}
else
{
for (const DataPartPtr & part : parts)
{
remove_part(*part);
}
}
}
void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK()
{
@ -5726,26 +5776,10 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK()
}
parts.clear();
auto remove_parts_from_filesystem = [log=log, this] (const DataPartsVector & parts_to_remove)
{
for (const auto & part : parts_to_remove)
{
try
{
bool keep_s3 = !this->unlockSharedData(*part);
part->remove(keep_s3);
}
catch (...)
{
tryLogCurrentException(log, "There is a problem with deleting part " + part->name + " from filesystem");
}
}
};
/// Delete duplicate parts from filesystem
if (!parts_to_delete_only_from_filesystem.empty())
{
remove_parts_from_filesystem(parts_to_delete_only_from_filesystem);
removePartsFromFilesystem(parts_to_delete_only_from_filesystem);
removePartsFinally(parts_to_delete_only_from_filesystem);
LOG_DEBUG(log, "Removed {} old duplicate parts", parts_to_delete_only_from_filesystem.size());
@ -5790,7 +5824,7 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK()
/// Remove parts from filesystem and finally from data_parts
if (!parts_to_remove_from_filesystem.empty())
{
remove_parts_from_filesystem(parts_to_remove_from_filesystem);
removePartsFromFilesystem(parts_to_remove_from_filesystem);
removePartsFinally(parts_to_remove_from_filesystem);
LOG_DEBUG(log, "Removed {} old parts", parts_to_remove_from_filesystem.size());

View File

@ -408,6 +408,8 @@ private:
/// Just removes part from ZooKeeper using previous method
void removePartFromZooKeeper(const String & part_name);
void removePartsFromFilesystem(const DataPartsVector & parts);
/// Quickly removes big set of parts from ZooKeeper (using async multi queries)
void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
NameSet * parts_should_be_retried = nullptr);

View File

@ -118,6 +118,7 @@ SRCS(
StorageBuffer.cpp
StorageDictionary.cpp
StorageDistributed.cpp
StorageExternalDistributed.cpp
StorageFactory.cpp
StorageFile.cpp
StorageGenerateRandom.cpp

View File

@ -0,0 +1,113 @@
-- { echo }
-- DateTime64 vs DateTime64 same scale
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 00:00:10', 0, 'UTC'));
10
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 00:10:00', 0, 'UTC'));
600
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 01:00:00', 0, 'UTC'));
3600
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 01:10:10', 0, 'UTC'));
4210
SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 00:10:00', 0, 'UTC'));
10
SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 10:00:00', 0, 'UTC'));
600
SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 10:00:00', 0, 'UTC'));
10
SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-02 00:00:00', 0, 'UTC'));
1
SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-02-01 00:00:00', 0, 'UTC'));
1
SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1928-01-01 00:00:00', 0, 'UTC'));
1
-- DateTime64 vs DateTime64 different scale
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 00:00:10', 3, 'UTC'));
10
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 00:10:00', 3, 'UTC'));
600
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 01:00:00', 3, 'UTC'));
3600
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 01:10:10', 3, 'UTC'));
4210
SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 00:10:00', 3, 'UTC'));
10
SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 10:00:00', 3, 'UTC'));
600
SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 10:00:00', 3, 'UTC'));
10
SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'));
1
SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-02-01 00:00:00', 3, 'UTC'));
1
SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1928-01-01 00:00:00', 3, 'UTC'));
1
-- With DateTime
-- DateTime64 vs DateTime
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 00:00:00', 'UTC'));
0
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 00:00:10', 'UTC'));
10
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 00:10:00', 'UTC'));
600
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 01:00:00', 'UTC'));
3600
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 01:10:10', 'UTC'));
4210
-- DateTime vs DateTime64
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 00:00:00', 3, 'UTC'));
0
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 00:00:10', 3, 'UTC'));
10
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 00:10:00', 3, 'UTC'));
600
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 01:00:00', 3, 'UTC'));
3600
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 01:10:10', 3, 'UTC'));
4210
-- With Date
-- DateTime64 vs Date
SELECT dateDiff('day', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDate('2015-08-19', 'UTC'));
1
-- Date vs DateTime64
SELECT dateDiff('day', toDate('2015-08-18', 'UTC'), toDateTime64('2015-08-19 00:00:00', 3, 'UTC'));
1
-- Same thing but const vs non-const columns
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), materialize(toDateTime64('1927-01-01 00:00:10', 0, 'UTC')));
10
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), materialize(toDateTime64('1927-01-01 00:00:10', 3, 'UTC')));
10
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), materialize(toDateTime('2015-08-18 00:00:10', 'UTC')));
10
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC')));
10
SELECT dateDiff('day', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), materialize(toDate('2015-08-19', 'UTC')));
1
SELECT dateDiff('day', toDate('2015-08-18', 'UTC'), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC')));
1
-- Same thing but non-const vs const columns
SELECT dateDiff('second', materialize(toDateTime64('1927-01-01 00:00:00', 0, 'UTC')), toDateTime64('1927-01-01 00:00:10', 0, 'UTC'));
10
SELECT dateDiff('second', materialize(toDateTime64('1927-01-01 00:00:00', 6, 'UTC')), toDateTime64('1927-01-01 00:00:10', 3, 'UTC'));
10
SELECT dateDiff('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), toDateTime('2015-08-18 00:00:10', 'UTC'));
10
SELECT dateDiff('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), toDateTime64('2015-08-18 00:00:10', 3, 'UTC'));
10
SELECT dateDiff('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), toDate('2015-08-19', 'UTC'));
1
SELECT dateDiff('day', materialize(toDate('2015-08-18', 'UTC')), toDateTime64('2015-08-19 00:00:00', 3, 'UTC'));
1
-- Same thing but non-const vs non-const columns
SELECT dateDiff('second', materialize(toDateTime64('1927-01-01 00:00:00', 0, 'UTC')), materialize(toDateTime64('1927-01-01 00:00:10', 0, 'UTC')));
10
SELECT dateDiff('second', materialize(toDateTime64('1927-01-01 00:00:00', 6, 'UTC')), materialize(toDateTime64('1927-01-01 00:00:10', 3, 'UTC')));
10
SELECT dateDiff('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDateTime('2015-08-18 00:00:10', 'UTC')));
10
SELECT dateDiff('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC')));
10
SELECT dateDiff('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC')));
1
SELECT dateDiff('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC')));
1

View File

@ -0,0 +1,77 @@
-- { echo }
-- DateTime64 vs DateTime64 same scale
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 00:00:10', 0, 'UTC'));
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 00:10:00', 0, 'UTC'));
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 01:00:00', 0, 'UTC'));
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 01:10:10', 0, 'UTC'));
SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 00:10:00', 0, 'UTC'));
SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 10:00:00', 0, 'UTC'));
SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-01 10:00:00', 0, 'UTC'));
SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-01-02 00:00:00', 0, 'UTC'));
SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1927-02-01 00:00:00', 0, 'UTC'));
SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), toDateTime64('1928-01-01 00:00:00', 0, 'UTC'));
-- DateTime64 vs DateTime64 different scale
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 00:00:10', 3, 'UTC'));
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 00:10:00', 3, 'UTC'));
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 01:00:00', 3, 'UTC'));
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 01:10:10', 3, 'UTC'));
SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 00:10:00', 3, 'UTC'));
SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 10:00:00', 3, 'UTC'));
SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-01 10:00:00', 3, 'UTC'));
SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'));
SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1927-02-01 00:00:00', 3, 'UTC'));
SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), toDateTime64('1928-01-01 00:00:00', 3, 'UTC'));
-- With DateTime
-- DateTime64 vs DateTime
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 00:00:00', 'UTC'));
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 00:00:10', 'UTC'));
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 00:10:00', 'UTC'));
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 01:00:00', 'UTC'));
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDateTime('2015-08-18 01:10:10', 'UTC'));
-- DateTime vs DateTime64
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 00:00:00', 3, 'UTC'));
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 00:00:10', 3, 'UTC'));
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 00:10:00', 3, 'UTC'));
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 01:00:00', 3, 'UTC'));
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDateTime64('2015-08-18 01:10:10', 3, 'UTC'));
-- With Date
-- DateTime64 vs Date
SELECT dateDiff('day', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), toDate('2015-08-19', 'UTC'));
-- Date vs DateTime64
SELECT dateDiff('day', toDate('2015-08-18', 'UTC'), toDateTime64('2015-08-19 00:00:00', 3, 'UTC'));
-- Same thing but const vs non-const columns
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 0, 'UTC'), materialize(toDateTime64('1927-01-01 00:00:10', 0, 'UTC')));
SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 6, 'UTC'), materialize(toDateTime64('1927-01-01 00:00:10', 3, 'UTC')));
SELECT dateDiff('second', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), materialize(toDateTime('2015-08-18 00:00:10', 'UTC')));
SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC')));
SELECT dateDiff('day', toDateTime64('2015-08-18 00:00:00', 0, 'UTC'), materialize(toDate('2015-08-19', 'UTC')));
SELECT dateDiff('day', toDate('2015-08-18', 'UTC'), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC')));
-- Same thing but non-const vs const columns
SELECT dateDiff('second', materialize(toDateTime64('1927-01-01 00:00:00', 0, 'UTC')), toDateTime64('1927-01-01 00:00:10', 0, 'UTC'));
SELECT dateDiff('second', materialize(toDateTime64('1927-01-01 00:00:00', 6, 'UTC')), toDateTime64('1927-01-01 00:00:10', 3, 'UTC'));
SELECT dateDiff('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), toDateTime('2015-08-18 00:00:10', 'UTC'));
SELECT dateDiff('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), toDateTime64('2015-08-18 00:00:10', 3, 'UTC'));
SELECT dateDiff('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), toDate('2015-08-19', 'UTC'));
SELECT dateDiff('day', materialize(toDate('2015-08-18', 'UTC')), toDateTime64('2015-08-19 00:00:00', 3, 'UTC'));
-- Same thing but non-const vs non-const columns
SELECT dateDiff('second', materialize(toDateTime64('1927-01-01 00:00:00', 0, 'UTC')), materialize(toDateTime64('1927-01-01 00:00:10', 0, 'UTC')));
SELECT dateDiff('second', materialize(toDateTime64('1927-01-01 00:00:00', 6, 'UTC')), materialize(toDateTime64('1927-01-01 00:00:10', 3, 'UTC')));
SELECT dateDiff('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDateTime('2015-08-18 00:00:10', 'UTC')));
SELECT dateDiff('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC')));
SELECT dateDiff('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC')));
SELECT dateDiff('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC')));

View File

@ -1,30 +0,0 @@
DROP TABLE IF EXISTS data_01802;
DROP DICTIONARY IF EXISTS dict_01802;
CREATE TABLE data_01802
(
id UInt64,
value UInt16
)
ENGINE = Memory();
INSERT INTO data_01802 VALUES(0, 0);
INSERT INTO data_01802 VALUES(1, 0);
INSERT INTO data_01802 VALUES(2, 0);
CREATE DICTIONARY dict_01802
(
id UInt64,
value UInt16
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'data_01802'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(SPARSE_HASHED());
SYSTEM RELOAD DICTIONARY dict_01802;
SELECT bytes_allocated FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_01802';
DROP TABLE data_01802;
DROP DICTIONARY dict_01802;

View File

@ -0,0 +1,14 @@
Flat dictionary
5.00000
Hashed dictionary
5.00000
Cache dictionary
5.00000
SSDCache dictionary
5.00000
Direct dictionary
5.00000
IPTrie dictionary
5.00000
Polygon dictionary
5.00000

View File

@ -0,0 +1,141 @@
SET allow_experimental_bigint_types = 1;
DROP TABLE IF EXISTS dictionary_decimal_source_table;
CREATE TABLE dictionary_decimal_source_table
(
id UInt64,
decimal_value Decimal256(5)
) ENGINE = TinyLog;
INSERT INTO dictionary_decimal_source_table VALUES (1, 5.0);
DROP DICTIONARY IF EXISTS flat_dictionary;
CREATE DICTIONARY flat_dictionary
(
id UInt64,
decimal_value Decimal256(5)
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_decimal_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(FLAT());
SELECT 'Flat dictionary';
SELECT dictGet('flat_dictionary', 'decimal_value', toUInt64(1));
DROP DICTIONARY IF EXISTS hashed_dictionary;
CREATE DICTIONARY hashed_dictionary
(
id UInt64,
decimal_value Decimal256(5)
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_decimal_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(HASHED());
SELECT 'Hashed dictionary';
SELECT dictGet('hashed_dictionary', 'decimal_value', toUInt64(1));
DROP DICTIONARY hashed_dictionary;
DROP DICTIONARY IF EXISTS cache_dictionary;
CREATE DICTIONARY cache_dictionary
(
id UInt64,
decimal_value Decimal256(5)
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_decimal_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(CACHE(SIZE_IN_CELLS 10));
SELECT 'Cache dictionary';
SELECT dictGet('cache_dictionary', 'decimal_value', toUInt64(1));
DROP DICTIONARY cache_dictionary;
DROP DICTIONARY IF EXISTS ssd_cache_dictionary;
CREATE DICTIONARY ssd_cache_dictionary
(
id UInt64,
decimal_value Decimal256(5)
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_decimal_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/0d'));
SELECT 'SSDCache dictionary';
SELECT dictGet('ssd_cache_dictionary', 'decimal_value', toUInt64(1));
DROP DICTIONARY ssd_cache_dictionary;
DROP DICTIONARY IF EXISTS direct_dictionary;
CREATE DICTIONARY direct_dictionary
(
id UInt64,
decimal_value Decimal256(5)
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_decimal_source_table'))
LAYOUT(DIRECT());
SELECT 'Direct dictionary';
SELECT dictGet('direct_dictionary', 'decimal_value', toUInt64(1));
DROP DICTIONARY direct_dictionary;
DROP TABLE dictionary_decimal_source_table;
DROP TABLE IF EXISTS ip_trie_dictionary_decimal_source_table;
CREATE TABLE ip_trie_dictionary_decimal_source_table
(
prefix String,
decimal_value Decimal256(5)
) ENGINE = TinyLog;
INSERT INTO ip_trie_dictionary_decimal_source_table VALUES ('127.0.0.0', 5.0);
DROP DICTIONARY IF EXISTS ip_trie_dictionary;
CREATE DICTIONARY ip_trie_dictionary
(
prefix String,
decimal_value Decimal256(5)
)
PRIMARY KEY prefix
SOURCE(CLICKHOUSE(HOST 'localhost' port tcpPort() TABLE 'ip_trie_dictionary_decimal_source_table'))
LIFETIME(MIN 10 MAX 1000)
LAYOUT(IP_TRIE());
SELECT 'IPTrie dictionary';
SELECT dictGet('ip_trie_dictionary', 'decimal_value', tuple(IPv4StringToNum('127.0.0.0')));
DROP DICTIONARY ip_trie_dictionary;
DROP TABLE ip_trie_dictionary_decimal_source_table;
DROP TABLE IF EXISTS dictionary_decimal_polygons_source_table;
CREATE TABLE dictionary_decimal_polygons_source_table
(
key Array(Array(Array(Tuple(Float64, Float64)))),
decimal_value Decimal256(5)
) ENGINE = TinyLog;
INSERT INTO dictionary_decimal_polygons_source_table VALUES ([[[(0, 0), (0, 1), (1, 1), (1, 0)]]], 5.0);
DROP DICTIONARY IF EXISTS polygon_dictionary;
CREATE DICTIONARY polygon_dictionary
(
key Array(Array(Array(Tuple(Float64, Float64)))),
decimal_value Decimal256(5)
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_decimal_polygons_source_table'))
LIFETIME(MIN 0 MAX 1000)
LAYOUT(POLYGON());
SELECT 'Polygon dictionary';
SELECT dictGet('polygon_dictionary', 'decimal_value', tuple(0.5, 0.5));
DROP DICTIONARY polygon_dictionary;
DROP TABLE dictionary_decimal_polygons_source_table;

View File

@ -0,0 +1,36 @@
#!/usr/bin/env bash
# NOTE: this done as not .sql since we need to Ordinary database
# (to account threads in query_log for DROP TABLE query)
# and we can do it compatible with parallel run only in .sh
# (via $CLICKHOUSE_DATABASE)
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -nm -q "create database ordinary_$CLICKHOUSE_DATABASE engine=Ordinary"
# MergeTree
$CLICKHOUSE_CLIENT -nm -q """
use ordinary_$CLICKHOUSE_DATABASE;
drop table if exists data_01810;
create table data_01810 (key Int) Engine=MergeTree() order by key partition by key settings max_part_removal_threads=10, concurrent_part_removal_threshold=49;
insert into data_01810 select * from numbers(50);
drop table data_01810 settings log_queries=1;
system flush logs;
select throwIf(length(thread_ids)<50) from system.query_log where event_date = today() and current_database = currentDatabase() and query = 'drop table data_01810 settings log_queries=1;' and type = 'QueryFinish' format Null;
"""
# ReplicatedMergeTree
$CLICKHOUSE_CLIENT -nm -q """
use ordinary_$CLICKHOUSE_DATABASE;
drop table if exists rep_data_01810;
create table rep_data_01810 (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rep_data_01810', '1') order by key partition by key settings max_part_removal_threads=10, concurrent_part_removal_threshold=49;
insert into rep_data_01810 select * from numbers(50);
drop table rep_data_01810 settings log_queries=1;
system flush logs;
select throwIf(length(thread_ids)<50) from system.query_log where event_date = today() and current_database = currentDatabase() and query = 'drop table rep_data_01810 settings log_queries=1;' and type = 'QueryFinish' format Null;
"""
$CLICKHOUSE_CLIENT -nm -q "drop database ordinary_$CLICKHOUSE_DATABASE"

View File

@ -228,3 +228,4 @@
01790_dist_INSERT_block_structure_mismatch_types_and_names
01791_dist_INSERT_block_structure_mismatch
01801_distinct_group_by_shard
01804_dictionary_decimal256_type

View File

@ -702,6 +702,7 @@
"01778_hierarchical_dictionaries",
"01780_clickhouse_dictionary_source_loop",
"01785_dictionary_element_count",
"01802_test_postgresql_protocol_with_row_policy" /// Creates database and users
"01802_test_postgresql_protocol_with_row_policy", /// Creates database and users
"01804_dictionary_decimal256_type"
]
}

View File

@ -1,4 +1,4 @@
var current_data_size = 1000000000;
var current_data_size = 100000000;
var current_systems = ["ClickHouse", "Vertica", "Greenplum"];

View File

@ -1,6 +1,6 @@
[
{
"system": "AMD Ryzen 9",
"system": "AMD Ryzen 9 (2020)",
"system_full": "AMD Ryzen 9 3950X 16-Core Processor, 64 GiB RAM, Intel Optane 900P 280 GB",
"time": "2020-03-14 00:00:00",
"kind": "desktop",
@ -52,7 +52,7 @@
]
},
{
"system": "AMD Ryzen 9",
"system": "AMD Ryzen 9 (2021)",
"system_full": "AMD Ryzen 9 3950X 16-Core Processor, 64 GiB RAM, Samsung evo 970 plus 1TB",
"time": "2021-03-08 00:00:00",
"kind": "desktop",