Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2019-01-02 18:03:56 +03:00
commit 051decfb75
13 changed files with 563 additions and 280 deletions

View File

@ -1,11 +1,11 @@
# This strings autochanged from release_lib.sh:
set(VERSION_REVISION 54412 CACHE STRING "") # changed manually for tests
set(VERSION_MAJOR 18 CACHE STRING "")
set(VERSION_MINOR 16 CACHE STRING "")
set(VERSION_REVISION 54413 CACHE STRING "") # changed manually for tests
set(VERSION_MAJOR 19 CACHE STRING "")
set(VERSION_MINOR 1 CACHE STRING "")
set(VERSION_PATCH 0 CACHE STRING "")
set(VERSION_GITHASH b9b48c646c253358340bd39fd57754e92f88cd8a CACHE STRING "")
set(VERSION_DESCRIBE v18.16.0-testing CACHE STRING "")
set(VERSION_STRING 18.16.0 CACHE STRING "")
set(VERSION_GITHASH 014e344a36bc19a58621e0add379984cf62b9067 CACHE STRING "")
set(VERSION_DESCRIBE v19.1.0-testing CACHE STRING "")
set(VERSION_STRING 19.1.0 CACHE STRING "")
# end of autochange
set(VERSION_EXTRA "" CACHE STRING "")

View File

@ -10,6 +10,7 @@
#include <Poco/ByteOrder.h>
#include <Poco/Net/IPAddress.h>
#include <Common/formatIPv6.h>
#include <common/itoa.h>
#include <ext/map.h>
#include <ext/range.h>
#include "DictionaryBlockInputStream.h"
@ -778,8 +779,8 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam
char * ptr = buffer;
formatIPv6(reinterpret_cast<const unsigned char *>(ip_column.getDataAt(row).data), ptr);
*(ptr - 1) = '/';
auto size = detail::writeUIntText(mask, ptr);
column->insertData(buffer, size + (ptr - buffer));
ptr = itoa(mask, ptr);
column->insertData(buffer, ptr - buffer);
}
return ColumnsWithTypeAndName{
ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), attributes.front().name)};

View File

@ -2,221 +2,33 @@
#include <limits>
#include <type_traits>
#include <common/likely.h>
#include <Core/Types.h>
#include <IO/WriteBuffer.h>
#include <common/itoa.h>
/// 20 digits or 19 digits and a sign
#define WRITE_HELPERS_MAX_INT_WIDTH 20U
/// 40 digits or 39 digits and a sign
#define WRITE_HELPERS_MAX_INT_WIDTH 40U
namespace DB
{
namespace detail
{
/** See:
* https://github.com/localvoid/cxx-benchmark-itoa
* http://www.slideshare.net/andreialexandrescu1/three-optimization-tips-for-c-15708507
* http://vimeo.com/55639112
*/
/// The usual way, if there is not enough space in the buffer for any number to fit there.
template <typename T>
void writeUIntTextFallback(T x, WriteBuffer & buf)
void NO_INLINE writeUIntTextFallback(T x, WriteBuffer & buf)
{
if (x == 0)
{
buf.nextIfAtEnd();
*buf.position() = '0';
++buf.position();
return;
}
char tmp[WRITE_HELPERS_MAX_INT_WIDTH];
char * pos;
for (pos = tmp + WRITE_HELPERS_MAX_INT_WIDTH - 1; x != 0; --pos)
{
*pos = '0' + x % 10;
x /= 10;
}
++pos;
buf.write(pos, tmp + WRITE_HELPERS_MAX_INT_WIDTH - pos);
int len = itoa(x, tmp) - tmp;
buf.write(tmp, len);
}
/** Count the number of decimal digits in the number.
* Works well for nonuniform distribution of numbers, which usually happens.
* If most of the numbers are long, then a "branchless" code with a `bsr` instruction and a smart conversion would work better.
*/
template <typename T>
UInt32 digits10(T x)
{
if (x < 10ULL)
return 1;
if (x < 100ULL)
return 2;
if (x < 1000ULL)
return 3;
if (x < 1000000000000ULL)
{
if (x < 100000000ULL)
{
if (x < 1000000ULL)
{
if (x < 10000ULL)
return 4;
else
return 5 + (x >= 100000ULL);
}
return 7 + (x >= 10000000ULL);
}
if (x < 10000000000ULL)
return 9 + (x >= 1000000000ULL);
return 11 + (x >= 100000000000ULL);
}
return 12 + digits10(x / 1000000000000ULL);
}
/** Converts two digits per iteration.
* Works well for the nonuniform distribution of numbers, which usually happens.
* If most of the numbers are long, and if you do care about the cache, then the variant
* with a large table and four digits per iteration.
*/
template <typename T>
UInt32 writeUIntText(T x, char * dst)
{
static const char digits[201] =
"00010203040506070809"
"10111213141516171819"
"20212223242526272829"
"30313233343536373839"
"40414243444546474849"
"50515253545556575859"
"60616263646566676869"
"70717273747576777879"
"80818283848586878889"
"90919293949596979899";
const UInt32 length = digits10(x);
UInt32 next = length - 1;
while (x >= 100)
{
const auto i = (x % 100) * 2;
x /= 100;
dst[next] = digits[i + 1];
dst[next - 1] = digits[i];
next -= 2;
}
if (x < 10)
{
dst[next] = '0' + x;
}
else
{
const auto i = x * 2;
dst[next] = digits[i + 1];
dst[next - 1] = digits[i];
}
return length;
}
/** If there is enough space in the buffer - calls an optimized version, otherwise - the normal version.
*/
template <typename T>
void writeUIntText(T x, WriteBuffer & buf)
{
if (likely(buf.position() + WRITE_HELPERS_MAX_INT_WIDTH < buf.buffer().end()))
buf.position() += writeUIntText(x, buf.position());
else
writeUIntTextFallback(x, buf);
}
inline void writeLeadingMinus(WriteBuffer & buf)
{
buf.nextIfAtEnd();
*buf.position() = '-';
++buf.position();
}
/** Wrapper for signed numbers.
*/
template <typename T>
void writeSIntText(T x, WriteBuffer & buf)
{
/// A special case for the smallest negative number
if (unlikely(x == std::numeric_limits<T>::min()))
{
if (sizeof(x) == 1)
buf.write("-128", 4);
else if (sizeof(x) == 2)
buf.write("-32768", 6);
else if (sizeof(x) == 4)
buf.write("-2147483648", 11);
else
buf.write("-9223372036854775808", 20);
return;
}
if (x < 0)
{
x = -x;
writeLeadingMinus(buf);
}
writeUIntText(static_cast<std::make_unsigned_t<T>>(x), buf);
}
inline void writeSIntText(__int128 x, WriteBuffer & buf)
{
static const __int128 min_int128 = __int128(0x8000000000000000ll) << 64;
if (unlikely(x == min_int128))
{
buf.write("-170141183460469231731687303715884105728", 40);
return;
}
if (x < 0)
{
x = -x;
writeLeadingMinus(buf);
}
writeUIntText(static_cast<unsigned __int128>(x), buf);
}
}
template <typename T>
std::enable_if_t<std::is_signed_v<T> || std::is_same_v<T, Int128>, void> writeIntText(T x, WriteBuffer & buf)
{
detail::writeSIntText(x, buf);
}
template <typename T>
std::enable_if_t<std::is_unsigned_v<T>, void> writeIntText(T x, WriteBuffer & buf)
void writeIntText(T x, WriteBuffer & buf)
{
detail::writeUIntText(x, buf);
if (likely(buf.position() + WRITE_HELPERS_MAX_INT_WIDTH < buf.buffer().end()))
buf.position() = itoa(x, buf.position());
else
detail::writeUIntTextFallback(x, buf);
}
}

View File

@ -48,12 +48,12 @@ ExternalLoader::ExternalLoader(const Poco::Util::AbstractConfiguration & config_
const ExternalLoaderConfigSettings & config_settings,
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
Logger * log, const std::string & loadable_object_name)
: config_main(config_main)
, update_settings(update_settings)
, config_settings(config_settings)
, config_repository(std::move(config_repository))
, log(log)
, object_name(loadable_object_name)
: config_main(config_main)
, update_settings(update_settings)
, config_settings(config_settings)
, config_repository(std::move(config_repository))
, log(log)
, object_name(loadable_object_name)
{
}
@ -92,7 +92,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error)
/// list of recreated loadable objects to perform delayed removal from unordered_map
std::list<std::string> recreated_failed_loadable_objects;
std::unique_lock all_lock(all_mutex);
std::lock_guard all_lock(all_mutex);
/// retry loading failed loadable objects
for (auto & failed_loadable_object : failed_loadable_objects)
@ -109,11 +109,11 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error)
{
/// recalculate next attempt time
std::uniform_int_distribution<UInt64> distribution(
0, static_cast<UInt64>(std::exp2(failed_loadable_object.second.error_count)));
0, static_cast<UInt64>(std::exp2(failed_loadable_object.second.error_count)));
std::chrono::seconds delay(std::min<UInt64>(
update_settings.backoff_max_sec,
update_settings.backoff_initial_sec + distribution(rnd_engine)));
update_settings.backoff_max_sec,
update_settings.backoff_initial_sec + distribution(rnd_engine)));
failed_loadable_object.second.next_attempt_time = std::chrono::system_clock::now() + delay;
++failed_loadable_object.second.error_count;
@ -122,7 +122,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error)
}
else
{
const std::lock_guard lock{map_mutex};
std::lock_guard lock{map_mutex};
const auto & lifetime = loadable_ptr->getLifetime();
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
@ -153,62 +153,80 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error)
failed_loadable_objects.erase(name);
/// periodic update
for (auto & loadable_object : loadable_objects)
{
const auto & name = loadable_object.first;
std::vector<std::pair<std::string, LoadablePtr>> objects_to_update;
try
/// Collect objects that needs to be updated under lock. Then create new versions without lock, and assign under lock.
{
std::lock_guard lock{map_mutex};
for (auto & loadable_object : loadable_objects)
{
/// If the loadable objects failed to load or even failed to initialize from the config.
if (!loadable_object.second.loadable)
continue;
auto current = loadable_object.second.loadable;
const LoadablePtr & current = loadable_object.second.loadable;
const auto & lifetime = current->getLifetime();
/// do not update loadable objects with zero as lifetime
if (lifetime.min_sec == 0 || lifetime.max_sec == 0)
continue;
if (current->supportUpdates())
{
auto & update_time = update_times[current->getName()];
if (!current->supportUpdates())
continue;
/// check that timeout has passed
if (std::chrono::system_clock::now() < update_time)
continue;
auto update_time = update_times[current->getName()];
SCOPE_EXIT({
/// calculate next update time
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
update_time = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
});
/// check that timeout has passed
if (std::chrono::system_clock::now() < update_time)
continue;
/// check source modified
if (current->isModified())
{
/// create new version of loadable object
auto new_version = current->clone();
if (!current->isModified())
continue;
if (const auto exception_ptr = new_version->getCreationException())
std::rethrow_exception(exception_ptr);
objects_to_update.emplace_back(loadable_object.first, current);
}
}
loadable_object.second.loadable.reset();
loadable_object.second.loadable = std::move(new_version);
}
}
for (auto & [name, current] : objects_to_update)
{
LoadablePtr new_version;
std::exception_ptr exception;
/// erase stored exception on success
loadable_object.second.exception = std::exception_ptr{};
try
{
/// create new version of loadable object
new_version = current->clone();
exception = new_version->getCreationException();
}
catch (...)
{
loadable_object.second.exception = std::current_exception();
exception = std::current_exception();
}
tryLogCurrentException(log, "Cannot update " + object_name + " '" + name + "', leaving old version");
{
std::lock_guard lock{map_mutex};
if (throw_on_error)
throw;
if (auto it = loadable_objects.find(name); it != loadable_objects.end())
{
/// calculate next update time
const auto & lifetime = current->getLifetime();
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
it->second.exception = exception;
if (!exception)
{
it->second.loadable.reset();
it->second.loadable = std::move(new_version);
}
else
{
tryLogCurrentException(log, "Cannot update " + object_name + " '" + name + "', leaving old version");
if (throw_on_error)
throw;
}
}
}
}
}
@ -233,6 +251,8 @@ void ExternalLoader::reloadFromConfigFiles(const bool throw_on_error, const bool
}
/// erase removed from config loadable objects
std::lock_guard lock{map_mutex};
std::list<std::string> removed_loadable_objects;
for (const auto & loadable : loadable_objects)
{
@ -253,7 +273,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
}
else
{
std::unique_lock all_lock(all_mutex);
std::lock_guard all_lock(all_mutex);
auto modification_time_it = last_modification_times.find(config_path);
if (modification_time_it == std::end(last_modification_times))
@ -307,13 +327,13 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
{
std::lock_guard lock{map_mutex};
object_it = loadable_objects.find(name);
}
/// Object with the same name was declared in other config file.
if (object_it != std::end(loadable_objects) && object_it->second.origin != config_path)
throw Exception(object_name + " '" + name + "' from file " + config_path
+ " already declared in file " + object_it->second.origin,
ErrorCodes::EXTERNAL_LOADABLE_ALREADY_EXISTS);
/// Object with the same name was declared in other config file.
if (object_it != std::end(loadable_objects) && object_it->second.origin != config_path)
throw Exception(object_name + " '" + name + "' from file " + config_path
+ " already declared in file " + object_it->second.origin,
ErrorCodes::EXTERNAL_LOADABLE_ALREADY_EXISTS);
}
auto object_ptr = create(name, *loaded_config, key);
@ -342,7 +362,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
}
}
const std::lock_guard lock{map_mutex};
std::lock_guard lock{map_mutex};
/// add new loadable object or update an existing version
if (object_it == std::end(loadable_objects))
@ -365,7 +385,7 @@ void ExternalLoader::reloadFromConfigFile(const std::string & config_path, const
/// If the loadable object could not load data or even failed to initialize from the config.
/// - all the same we insert information into the `loadable_objects`, with the zero pointer `loadable`.
const std::lock_guard lock{map_mutex};
std::lock_guard lock{map_mutex};
const auto exception_ptr = std::current_exception();
const auto loadable_it = loadable_objects.find(name);
@ -397,14 +417,14 @@ void ExternalLoader::reload(const std::string & name)
reloadFromConfigFiles(true, true, name);
/// Check that specified object was loaded
const std::lock_guard lock{map_mutex};
std::lock_guard lock{map_mutex};
if (!loadable_objects.count(name))
throw Exception("Failed to load " + object_name + " '" + name + "' during the reload process", ErrorCodes::BAD_ARGUMENTS);
}
ExternalLoader::LoadablePtr ExternalLoader::getLoadableImpl(const std::string & name, bool throw_on_error) const
{
const std::lock_guard lock{map_mutex};
std::lock_guard lock{map_mutex};
const auto it = loadable_objects.find(name);
if (it == std::end(loadable_objects))

View File

@ -132,6 +132,9 @@ private:
bool is_initialized = false;
/// Protects only objects map.
/** Reading and assignment of "loadable" should be done under mutex.
* Creating new versions of "loadable" should not be done under mutex.
*/
mutable std::mutex map_mutex;
/// Protects all data, currently used to avoid races between updating thread and SYSTEM queries

View File

@ -37,16 +37,10 @@ CC=clang CXX=clang++ cmake -D SANITIZE=thread ..
ninja
```
## Copy binary to your server
```
scp ./dbms/programs/clickhouse yourserver:~/clickhouse-tsan
```
## Start ClickHouse and run tests
```
sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml
sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1,suppressions=../dbms/tests/tsan_suppressions.txt' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml
```
@ -63,12 +57,6 @@ cmake -D SANITIZE=undefined ..
ninja
```
## Copy binary to your server
```
scp ./dbms/programs/clickhouse yourserver:~/clickhouse-ubsan
```
## Start ClickHouse and run tests
```

View File

@ -0,0 +1,107 @@
<test>
<name>int_parsing</name>
<type>loop</type>
<stop_conditions>
<all_of>
<iterations>3</iterations>
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>5</iterations>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<preconditions>
<table_exists>test.hits</table_exists>
</preconditions>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(WatchID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(JavaEnable)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(GoodEvent)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(CounterID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ClientIP)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(RegionID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(UserID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(CounterClass)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(OS)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(UserAgent)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(Refresh)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IsRobot)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ResolutionWidth)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ResolutionHeight)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ResolutionDepth)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(FlashMajor)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(FlashMinor)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(NetMajor)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(NetMinor)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(UserAgentMajor)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(CookieEnable)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(JavascriptEnable)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IsMobile)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(MobilePhone)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IPNetworkID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(TraficSourceID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(SearchEngineID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(SearchPhrase)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(AdvEngineID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IsArtifical)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(WindowClientWidth)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(WindowClientHeight)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ClientTimeZone)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(SilverlightVersion1)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(SilverlightVersion2)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(SilverlightVersion3)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(SilverlightVersion4)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(CodeVersion)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IsLink)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IsDownload)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IsNotBounce)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(FUniqID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(HID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IsOldCounter)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IsEvent)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(IsParameter)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(DontCountHits)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(WithHash)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(Age)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(Sex)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(Income)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(Interests)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(Robotness)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(RemoteIP)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(WindowName)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(OpenerName)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(HistoryLength)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(HTTPError)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(SendTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(DNSTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ConnectTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ResponseStartTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ResponseEndTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(FetchTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(RedirectTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(DOMInteractiveTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(DOMContentLoadedTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(DOMCompleteTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(LoadEventStartTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(LoadEventEndTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(NSToDOMContentLoadedTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(FirstPaintTiming)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(RedirectCount)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(SocialSourceNetworkID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ParamPrice)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(ParamCurrencyID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(HasGCLID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(RefererHash)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(URLHash)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(CLID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(YCLID)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(RequestNum)) SETTINGS max_threads = 1</query>
<query>SELECT count() FROM test.hits WHERE NOT ignore(toString(RequestTry)) SETTINGS max_threads = 1</query>
</test>

View File

@ -0,0 +1,2 @@
# libc++
race:locale

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (18.16.0) unstable; urgency=low
clickhouse (19.1.0) unstable; urgency=low
* Modified source code
-- <root@yandex-team.ru> Fri, 14 Dec 2018 20:26:45 +0300
-- <root@yandex-team.ru> Tue, 01 Jan 2019 07:16:20 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=18.16.0
ARG version=19.1.0
RUN apt-get update \
&& apt-get install --yes --no-install-recommends \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=18.16.0
ARG version=19.1.0
ARG gosu_ver=1.10
RUN apt-get update \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=18.16.0
ARG version=19.1.0
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -0,0 +1,350 @@
#pragma once
// Based on https://github.com/amdn/itoa and combined with our optimizations
//
//=== itoa.h - Fast integer to ascii conversion --*- C++ -*-//
//
// The MIT License (MIT)
// Copyright (c) 2016 Arturo Martin-de-Nicolas
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included
// in all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
// SOFTWARE.
//===----------------------------------------------------------------------===//
#include <cstdint>
#include <type_traits>
#include "likely.h"
#include "unaligned.h"
using uint128_t = unsigned __int128;
namespace impl
{
// Using a lookup table to convert binary numbers from 0 to 99
// into ascii characters as described by Andrei Alexandrescu in
// https://www.facebook.com/notes/facebook-engineering/three-optimization-tips-for-c/10151361643253920/
static const char digits[201] = "00010203040506070809"
"10111213141516171819"
"20212223242526272829"
"30313233343536373839"
"40414243444546474849"
"50515253545556575859"
"60616263646566676869"
"70717273747576777879"
"80818283848586878889"
"90919293949596979899";
static inline uint16_t const & dd(uint8_t u)
{
return reinterpret_cast<uint16_t const *>(digits)[u];
}
template <typename T>
static constexpr T pow10(size_t x)
{
return x ? 10 * pow10<T>(x - 1) : 1;
}
// Division by a power of 10 is implemented using a multiplicative inverse.
// This strength reduction is also done by optimizing compilers, but
// presently the fastest results are produced by using the values
// for the multiplication and the shift as given by the algorithm
// described by Agner Fog in "Optimizing Subroutines in Assembly Language"
//
// http://www.agner.org/optimize/optimizing_assembly.pdf
//
// "Integer division by a constant (all processors)
// A floating point number can be divided by a constant by multiplying
// with the reciprocal. If we want to do the same with integers, we have
// to scale the reciprocal by 2n and then shift the product to the right
// by n. There are various algorithms for finding a suitable value of n
// and compensating for rounding errors. The algorithm described below
// was invented by Terje Mathisen, Norway, and not published elsewhere."
template <typename UInt, bool A, UInt M, unsigned S>
struct MulInv
{
using type = UInt;
static constexpr bool a{A};
static constexpr UInt m{M};
static constexpr unsigned s{S};
};
template <int, int, class...>
struct UT;
template <int N, class T, class... Ts>
struct UT<N, N, T, Ts...>
{
using U = T;
};
template <int N, int M, class T, class... Ts>
struct UT<N, M, T, Ts...>
{
using U = typename UT<N, 2 * M, Ts...>::U;
};
template <int N>
using MI = typename UT<
N,
1,
MulInv<uint8_t, 0, 205U, 11>,
MulInv<uint16_t, 1, 41943U, 22>,
MulInv<uint32_t, 0, 3518437209U, 45>,
MulInv<uint64_t, 0, 12379400392853802749U, 90>,
MulInv<uint128_t, 0, 0, 0>>::U;
template <int N>
using U = typename MI<N>::type;
// struct QR holds the result of dividing an unsigned N-byte variable
// by 10^N resulting in
template <size_t N>
struct QR
{
U<N> q; // quotient with fewer than 2*N decimal digits
U<N / 2> r; // remainder with at most N decimal digits
};
template <size_t N>
QR<N> static inline split(U<N> u)
{
constexpr MI<N> mi{};
U<N> q = (mi.m * (U<2 * N>(u) + mi.a)) >> mi.s;
return {q, U<N / 2>(u - q * pow10<U<N / 2>>(N))};
}
template <typename T>
static inline char * out(char * p, T && obj)
{
memcpy(p, reinterpret_cast<const void *>(&obj), sizeof(T));
p += sizeof(T);
return p;
}
struct convert
{
//===----------------------------------------------------------===//
// head: find most significant digit, skip leading zeros
//===----------------------------------------------------------===//
// "x" contains quotient and remainder after division by 10^N
// quotient is less than 10^N
template <size_t N>
static inline char * head(char * p, QR<N> x)
{
return tail(head(p, U<N / 2>(x.q)), x.r);
}
// "u" is less than 10^2*N
template <typename UInt, size_t N = sizeof(UInt)>
static inline char * head(char * p, UInt u)
{
return (u < pow10<U<N>>(N) ? (head(p, U<N / 2>(u))) : (head<N>(p, split<N>(u))));
}
// recursion base case, selected when "u" is one byte
static inline char * head(char * p, U<1> u) { return (u < 10 ? (out<char>(p, '0' + u)) : (out(p, dd(u)))); }
//===----------------------------------------------------------===//
// tail: produce all digits including leading zeros
//===----------------------------------------------------------===//
// recursive step, "u" is less than 10^2*N
template <typename UInt, size_t N = sizeof(UInt)>
static inline char * tail(char * p, UInt u)
{
QR<N> x = split<N>(u);
return tail(tail(p, U<N / 2>(x.q)), x.r);
}
// recursion base case, selected when "u" is one byte
static inline char * tail(char * p, U<1> u) { return out(p, dd(u)); }
//===----------------------------------------------------------===//
// large values are >= 10^2*N
// where x contains quotient and remainder after division by 10^N
//===----------------------------------------------------------===//
template <size_t N>
static inline char * large(char * p, QR<N> x)
{
QR<N> y = split<N>(x.q);
return tail(tail(head(p, U<N / 2>(y.q)), y.r), x.r);
}
//===----------------------------------------------------------===//
// handle values of "u" that might be >= 10^2*N
// where N is the size of "u" in bytes
//===----------------------------------------------------------===//
template <typename UInt, size_t N = sizeof(UInt)>
static inline char * itoa(char * p, UInt u)
{
if (u < pow10<U<N>>(N))
return head(p, U<N / 2>(u));
QR<N> x = split<N>(u);
return (u < pow10<U<N>>(2 * N) ? (head<N>(p, x)) : (large<N>(p, x)));
}
// selected when "u" is one byte
static inline char * itoa(char * p, U<1> u)
{
if (u < 10)
return out<char>(p, '0' + u);
if (u < 100)
return out(p, dd(u));
return out(out<char>(p, '0' + u / 100), dd(u % 100));
}
//===----------------------------------------------------------===//
// handle unsigned and signed integral operands
//===----------------------------------------------------------===//
// itoa: handle unsigned integral operands (selected by SFINAE)
template <typename U, std::enable_if_t<not std::is_signed<U>::value && std::is_integral<U>::value> * = nullptr>
static inline char * itoa(U u, char * p)
{
return convert::itoa(p, u);
}
// itoa: handle signed integral operands (selected by SFINAE)
template <typename I, size_t N = sizeof(I), std::enable_if_t<std::is_signed<I>::value && std::is_integral<I>::value> * = nullptr>
static inline char * itoa(I i, char * p)
{
// Need "mask" to be filled with a copy of the sign bit.
// If "i" is a negative value, then the result of "operator >>"
// is implementation-defined, though usually it is an arithmetic
// right shift that replicates the sign bit.
// Use a conditional expression to be portable,
// a good optimizing compiler generates an arithmetic right shift
// and avoids the conditional branch.
U<N> mask = i < 0 ? ~U<N>(0) : 0;
// Now get the absolute value of "i" and cast to unsigned type U<N>.
// Cannot use std::abs() because the result is undefined
// in 2's complement systems for the most-negative value.
// Want to avoid conditional branch for performance reasons since
// CPU branch prediction will be ineffective when negative values
// occur randomly.
// Let "u" be "i" cast to unsigned type U<N>.
// Subtract "u" from 2*u if "i" is positive or 0 if "i" is negative.
// This yields the absolute value with the desired type without
// using a conditional branch and without invoking undefined or
// implementation defined behavior:
U<N> u = ((2 * U<N>(i)) & ~mask) - U<N>(i);
// Unconditionally store a minus sign when producing digits
// in a forward direction and increment the pointer only if
// the value is in fact negative.
// This avoids a conditional branch and is safe because we will
// always produce at least one digit and it will overwrite the
// minus sign when the value is not negative.
*p = '-';
p += (mask & 1);
p = convert::itoa(p, u);
return p;
}
};
static inline int digits10(uint128_t x)
{
if (x < 10ULL)
return 1;
if (x < 100ULL)
return 2;
if (x < 1000ULL)
return 3;
if (x < 1000000000000ULL)
{
if (x < 100000000ULL)
{
if (x < 1000000ULL)
{
if (x < 10000ULL)
return 4;
else
return 5 + (x >= 100000ULL);
}
return 7 + (x >= 10000000ULL);
}
if (x < 10000000000ULL)
return 9 + (x >= 1000000000ULL);
return 11 + (x >= 100000000000ULL);
}
return 12 + digits10(x / 1000000000000ULL);
}
static inline char * writeUIntText(uint128_t x, char * p)
{
int len = digits10(x);
auto pp = p + len;
while (x >= 100)
{
const auto i = x % 100;
x /= 100;
pp -= 2;
unalignedStore(pp, dd(i));
}
if (x < 10)
*p = '0' + x;
else
unalignedStore(p, dd(x));
return p + len;
}
static inline char * writeLeadingMinus(char * pos)
{
*pos = '-';
return pos + 1;
}
static inline char * writeSIntText(__int128 x, char * pos)
{
static const __int128 min_int128 = __int128(0x8000000000000000ll) << 64;
if (unlikely(x == min_int128))
{
memcpy(pos, "-170141183460469231731687303715884105728", 40);
return pos + 40;
}
if (x < 0)
{
x = -x;
pos = writeLeadingMinus(pos);
}
return writeUIntText(static_cast<unsigned __int128>(x), pos);
}
}
template <typename I>
char * itoa(I i, char * p)
{
return impl::convert::itoa(i, p);
}
static inline char * itoa(uint128_t i, char * p)
{
return impl::writeUIntText(i, p);
}
static inline char * itoa(__int128 i, char * p)
{
return impl::writeSIntText(i, p);
}