Removed "experimental" namespace [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-11-20 07:15:43 +03:00
parent 31fc79b809
commit 01e8adbfff
28 changed files with 72 additions and 72 deletions

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit 4746a846952808f220595602f67831516822dc13
Subproject commit e30352c2c24eebecbd82d41f7054d908ac7fdc37

View File

@ -320,7 +320,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

@ -1,12 +1,12 @@
#pragma once
#include <experimental/string_view>
#include <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

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

View File

@ -4,7 +4,7 @@
#include <unordered_set>
#include <mutex>
#include <memory>
#include <experimental/optional>
#include <optional>
#include <Poco/Event.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/Common.h>
@ -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

@ -120,9 +120,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

@ -1,7 +1,7 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <experimental/optional>
#include <optional>
namespace DB
{
@ -53,7 +53,7 @@ private:
private:
Actions actions;
std::vector<std::experimental::optional<String>> rename;
std::vector<std::optional<String>> rename;
bool must_transform = false;
};

View File

@ -1,6 +1,6 @@
#pragma once
#include <experimental/optional>
#include <optional>
#include <common/logger_useful.h>
@ -146,7 +146,7 @@ private:
bool append_extra_info = false;
PoolMode pool_mode = PoolMode::GET_MANY;
std::experimental::optional<QualifiedTableName> main_table;
std::optional<QualifiedTableName> main_table;
Logger * log = &Logger::get("RemoteBlockInputStream");
};

View File

@ -11,7 +11,7 @@
#include <vector>
#include <string>
#include <map>
#include <experimental/optional>
#include <optional>
namespace DB
@ -79,11 +79,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

@ -50,7 +50,7 @@ private:
template <typename T>
void addSpecialColumn(
const std::experimental::optional<DictionarySpecialAttribute>& attribute, DataTypePtr type,
const std::optional<DictionarySpecialAttribute>& attribute, DataTypePtr type,
const std::string & default_name, const std::unordered_set<std::string> & column_names,
const PaddedPODArray<T> & values, ColumnsWithTypeAndName& columns) const;
@ -133,7 +133,7 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromPOD
template <typename DictionaryType, typename Key>
template <typename T>
void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
const std::experimental::optional<DictionarySpecialAttribute> & attribute, DataTypePtr type,
const std::optional<DictionarySpecialAttribute> & attribute, DataTypePtr type,
const std::string& default_name, const std::unordered_set<std::string> & column_names,
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const
{

View File

@ -6,7 +6,7 @@
#include <sys/types.h>
#include <sys/stat.h>
#include <experimental/optional>
#include <optional>
namespace ProfileEvents
@ -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

@ -1,7 +1,7 @@
#include <errno.h>
#include <time.h>
#include <experimental/optional>
#include <optional>
#include <Common/ProfileEvents.h>
#include <Common/Stopwatch.h>
@ -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

@ -1,6 +1,6 @@
#pragma once
#include <experimental/optional>
#include <optional>
#include <mutex>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
@ -58,8 +58,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

@ -1,4 +1,4 @@
#include <experimental/optional>
#include <optional>
#include <DataStreams/ExpressionBlockInputStream.h>
#include <DataStreams/FilterBlockInputStream.h>
@ -690,7 +690,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
return QueryProcessingStage::FetchColumns;
/// The subquery interpreter, if the subquery
std::experimental::optional<InterpreterSelectQuery> interpreter_subquery;
std::optional<InterpreterSelectQuery> interpreter_subquery;
/// List of columns to read to execute the query.
Names required_columns = query_analyzer->getRequiredColumns();

View File

@ -427,7 +427,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

@ -8,7 +8,7 @@
#include <iomanip>
#include <unordered_set>
#include <algorithm>
#include <experimental/optional>
#include <optional>
#include <boost/program_options.hpp>
#include <Poco/File.h>
@ -125,7 +125,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;
@ -832,7 +832,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

@ -7,7 +7,7 @@
#include <atomic>
#include <memory>
#include <chrono>
#include <experimental/optional>
#include <optional>
#include <Interpreters/Cluster.h>
namespace Poco
@ -94,7 +94,7 @@ private:
std::chrono::steady_clock::time_point deadline;
size_t remote_jobs_count;
size_t local_jobs_count;
std::experimental::optional<ThreadPool> pool;
std::optional<ThreadPool> pool;
};
}

View File

@ -8,7 +8,7 @@
#include <Storages/SelectQueryInfo.h>
#include <shared_mutex>
#include <memory>
#include <experimental/optional>
#include <optional>
namespace DB

View File

@ -159,7 +159,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
MarkRanges ranges_to_read;
/// Last range may be partl read. The same number of rows we need to read after prewhere
size_t rows_was_read_in_last_range = 0;
std::experimental::optional<MergeTreeRangeReader> pre_range_reader;
std::optional<MergeTreeRangeReader> pre_range_reader;
auto processNextRange = [& ranges_to_read, & rows_was_read_in_last_range, & pre_range_reader](
MergeTreeReadTask & task, MergeTreeReader & pre_reader)
@ -200,7 +200,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
size_t read_rows = pre_range_reader->read(res, rows_to_read);
rows_was_read_in_last_range += read_rows;
if (pre_range_reader->isReadingFinished())
pre_range_reader = std::experimental::nullopt;
pre_range_reader = std::nullopt;
space_left -= read_rows;
}
@ -208,7 +208,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
/// In case of isCancelled.
if (!res)
{
task->current_range_reader = std::experimental::nullopt;
task->current_range_reader = std::nullopt;
return res;
}
@ -252,7 +252,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
task->number_of_rows_to_skip = rows_was_read_in_last_range;
}
else
task->current_range_reader = std::experimental::nullopt;
task->current_range_reader = std::nullopt;
res.clear();
return res;
@ -277,7 +277,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
}
if (!pre_range_reader)
task->current_range_reader = std::experimental::nullopt;
task->current_range_reader = std::nullopt;
task->number_of_rows_to_skip = 0;
progressImpl({ 0, res.bytes() - pre_bytes });
@ -383,11 +383,11 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
readRows();
if (next_range_idx != ranges_to_read.size())
task->current_range_reader = std::experimental::nullopt;
task->current_range_reader = std::nullopt;
}
if (!pre_range_reader)
task->current_range_reader = std::experimental::nullopt;
task->current_range_reader = std::nullopt;
if (!post_filter_pos)
{
@ -457,7 +457,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
size_t rows_was_read = task->current_range_reader->read(res, rows_to_read);
if (task->current_range_reader->isReadingFinished())
task->current_range_reader = std::experimental::nullopt;
task->current_range_reader = std::nullopt;
if (res && task->size_predictor)
{

View File

@ -47,7 +47,7 @@ struct MergeTreeReadTask
/// Used to satistfy preferred_block_size_bytes limitation
MergeTreeBlockSizePredictorPtr size_predictor;
/// used to save current range processing status
std::experimental::optional<MergeTreeRangeReader> current_range_reader;
std::optional<MergeTreeRangeReader> current_range_reader;
/// the number of rows wasn't read by range_reader if condition in prewhere was false
/// helps to skip graunule if all conditions will be aslo false
size_t number_of_rows_to_skip;

View File

@ -48,7 +48,7 @@
#include <thread>
#include <typeinfo>
#include <typeindex>
#include <experimental/optional>
#include <optional>
namespace ProfileEvents
@ -1799,7 +1799,7 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part
void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context)
{
std::experimental::optional<String> prefix;
std::optional<String> prefix;
String partition_id;
if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)

View File

@ -18,7 +18,7 @@
#endif
#include <boost/rational.hpp> /// For calculations related to sampling coefficients.
#include <experimental/optional>
#include <optional>
#include <Core/FieldVisitors.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
@ -215,7 +215,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
throw Exception(exception_message.str(), ErrorCodes::INDEX_NOT_USED);
}
std::experimental::optional<PKCondition> minmax_idx_condition;
std::optional<PKCondition> minmax_idx_condition;
if (data.minmax_idx_expr)
{
minmax_idx_condition.emplace(

View File

@ -141,7 +141,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

@ -1,4 +1,4 @@
#include <experimental/optional>
#include <optional>
#include <Core/FieldVisitors.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/MergeTree/MergeTreeBlockOutputStream.h>
@ -302,7 +302,7 @@ bool StorageMergeTree::merge(
MergeTreeDataMerger::FuturePart future_part;
/// You must call destructor with unlocked `currently_merging_mutex`.
std::experimental::optional<CurrentlyMergingPartsTagger> merging_tagger;
std::optional<CurrentlyMergingPartsTagger> merging_tagger;
{
std::lock_guard<std::mutex> lock(currently_merging_mutex);

View File

@ -2,7 +2,7 @@
#include <sys/types.h>
#include <map>
#include <experimental/optional>
#include <optional>
#include <Common/escapeForFileName.h>
@ -84,8 +84,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();
}
}
@ -106,8 +106,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

@ -166,10 +166,10 @@ std::ostream & operator<<(std::ostream & stream, const std::unique_ptr<T> & what
}
#include <experimental/optional>
#include <optional>
template <typename T>
std::ostream & operator<<(std::ostream & stream, const std::experimental::optional<T> & what)
std::ostream & operator<<(std::ostream & stream, const std::optional<T> & what)
{
stream << "optional{";
if (what)

View File

@ -5,7 +5,7 @@
#include <iostream>
#include <memory>
#include <functional>
#include <experimental/optional>
#include <optional>
#include <mutex>
#include <condition_variable>
#include <atomic>
@ -93,7 +93,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);
@ -131,7 +131,7 @@ public:
return nullptr;
}
std::experimental::optional<size_t> getLayer() const
std::optional<size_t> getLayer() const
{
return layer; /// layer выставляется в классе-наследнике BaseDaemonApplication.
}
@ -160,7 +160,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;
@ -209,7 +209,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;
@ -223,7 +223,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
@ -236,7 +236,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

@ -3,7 +3,7 @@
#include <functional>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <experimental/optional>
#include <optional>
#include <sys/time.h>
#include <Poco/Ext/ThreadNumber.h>
#include <daemon/BaseDaemon.h>