Using C++17 [#CLICKHOUSE-3].

This commit is contained in:
Alexey Milovidov 2017-04-01 11:38:31 +03:00 committed by alexey-milovidov
parent 34b51f61a6
commit ecb0b87bd8
18 changed files with 38 additions and 38 deletions

View File

@ -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);
}

View File

@ -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;
};

View File

@ -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);

View File

@ -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.

View File

@ -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);

View File

@ -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();
}

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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);

View File

@ -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();

View File

@ -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)

View File

@ -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.

View File

@ -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;
{

View File

@ -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;
};

View File

@ -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 {};
}

View File

@ -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; }

View File

@ -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;
}
}