提交 1a8e22c3 编写于 作者: P proller 提交者: alexey-milovidov

Allow compile on 32bit systems (#1175)

* Trying compile under 32bit..

* PerformanceTest: use getMultiple*FromConfig, fix debug helpers

* Missing file

* clnag-format of Split ComplexKeyCacheDictionary

* wip

* Requested changes

* wip

* wip

* Fix boost 1.64 and gcc7+ compile errors

* More fixes

* wip

* Fix arm build

* wip

* wip

* wip

* wip

* wip

* wip

* wip

* clean

* fix

* wip

* wip

* clean

* clean

* wip

* wip

* Update MergeTreeSettings.h

* Requested changes

* Requested changes

* Requested changes

* Requested changes
上级 0140c140
......@@ -50,6 +50,13 @@ endif ()
if (ARCH_AARCH64 OR CMAKE_SYSTEM_PROCESSOR MATCHES "arm")
set (ARCH_ARM 1)
endif ()
if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "i386")
set (ARCH_I386 1)
endif ()
if ( ( ARCH_ARM AND NOT ARCH_AARCH64 ) OR ARCH_I386)
set (ARCH_32 1)
message (WARNING "Support 32bit platforms is highly experimental")
endif ()
set (COMMON_WARNING_FLAGS "-Wall") # -Werror is also added inside directories with our own code.
set (CXX_WARNING_FLAGS "-Wnon-virtual-dtor")
......@@ -162,7 +169,7 @@ else ()
set(NOT_UNBUNDLED 1)
endif ()
# Using system libs can cause lot of warnings in includes.
if (UNBUNDLED OR NOT (CMAKE_SYSTEM MATCHES "Linux" OR APPLE))
if (UNBUNDLED OR NOT (CMAKE_SYSTEM MATCHES "Linux" OR APPLE) OR ARCH_32)
option (NO_WERROR "Disable -Werror compiler option" ON)
endif ()
......
......@@ -334,7 +334,7 @@ else()
add_feature_info(SSE2 1 "Support the SSE2 instruction set, using \"${SSE2FLAG}\"")
endif()
if(WITH_OPTIM)
if(NOT ARCH MATCHES "aarch64")
if(NOT CMAKE_SYSTEM_PROCESSOR MATCHES "arm")
set(ZLIB_ARCH_SRCS ${ZLIB_ARCH_SRCS} ${ARCHDIR}/x86.c)
endif()
if(HAVE_SSE42_INTRIN)
......
......@@ -89,7 +89,7 @@ public:
}
else
{
UInt64 elems_to_insert = std::min(max_elems - cur_elems.value.size(), rhs_elems.value.size());
UInt64 elems_to_insert = std::min(static_cast<size_t>(max_elems) - cur_elems.value.size(), rhs_elems.value.size());
cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.begin() + elems_to_insert, arena);
}
}
......
......@@ -43,7 +43,7 @@ private:
public:
LRUCache(size_t max_size_, const Delay & expiration_delay_ = Delay::zero())
: max_size(std::max(1ul, max_size_)), expiration_delay(expiration_delay_) {}
: max_size(std::max(static_cast<size_t>(1), max_size_)), expiration_delay(expiration_delay_) {}
MappedPtr get(const Key & key)
{
......
......@@ -44,7 +44,7 @@
#define DEFAULT_MERGE_BLOCK_SIZE 8192
#define DEFAULT_MAX_QUERY_SIZE 262144
#define SHOW_CHARS_ON_SYNTAX_ERROR 160L
#define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160)
#define DEFAULT_MAX_DISTRIBUTED_CONNECTIONS 1024
#define DEFAULT_INTERACTIVE_DELAY 100000
#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024
......@@ -82,7 +82,7 @@
#define PLATFORM_NOT_SUPPORTED "The only supported platforms are x86_64 and AArch64 (work in progress)"
#if !defined(__x86_64__) && !defined(__aarch64__)
#error PLATFORM_NOT_SUPPORTED
// #error PLATFORM_NOT_SUPPORTED
#endif
/// Check for presence of address sanitizer
......
......@@ -39,7 +39,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block)
if (col.type->isNumeric())
{
for (ssize_t k = 0; k < std::max(0L, static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
for (ssize_t k = 0; k < std::max(static_cast<ssize_t>(0), static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
writeChar(' ', ostr);
if (!no_escapes)
......@@ -56,7 +56,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block)
if (!no_escapes)
writeCString("\033[0m", ostr);
for (ssize_t k = 0; k < std::max(0L, static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
for (ssize_t k = 0; k < std::max(static_cast<ssize_t>(0), static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
writeChar(' ', ostr);
}
}
......
......@@ -52,7 +52,7 @@ namespace ErrorCodes
}
inline UInt64 CacheDictionary::getCellIdx(const Key id) const
inline size_t CacheDictionary::getCellIdx(const Key id) const
{
const auto hash = intHash64(id);
const auto idx = hash & size_overlap_mask;
......
......@@ -214,7 +214,7 @@ private:
bool isEmptyCell(const UInt64 idx) const;
UInt64 getCellIdx(const Key id) const;
size_t getCellIdx(const Key id) const;
void setDefaultAttributeValue(Attribute & attribute, const Key idx) const;
......@@ -253,7 +253,7 @@ private:
/// Max tries to find cell, overlaped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3
static constexpr size_t max_collision_length = 10;
const UInt64 zero_cell_idx{getCellIdx(0)};
const size_t zero_cell_idx{getCellIdx(0)};
std::map<std::string, size_t> attribute_index_by_name;
mutable std::vector<Attribute> attributes;
mutable std::vector<CellMetadata> cells;
......
......@@ -24,6 +24,7 @@ namespace ErrorCodes
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int DICTIONARY_IS_EMPTY;
extern const int NOT_IMPLEMENTED;
}
TrieDictionary::TrieDictionary(
......@@ -598,6 +599,7 @@ Columns TrieDictionary::getKeyColumns() const
auto ip_column = std::make_shared<ColumnFixedString>(IPV6_BINARY_LENGTH);
auto mask_column = std::make_shared<ColumnVector<UInt8>>();
#if defined(__SIZEOF_INT128__)
auto getter = [& ip_column, & mask_column](__uint128_t ip, size_t mask) {
UInt64 * ip_array = reinterpret_cast<UInt64 *>(&ip);
ip_array[0] = Poco::ByteOrder::fromNetwork(ip_array[0]);
......@@ -608,6 +610,9 @@ Columns TrieDictionary::getKeyColumns() const
};
trieTraverse<decltype(getter), __uint128_t>(trie, std::move(getter));
#else
throw Exception("TrieDictionary::getKeyColumns is not implemented for 32bit arch", ErrorCodes::NOT_IMPLEMENTED);
#endif
return {ip_column, mask_column};
}
......
......@@ -1256,7 +1256,7 @@ void FunctionArrayEnumerate::executeImpl(Block & block, const ColumnNumbers & ar
Array res_values(values.size());
for (size_t i = 0; i < values.size(); ++i)
{
res_values[i] = i + 1;
res_values[i] = static_cast<UInt64>(i + 1);
}
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(array->size(), res_values);
......@@ -1485,7 +1485,7 @@ bool FunctionArrayUniq::executeConst(Block & block, const ColumnNumbers & argume
for (size_t i = 0; i < values.size(); ++i)
set.insert(values[i]);
block.getByPosition(result).column = DataTypeUInt32().createConstColumn(array->size(), set.size());
block.getByPosition(result).column = DataTypeUInt32().createConstColumn(array->size(), static_cast<UInt64>(set.size()));
return true;
}
......
......@@ -651,7 +651,7 @@ public:
{
block.getByPosition(result).column = DataTypeUInt32().createConstColumn(
block.rows(),
time(0));
static_cast<UInt64>(time(0)));
}
};
......
......@@ -1301,7 +1301,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = DataTypeUInt32().createConstColumn(block.rows(), uptime);
block.getByPosition(result).column = DataTypeUInt32().createConstColumn(block.rows(), static_cast<UInt64>(uptime));
}
private:
......
......@@ -157,7 +157,7 @@ public:
for (size_t i = 0; i < size; ++i)
{
ToFieldType value = 0;
memcpy(&value, &data_from[offset], std::min(sizeof(ToFieldType), offsets_from[i] - offset - 1));
memcpy(&value, &data_from[offset], std::min(static_cast<UInt64>(sizeof(ToFieldType)), offsets_from[i] - offset - 1));
vec_res[i] = value;
offset = offsets_from[i];
}
......
......@@ -103,7 +103,7 @@ void MemoryWriteBuffer::addChunk()
}
else
{
next_chunk_size = std::max(1ul, static_cast<size_t>(chunk_tail->size() * growth_rate));
next_chunk_size = std::max(static_cast<size_t>(1), static_cast<size_t>(chunk_tail->size() * growth_rate));
next_chunk_size = std::min(next_chunk_size, max_chunk_size);
}
......
......@@ -271,7 +271,7 @@ void ReadBufferAIO::finalize()
bytes_read -= region_left_padding;
/// Ignore redundant bytes on the right.
bytes_read = std::min(bytes_read, static_cast<off_t>(requested_byte_count));
bytes_read = std::min(static_cast<off_t>(bytes_read), static_cast<off_t>(requested_byte_count));
if (bytes_read > 0)
fill_buffer.buffer().resize(region_left_padding + bytes_read);
......
......@@ -36,7 +36,7 @@ namespace
WriteBufferValidUTF8::WriteBufferValidUTF8(
WriteBuffer & output_buffer, bool group_replacements, const char * replacement, size_t size)
: BufferWithOwnMemory<WriteBuffer>(std::max(32LU, size)), output_buffer(output_buffer),
: BufferWithOwnMemory<WriteBuffer>(std::max(static_cast<size_t>(32), size)), output_buffer(output_buffer),
group_replacements(group_replacements), replacement(replacement)
{
}
......
......@@ -124,7 +124,7 @@ public:
std::unique_ptr<Cluster> getClusterWithSingleShard(size_t index) const;
private:
using SlotToShard = std::vector<size_t>;
using SlotToShard = std::vector<UInt64>;
SlotToShard slot_to_shard;
public:
......
......@@ -220,7 +220,7 @@ DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_, const
{
task_max_lifetime = config->getUInt64(prefix + ".task_max_lifetime", static_cast<UInt64>(task_max_lifetime));
cleanup_delay_period = config->getUInt64(prefix + ".cleanup_delay_period", static_cast<UInt64>(cleanup_delay_period));
max_tasks_in_queue = std::max(1UL, config->getUInt64(prefix + ".max_tasks_in_queue", max_tasks_in_queue));
max_tasks_in_queue = std::max(static_cast<UInt64>(1), config->getUInt64(prefix + ".max_tasks_in_queue", max_tasks_in_queue));
}
host_fqdn = getFQDNOrHostName();
......@@ -983,7 +983,7 @@ public:
}
if (num_hosts_finished != 0 || try_number != 0)
std::this_thread::sleep_for(std::chrono::milliseconds(50 * std::min(20LU, try_number + 1)));
std::this_thread::sleep_for(std::chrono::milliseconds(50 * std::min(static_cast<size_t>(20), try_number + 1)));
/// TODO: add shared lock
if (!zookeeper->exists(node_path))
......@@ -1018,8 +1018,8 @@ public:
res.getByName("port").column->insert(static_cast<UInt64>(port));
res.getByName("status").column->insert(static_cast<Int64>(status.code));
res.getByName("error").column->insert(status.message);
res.getByName("num_hosts_remaining").column->insert(waiting_hosts.size() - (++num_hosts_finished));
res.getByName("num_hosts_active").column->insert(cur_active_hosts.size());
res.getByName("num_hosts_remaining").column->insert(static_cast<UInt64>(waiting_hosts.size() - (++num_hosts_finished)));
res.getByName("num_hosts_active").column->insert(static_cast<UInt64>(cur_active_hosts.size()));
}
}
......
......@@ -18,7 +18,7 @@ template <typename T>
IColumn::Selector createBlockSelector(
const IColumn & column,
size_t num_shards,
const std::vector<size_t> & slots)
const std::vector<UInt64> & slots)
{
const auto total_weight = slots.size();
size_t num_rows = column.size();
......@@ -55,13 +55,13 @@ IColumn::Selector createBlockSelector(
/// Explicit instantiations to avoid code bloat in headers.
template IColumn::Selector createBlockSelector<UInt8>(const IColumn & column, size_t num_shards, const std::vector<size_t> & slots);
template IColumn::Selector createBlockSelector<UInt16>(const IColumn & column, size_t num_shards, const std::vector<size_t> & slots);
template IColumn::Selector createBlockSelector<UInt32>(const IColumn & column, size_t num_shards, const std::vector<size_t> & slots);
template IColumn::Selector createBlockSelector<UInt64>(const IColumn & column, size_t num_shards, const std::vector<size_t> & slots);
template IColumn::Selector createBlockSelector<Int8>(const IColumn & column, size_t num_shards, const std::vector<size_t> & slots);
template IColumn::Selector createBlockSelector<Int16>(const IColumn & column, size_t num_shards, const std::vector<size_t> & slots);
template IColumn::Selector createBlockSelector<Int32>(const IColumn & column, size_t num_shards, const std::vector<size_t> & slots);
template IColumn::Selector createBlockSelector<Int64>(const IColumn & column, size_t num_shards, const std::vector<size_t> & slots);
template IColumn::Selector createBlockSelector<UInt8>(const IColumn & column, size_t num_shards, const std::vector<UInt64> & slots);
template IColumn::Selector createBlockSelector<UInt16>(const IColumn & column, size_t num_shards, const std::vector<UInt64> & slots);
template IColumn::Selector createBlockSelector<UInt32>(const IColumn & column, size_t num_shards, const std::vector<UInt64> & slots);
template IColumn::Selector createBlockSelector<UInt64>(const IColumn & column, size_t num_shards, const std::vector<UInt64> & slots);
template IColumn::Selector createBlockSelector<Int8>(const IColumn & column, size_t num_shards, const std::vector<UInt64> & slots);
template IColumn::Selector createBlockSelector<Int16>(const IColumn & column, size_t num_shards, const std::vector<UInt64> & slots);
template IColumn::Selector createBlockSelector<Int32>(const IColumn & column, size_t num_shards, const std::vector<UInt64> & slots);
template IColumn::Selector createBlockSelector<Int64>(const IColumn & column, size_t num_shards, const std::vector<UInt64> & slots);
}
......@@ -22,6 +22,6 @@ template <typename T>
IColumn::Selector createBlockSelector(
const IColumn & column,
size_t num_shards,
const std::vector<size_t> & slots);
const std::vector<UInt64> & slots);
}
......@@ -2,7 +2,6 @@
#include <Parsers/IAST.h>
namespace DB
{
......@@ -12,7 +11,12 @@ namespace DB
class ASTSampleRatio : public IAST
{
public:
#ifdef __SIZEOF_INT128__
using BigNum = __uint128_t; /// Must contain the result of multiplying two UInt64.
#else
// TODO: incomplete temporary fallback. change with boost::multiprecision
using BigNum = uint64_t;
#endif
struct Rational
{
......
......@@ -106,7 +106,7 @@ public:
size_t res = fs.f_bfree * fs.f_bsize;
/// Heuristic by Michael Kolupaev: reserve 30 MB more, because statvfs shows few megabytes more space than df.
res -= std::min(res, 30 * (1ul << 20));
res -= std::min(res, static_cast<size_t>(30 * (1ul << 20)));
std::lock_guard<std::mutex> lock(mutex);
......
......@@ -189,7 +189,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
size_t recommended_rows = estimateNumRows(*task, *pre_range_reader);
if (res && recommended_rows < 1)
break;
size_t space_left = std::max(1LU, std::min(max_block_size_rows, recommended_rows));
size_t space_left = std::max(static_cast<decltype(max_block_size_rows)>(1), std::min(max_block_size_rows, recommended_rows));
while ((pre_range_reader || !task->mark_ranges.empty()) && space_left && !isCancelled())
{
......@@ -437,7 +437,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
}
else
{
size_t space_left = std::max(1LU, max_block_size_rows);
size_t space_left = std::max(static_cast<decltype(max_block_size_rows)>(1), max_block_size_rows);
while (!task->isFinished() && space_left && !isCancelled())
{
if (!task->current_range_reader)
......@@ -451,7 +451,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
size_t recommended_rows = estimateNumRows(*task, *task->current_range_reader);
if (res && recommended_rows < 1)
break;
rows_to_read = std::min(rows_to_read, std::max(1LU, recommended_rows));
rows_to_read = std::min(rows_to_read, std::max(static_cast<decltype(recommended_rows)>(1), recommended_rows));
size_t rows_was_read = task->current_range_reader->read(res, rows_to_read);
if (task->current_range_reader->isReadingFinished())
......@@ -497,7 +497,7 @@ void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block)
else if (virt_column_name == "_part_index")
{
block.insert(ColumnWithTypeAndName{
DataTypeUInt64().createConstColumn(rows, task->part_index_in_query)->convertToFullColumnIfConst(),
DataTypeUInt64().createConstColumn(rows, static_cast<UInt64>(task->part_index_in_query))->convertToFullColumnIfConst(),
std::make_shared<DataTypeUInt64>(),
virt_column_name});
}
......
......@@ -352,7 +352,7 @@ public:
for (const auto & name : ordinary_columns)
sum_ordinary_columns += map.at(name);
sum_total = std::max(1UL, sum_index_columns + sum_ordinary_columns);
sum_total = std::max(static_cast<decltype(sum_index_columns)>(1), sum_index_columns + sum_ordinary_columns);
}
/// Approximate size of num_rows column elements if column contains num_total_rows elements
......
......@@ -636,7 +636,7 @@ void MergeTreeDataPart::loadIndex()
String index_path = getFullPath() + "primary.idx";
ReadBufferFromFile index_file(index_path,
std::min(static_cast<size_t>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(index_path).getSize()));
std::min(static_cast<Poco::File::FileSize>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(index_path).getSize()));
for (size_t i = 0; i < size; ++i)
for (size_t j = 0; j < key_size; ++j)
......@@ -679,7 +679,7 @@ void MergeTreeDataPart::loadChecksums(bool require)
return;
}
ReadBufferFromFile file(path, std::min(static_cast<size_t>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(path).getSize()));
ReadBufferFromFile file(path, std::min(static_cast<Poco::File::FileSize>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(path).getSize()));
if (checksums.read(file))
assertEOF(file);
}
......@@ -717,7 +717,7 @@ void MergeTreeDataPart::loadColumns(bool require)
return;
}
ReadBufferFromFile file(path, std::min(static_cast<size_t>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(path).getSize()));
ReadBufferFromFile file(path, std::min(static_cast<Poco::File::FileSize>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(path).getSize()));
columns.readText(file);
}
......
......@@ -22,7 +22,7 @@ struct MergeTreeSettings
/** Merge settings. */
/// Maximum in total size of parts to merge, when there are maximum (minimum) free threads in background pool (or entries in replication queue).
size_t max_bytes_to_merge_at_max_space_in_pool = 150ULL * 1024 * 1024 * 1024;
size_t max_bytes_to_merge_at_max_space_in_pool = size_t(150) * 1024 * 1024 * 1024;
size_t max_bytes_to_merge_at_min_space_in_pool = 1024 * 1024;
/// How many tasks of merging parts are allowed simultaneously in ReplicatedMergeTree queue.
......
......@@ -50,7 +50,7 @@ private:
MergeTreeData & data;
const ReshardingJob & job;
Logger * log;
std::vector<size_t> slots;
std::vector<UInt64> slots;
ExpressionActionsPtr sharding_key_expr;
std::string sharding_key_column_name;
};
......
......@@ -159,7 +159,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
/// Use ZooKeeper's first node (last according to time) timestamp as "current" time.
Int64 current_time = timed_blocks.front().stat.ctime;
Int64 time_threshold = std::max(0L, current_time - static_cast<Int64>(1000 * storage.data.settings.replicated_deduplication_window_seconds));
Int64 time_threshold = std::max(static_cast<Int64>(0), current_time - static_cast<Int64>(1000 * storage.data.settings.replicated_deduplication_window_seconds));
/// Virtual node, all nodes that are "greater" than this one will be deleted
NodeWithStat block_threshold("", RequiredStat(time_threshold));
......
......@@ -111,7 +111,7 @@ private:
struct Stream
{
Stream(const std::string & data_path, size_t offset, size_t max_read_buffer_size)
: plain(data_path, std::min(max_read_buffer_size, Poco::File(data_path).getSize())),
: plain(data_path, std::min(static_cast<Poco::File::FileSize>(max_read_buffer_size), Poco::File(data_path).getSize())),
compressed(plain)
{
if (offset)
......
......@@ -72,7 +72,7 @@ protected:
data_in.emplace(
storage.full_path() + "data.bin", 0, 0,
std::min(max_read_buffer_size, Poco::File(storage.full_path() + "data.bin").getSize()));
std::min(static_cast<Poco::File::FileSize>(max_read_buffer_size), Poco::File(storage.full_path() + "data.bin").getSize()));
block_in.emplace(*data_in, 0, true, index_begin, index_end);
}
......
......@@ -76,7 +76,7 @@ private:
struct Stream
{
Stream(const std::string & data_path, size_t max_read_buffer_size)
: plain(data_path, std::min(max_read_buffer_size, Poco::File(data_path).getSize())),
: plain(data_path, std::min(static_cast<Poco::File::FileSize>(max_read_buffer_size), Poco::File(data_path).getSize())),
compressed(plain)
{
}
......
......@@ -193,9 +193,9 @@ BlockInputStreams StorageSystemColumns::read(
}
else
{
data_compressed_bytes_column->insert(it->second.data_compressed);
data_uncompressed_bytes_column->insert(it->second.data_uncompressed);
marks_bytes_column->insert(it->second.marks);
data_compressed_bytes_column->insert(static_cast<UInt64>(it->second.data_compressed));
data_uncompressed_bytes_column->insert(static_cast<UInt64>(it->second.data_uncompressed));
marks_bytes_column->insert(static_cast<UInt64>(it->second.marks));
}
}
}
......
......@@ -98,12 +98,12 @@ BlockInputStreams StorageSystemDictionaries::read(
col_attribute_types.column->insert(ext::map<Array>(dict_struct.attributes, [] (auto & attr) -> decltype(auto) {
return attr.type->getName();
}));
col_bytes_allocated.column->insert(dict_ptr->getBytesAllocated());
col_query_count.column->insert(dict_ptr->getQueryCount());
col_bytes_allocated.column->insert(static_cast<UInt64>(dict_ptr->getBytesAllocated()));
col_query_count.column->insert(static_cast<UInt64>(dict_ptr->getQueryCount()));
col_hit_rate.column->insert(dict_ptr->getHitRate());
col_element_count.column->insert(dict_ptr->getElementCount());
col_element_count.column->insert(static_cast<UInt64>(dict_ptr->getElementCount()));
col_load_factor.column->insert(dict_ptr->getLoadFactor());
col_creation_time.column->insert(std::chrono::system_clock::to_time_t(dict_ptr->getCreationTime()));
col_creation_time.column->insert(static_cast<UInt64>(std::chrono::system_clock::to_time_t(dict_ptr->getCreationTime())));
col_source.column->insert(dict_ptr->getSource()->toString());
}
else
......
......@@ -205,7 +205,7 @@ BlockInputStreams StorageSystemParts::read(
block.getByPosition(i++).column->insert(part->info.partition_id);
block.getByPosition(i++).column->insert(part->name);
block.getByPosition(i++).column->insert(static_cast<UInt64>(active_parts.count(part)));
block.getByPosition(i++).column->insert(part->size);
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->size));
size_t marks_size = 0;
for (const NameAndTypePair & it : part->columns)
......@@ -215,23 +215,23 @@ BlockInputStreams StorageSystemParts::read(
if (checksum != part->checksums.files.end())
marks_size += checksum->second.file_size;
}
block.getByPosition(i++).column->insert(marks_size);
block.getByPosition(i++).column->insert(static_cast<UInt64>(marks_size));
block.getByPosition(i++).column->insert(part->getExactSizeRows());
block.getByPosition(i++).column->insert(static_cast<size_t>(part->size_in_bytes));
block.getByPosition(i++).column->insert(part->modification_time);
block.getByPosition(i++).column->insert(part->remove_time);
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->getExactSizeRows()));
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->size_in_bytes));
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->modification_time));
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->remove_time));
/// For convenience, in returned refcount, don't add references that was due to local variables in this method: all_parts, active_parts.
block.getByPosition(i++).column->insert(part.use_count() - (active_parts.count(part) ? 2 : 1));
block.getByPosition(i++).column->insert(static_cast<UInt64>(part.use_count() - (active_parts.count(part) ? 2 : 1)));
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->getMinDate()));
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->getMaxDate()));
block.getByPosition(i++).column->insert(part->info.min_block);
block.getByPosition(i++).column->insert(part->info.max_block);
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->info.level));
block.getByPosition(i++).column->insert(part->getIndexSizeInBytes());
block.getByPosition(i++).column->insert(part->getIndexSizeInAllocatedBytes());
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->getIndexSizeInBytes()));
block.getByPosition(i++).column->insert(static_cast<UInt64>(part->getIndexSizeInAllocatedBytes()));
block.getByPosition(i++).column->insert(database);
block.getByPosition(i++).column->insert(table);
......
......@@ -91,11 +91,11 @@ BlockInputStreams StorageSystemProcesses::read(
block.getByPosition(i++).column->insert(process.client_info.http_user_agent);
block.getByPosition(i++).column->insert(process.client_info.quota_key);
block.getByPosition(i++).column->insert(process.elapsed_seconds);
block.getByPosition(i++).column->insert(process.read_rows);
block.getByPosition(i++).column->insert(process.read_bytes);
block.getByPosition(i++).column->insert(process.total_rows);
block.getByPosition(i++).column->insert(process.written_rows);
block.getByPosition(i++).column->insert(process.written_bytes);
block.getByPosition(i++).column->insert(UInt64(process.read_rows));
block.getByPosition(i++).column->insert(UInt64(process.read_bytes));
block.getByPosition(i++).column->insert(UInt64(process.total_rows));
block.getByPosition(i++).column->insert(UInt64(process.written_rows));
block.getByPosition(i++).column->insert(UInt64(process.written_bytes));
block.getByPosition(i++).column->insert(process.memory_usage);
block.getByPosition(i++).column->insert(process.query);
}
......
if (CMAKE_SYSTEM MATCHES "FreeBSD")
if (CMAKE_SYSTEM MATCHES "FreeBSD" OR ARCH_32)
option (USE_INTERNAL_GPERFTOOLS_LIBRARY "Set to FALSE to use system gperftools (tcmalloc) library instead of bundled" OFF)
else ()
option (USE_INTERNAL_GPERFTOOLS_LIBRARY "Set to FALSE to use system gperftools (tcmalloc) library instead of bundled" ${NOT_UNBUNDLED})
......@@ -12,13 +12,15 @@ if (ENABLE_LIBTCMALLOC)
find_package (Gperftools)
endif ()
if (NOT (GPERFTOOLS_FOUND AND GPERFTOOLS_INCLUDE_DIR AND GPERFTOOLS_TCMALLOC_MINIMAL))
if (NOT (GPERFTOOLS_FOUND AND GPERFTOOLS_INCLUDE_DIR AND GPERFTOOLS_TCMALLOC_MINIMAL) AND NOT (CMAKE_SYSTEM MATCHES "FreeBSD" OR ARCH_32))
set (USE_INTERNAL_GPERFTOOLS_LIBRARY 1)
set (GPERFTOOLS_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libtcmalloc/include")
set (GPERFTOOLS_TCMALLOC_MINIMAL tcmalloc_minimal_internal)
endif ()
set (USE_TCMALLOC 1)
if (GPERFTOOLS_FOUND OR USE_INTERNAL_GPERFTOOLS_LIBRARY)
set (USE_TCMALLOC 1)
endif ()
message (STATUS "Using tcmalloc=${USE_TCMALLOC}: ${GPERFTOOLS_INCLUDE_DIR} : ${GPERFTOOLS_TCMALLOC_MINIMAL}")
endif ()
include (CMakePushCheckState)
cmake_push_check_state ()
if (CMAKE_SYSTEM MATCHES "Linux" AND NOT ARCH_ARM)
if (CMAKE_SYSTEM MATCHES "Linux" AND NOT ARCH_ARM AND NOT ARCH_32)
option (USE_INTERNAL_UNWIND_LIBRARY "Set to FALSE to use system unwind library instead of bundled" ${NOT_UNBUNDLED})
else ()
option (USE_INTERNAL_UNWIND_LIBRARY "Set to FALSE to use system unwind library instead of bundled" OFF)
......@@ -34,7 +34,7 @@ endif ()
if (UNWIND_LIBRARY AND UNWIND_INCLUDE_DIR)
set (USE_UNWIND 1)
elseif (CMAKE_SYSTEM MATCHES "Linux" AND NOT ARCH_ARM)
elseif (CMAKE_SYSTEM MATCHES "Linux" AND NOT ARCH_ARM AND NOT ARCH_32)
set (USE_INTERNAL_UNWIND_LIBRARY 1)
set (UNWIND_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libunwind/include")
set (UNWIND_LIBRARY unwind)
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册