mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Using C++17 [#CLICKHOUSE-3].
This commit is contained in:
parent
34b51f61a6
commit
ecb0b87bd8
@ -2,11 +2,11 @@
|
||||
|
||||
#include <experimental/string_view>
|
||||
|
||||
using StringView = std::experimental::string_view;
|
||||
using StringView = std::string_view;
|
||||
|
||||
/// It creates StringView from literal constant at compile time.
|
||||
template <typename TChar, size_t size>
|
||||
constexpr inline std::experimental::basic_string_view<TChar> makeStringView(const TChar (&str)[size])
|
||||
constexpr inline std::basic_string_view<TChar> makeStringView(const TChar (&str)[size])
|
||||
{
|
||||
return std::experimental::basic_string_view<TChar>(str, size - 1);
|
||||
return std::basic_string_view<TChar>(str, size - 1);
|
||||
}
|
||||
|
@ -53,7 +53,7 @@ private:
|
||||
private:
|
||||
NamesAndTypesListPtr required_columns;
|
||||
Actions actions;
|
||||
std::vector<std::experimental::optional<String>> rename;
|
||||
std::vector<std::optional<String>> rename;
|
||||
bool must_transform;
|
||||
};
|
||||
|
||||
|
@ -83,11 +83,11 @@ struct DictionarySpecialAttribute final
|
||||
/// Name of identifier plus list of attributes
|
||||
struct DictionaryStructure final
|
||||
{
|
||||
std::experimental::optional<DictionarySpecialAttribute> id;
|
||||
std::experimental::optional<std::vector<DictionaryAttribute>> key;
|
||||
std::optional<DictionarySpecialAttribute> id;
|
||||
std::optional<std::vector<DictionaryAttribute>> key;
|
||||
std::vector<DictionaryAttribute> attributes;
|
||||
std::experimental::optional<DictionarySpecialAttribute> range_min;
|
||||
std::experimental::optional<DictionarySpecialAttribute> range_max;
|
||||
std::optional<DictionarySpecialAttribute> range_min;
|
||||
std::optional<DictionarySpecialAttribute> range_max;
|
||||
bool has_expressions = false;
|
||||
|
||||
DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
|
||||
|
@ -54,8 +54,8 @@ private:
|
||||
std::ostream * response_header_ostr = nullptr;
|
||||
#endif
|
||||
|
||||
std::experimental::optional<WriteBufferFromOStream> out_raw;
|
||||
std::experimental::optional<ZlibDeflatingWriteBuffer> deflating_buf;
|
||||
std::optional<WriteBufferFromOStream> out_raw;
|
||||
std::optional<ZlibDeflatingWriteBuffer> deflating_buf;
|
||||
|
||||
WriteBuffer * out = nullptr; /// Uncompressed HTTP body is written to this buffer. Points to out_raw or possibly to deflating_buf.
|
||||
|
||||
|
@ -57,8 +57,8 @@ private:
|
||||
|
||||
StoragePtr storage;
|
||||
/// Порядок важен.
|
||||
std::experimental::optional<Poco::ScopedReadRWLock> data_lock;
|
||||
std::experimental::optional<Poco::ScopedReadRWLock> structure_lock;
|
||||
std::optional<Poco::ScopedReadRWLock> data_lock;
|
||||
std::optional<Poco::ScopedReadRWLock> structure_lock;
|
||||
|
||||
public:
|
||||
TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data);
|
||||
|
@ -118,7 +118,7 @@ private:
|
||||
WriteBufferFromFileDescriptor std_out {STDOUT_FILENO};
|
||||
std::unique_ptr<ShellCommand> pager_cmd;
|
||||
/// The user can specify to redirect query output to a file.
|
||||
std::experimental::optional<WriteBufferFromFile> out_file_buf;
|
||||
std::optional<WriteBufferFromFile> out_file_buf;
|
||||
BlockOutputStreamPtr block_out_stream;
|
||||
|
||||
String home_path;
|
||||
@ -816,7 +816,7 @@ private:
|
||||
if (out_file_buf)
|
||||
{
|
||||
out_file_buf->next();
|
||||
out_file_buf = std::experimental::nullopt;
|
||||
out_file_buf = std::nullopt;
|
||||
}
|
||||
std_out.next();
|
||||
}
|
||||
|
@ -324,7 +324,7 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
XMLDocumentPtr zk_document;
|
||||
auto get_zk_node = [&](const std::string & name) -> Node *
|
||||
{
|
||||
std::experimental::optional<std::string> contents = zk_node_cache->get(name);
|
||||
std::optional<std::string> contents = zk_node_cache->get(name);
|
||||
if (!contents)
|
||||
return nullptr;
|
||||
|
||||
|
@ -121,9 +121,9 @@ void NullableAdapterBlockInputStream::buildActions(
|
||||
actions.push_back(NONE);
|
||||
|
||||
if (in_elem.name != out_elem.name)
|
||||
rename.push_back(std::experimental::make_optional(out_elem.name));
|
||||
rename.push_back(std::make_optional(out_elem.name));
|
||||
else
|
||||
rename.push_back(std::experimental::nullopt);
|
||||
rename.push_back(std::nullopt);
|
||||
|
||||
if (actions.back() != NONE || rename.back())
|
||||
must_transform = true;
|
||||
|
@ -88,7 +88,7 @@ bool ReadBufferAIO::nextImpl()
|
||||
if (is_eof)
|
||||
return false;
|
||||
|
||||
std::experimental::optional<Stopwatch> watch;
|
||||
std::optional<Stopwatch> watch;
|
||||
if (profile_callback)
|
||||
watch.emplace(clock_type);
|
||||
|
||||
|
@ -51,7 +51,7 @@ bool ReadBufferFromFileDescriptor::nextImpl()
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorRead);
|
||||
|
||||
std::experimental::optional<Stopwatch> watch;
|
||||
std::optional<Stopwatch> watch;
|
||||
if (profile_callback)
|
||||
watch.emplace(clock_type);
|
||||
|
||||
|
@ -691,7 +691,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
return QueryProcessingStage::FetchColumns;
|
||||
|
||||
/// Интерпретатор подзапроса, если подзапрос
|
||||
std::experimental::optional<InterpreterSelectQuery> interpreter_subquery;
|
||||
std::optional<InterpreterSelectQuery> interpreter_subquery;
|
||||
|
||||
/// Список столбцов, которых нужно прочитать, чтобы выполнить запрос.
|
||||
Names required_columns = query_analyzer->getRequiredColumns();
|
||||
|
@ -409,7 +409,7 @@ void executeQuery(
|
||||
const ASTQueryWithOutput * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
|
||||
|
||||
WriteBuffer * out_buf = &ostr;
|
||||
std::experimental::optional<WriteBufferFromFile> out_file_buf;
|
||||
std::optional<WriteBufferFromFile> out_file_buf;
|
||||
if (ast_query_with_output && ast_query_with_output->out_file)
|
||||
{
|
||||
if (!allow_into_outfile)
|
||||
|
@ -110,7 +110,7 @@ BlockInputStreams StorageMerge::read(
|
||||
else
|
||||
virt_column_names.push_back(it);
|
||||
|
||||
std::experimental::optional<QueryProcessingStage::Enum> processed_stage_in_source_tables;
|
||||
std::optional<QueryProcessingStage::Enum> processed_stage_in_source_tables;
|
||||
|
||||
/** First we make list of selected tables to find out its size.
|
||||
* This is necessary to correctly pass the recommended number of threads to each table.
|
||||
|
@ -302,7 +302,7 @@ bool StorageMergeTree::merge(
|
||||
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
||||
|
||||
/// You must call destructor under unlocked `currently_merging_mutex`.
|
||||
std::experimental::optional<CurrentlyMergingPartsTagger> merging_tagger;
|
||||
std::optional<CurrentlyMergingPartsTagger> merging_tagger;
|
||||
String merged_name;
|
||||
|
||||
{
|
||||
|
@ -91,8 +91,8 @@ protected:
|
||||
/// Freeing memory before destroying the object.
|
||||
if (!res)
|
||||
{
|
||||
block_in = std::experimental::nullopt;
|
||||
data_in = std::experimental::nullopt;
|
||||
block_in = std::nullopt;
|
||||
data_in = std::nullopt;
|
||||
index.reset();
|
||||
}
|
||||
}
|
||||
@ -113,8 +113,8 @@ private:
|
||||
* - to save RAM when using a large number of sources.
|
||||
*/
|
||||
bool started = false;
|
||||
std::experimental::optional<CompressedReadBufferFromFile> data_in;
|
||||
std::experimental::optional<NativeBlockInputStream> block_in;
|
||||
std::optional<CompressedReadBufferFromFile> data_in;
|
||||
std::optional<NativeBlockInputStream> block_in;
|
||||
};
|
||||
|
||||
|
||||
|
@ -91,7 +91,7 @@ public:
|
||||
}
|
||||
|
||||
/// return none if daemon doesn't exist, reference to the daemon otherwise
|
||||
static std::experimental::optional<std::reference_wrapper<BaseDaemon>> tryGetInstance() { return tryGetInstance<BaseDaemon>(); }
|
||||
static std::optional<std::reference_wrapper<BaseDaemon>> tryGetInstance() { return tryGetInstance<BaseDaemon>(); }
|
||||
|
||||
/// Спит заданное количество секунд или до события wakeup
|
||||
void sleep(double seconds);
|
||||
@ -129,7 +129,7 @@ public:
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
std::experimental::optional<size_t> getLayer() const
|
||||
std::optional<size_t> getLayer() const
|
||||
{
|
||||
return layer; /// layer выставляется в классе-наследнике BaseDaemonApplication.
|
||||
}
|
||||
@ -158,7 +158,7 @@ protected:
|
||||
virtual void onInterruptSignals(int signal_id);
|
||||
|
||||
template <class Daemon>
|
||||
static std::experimental::optional<std::reference_wrapper<Daemon>> tryGetInstance();
|
||||
static std::optional<std::reference_wrapper<Daemon>> tryGetInstance();
|
||||
|
||||
virtual std::string getDefaultCorePath() const;
|
||||
|
||||
@ -207,7 +207,7 @@ protected:
|
||||
|
||||
std::map<std::string, std::unique_ptr<GraphiteWriter>> graphite_writers;
|
||||
|
||||
std::experimental::optional<size_t> layer;
|
||||
std::optional<size_t> layer;
|
||||
|
||||
std::mutex signal_handler_mutex;
|
||||
std::condition_variable signal_event;
|
||||
@ -220,7 +220,7 @@ protected:
|
||||
|
||||
|
||||
template <class Daemon>
|
||||
std::experimental::optional<std::reference_wrapper<Daemon>> BaseDaemon::tryGetInstance()
|
||||
std::optional<std::reference_wrapper<Daemon>> BaseDaemon::tryGetInstance()
|
||||
{
|
||||
Daemon * ptr = nullptr;
|
||||
try
|
||||
@ -233,7 +233,7 @@ std::experimental::optional<std::reference_wrapper<Daemon>> BaseDaemon::tryGetIn
|
||||
}
|
||||
|
||||
if (ptr)
|
||||
return std::experimental::optional<std::reference_wrapper<Daemon>>(*ptr);
|
||||
return std::optional<std::reference_wrapper<Daemon>>(*ptr);
|
||||
else
|
||||
return {};
|
||||
}
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
ZooKeeperNodeCache(const ZooKeeperNodeCache &) = delete;
|
||||
ZooKeeperNodeCache(ZooKeeperNodeCache &&) = default;
|
||||
|
||||
std::experimental::optional<std::string> get(const std::string & path);
|
||||
std::optional<std::string> get(const std::string & path);
|
||||
|
||||
Poco::Event & getChangedEvent() { return context->changed_event; }
|
||||
|
||||
|
@ -9,7 +9,7 @@ ZooKeeperNodeCache::ZooKeeperNodeCache(GetZooKeeper get_zookeeper_)
|
||||
{
|
||||
}
|
||||
|
||||
std::experimental::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
|
||||
std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
|
||||
{
|
||||
zkutil::ZooKeeperPtr zookeeper;
|
||||
std::unordered_set<std::string> invalidated_paths;
|
||||
@ -39,7 +39,7 @@ std::experimental::optional<std::string> ZooKeeperNodeCache::get(const std::stri
|
||||
}
|
||||
|
||||
if (nonexistent_nodes.count(path))
|
||||
return std::experimental::nullopt;
|
||||
return std::nullopt;
|
||||
|
||||
auto watch_callback = [context=context](zkutil::ZooKeeper & zookeeper, int type, int state, const char * path)
|
||||
{
|
||||
@ -83,7 +83,7 @@ std::experimental::optional<std::string> ZooKeeperNodeCache::get(const std::stri
|
||||
nonexistent_nodes.insert(path);
|
||||
|
||||
if (!zookeeper->existsWatch(path, /* stat = */nullptr, watch_callback))
|
||||
return std::experimental::nullopt;
|
||||
return std::nullopt;
|
||||
|
||||
/// Node was created between the two previous calls, try again. Watch is already set.
|
||||
if (zookeeper->tryGet(path, contents))
|
||||
@ -93,7 +93,7 @@ std::experimental::optional<std::string> ZooKeeperNodeCache::get(const std::stri
|
||||
return contents;
|
||||
}
|
||||
|
||||
return std::experimental::nullopt;
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user