dbms: fix build with OS X built-in clang

The built-in clang doesn’t support value() for
`std::experimental::optional`. It however supports
dereference operator, which is basically the
same thing:

```
/clickhouse/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp:83:67: error: call to unavailable member function 'value':
                    res.insert({elem.column, elem.type, rename[i].value()});
                                                        ~~~~~~~~~~^~~~~
/Applications/Xcode.app/Contents/Developer/Toolchains/XcodeDefault.xctoolchain/usr/include/c++/v1/experimental/optional:547:17: note: candidate function has been explicitly made unavailable
    value_type& value()
                ^
/Applications/Xcode.app/Contents/Developer/Toolchains/XcodeDefault.xctoolchain/usr/include/c++/v1/experimental/optional:539:33: note: candidate function has been explicitly made unavailable
    constexpr value_type const& value() const
```
This commit is contained in:
Marek Vavruša 2017-10-03 17:22:00 -07:00 committed by Marek Vavruša
parent 4c7850fdc2
commit 2ac5a743cc
17 changed files with 38 additions and 38 deletions

View File

@ -325,7 +325,7 @@ void ConfigProcessor::doIncludesRecursive(
return nullptr;
/// Enclose contents into a fake <from_zk> tag to allow pure text substitutions.
zk_document = dom_parser.parseString("<from_zk>" + contents.value() + "</from_zk>");
zk_document = dom_parser.parseString("<from_zk>" + *contents + "</from_zk>");
return getRootNode(zk_document.get());
};

View File

@ -79,7 +79,7 @@ Block NullableAdapterBlockInputStream::readImpl()
case NONE:
{
if (rename[i])
res.insert({elem.column, elem.type, rename[i].value()});
res.insert({elem.column, elem.type, *rename[i]});
else
res.insert(elem);
break;

View File

@ -63,7 +63,7 @@ RemoteBlockInputStream::RemoteBlockInputStream(
std::vector<IConnectionPool::Entry> connections;
if (main_table)
{
auto try_results = pool->getManyChecked(&settings, pool_mode, main_table.value());
auto try_results = pool->getManyChecked(&settings, pool_mode, *main_table);
connections.reserve(try_results.size());
for (auto & try_result : try_results)
connections.emplace_back(std::move(try_result.entry));

View File

@ -170,7 +170,7 @@ void ComplexKeyCacheDictionary::has(const Columns & key_columns, const DataTypes
const auto rows_num = key_columns.front()->size();
const auto keys_size = dict_struct.key.value().size();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
PODArray<StringRef> keys_array(rows_num);

View File

@ -296,7 +296,7 @@ private:
auto & attribute_array = std::get<ContainerPtrType<AttributeType>>(attribute.arrays);
const auto rows_num = key_columns.front()->size();
const auto keys_size = dict_struct.key.value().size();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
PODArray<StringRef> keys_array(rows_num);
@ -371,7 +371,7 @@ private:
/// save on some allocations
out->getOffsets().reserve(rows_num);
const auto keys_size = dict_struct.key.value().size();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
@ -523,7 +523,7 @@ private:
auto stream = source_ptr->loadKeys(in_key_columns, in_requested_rows);
stream->readPrefix();
const auto keys_size = dict_struct.key.value().size();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
const auto attributes_size = attributes.size();

View File

@ -229,7 +229,7 @@ void ComplexKeyHashedDictionary::loadData()
stream->readPrefix();
/// created upfront to avoid excess allocations
const auto keys_size = dict_struct.key.value().size();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
const auto attributes_size = attributes.size();

View File

@ -49,7 +49,7 @@ Block createSampleBlock(const DictionaryStructure & dict_struct)
if (dict_struct.id)
block.insert(ColumnWithTypeAndName{
std::make_shared<ColumnUInt64>(1), std::make_shared<DataTypeUInt64>(), dict_struct.id.value().name});
std::make_shared<ColumnUInt64>(1), std::make_shared<DataTypeUInt64>(), dict_struct.id->name});
if (dict_struct.key)
{
@ -65,7 +65,7 @@ Block createSampleBlock(const DictionaryStructure & dict_struct)
if (dict_struct.range_min)
for (const auto & attribute : { dict_struct.range_min, dict_struct.range_max })
block.insert(ColumnWithTypeAndName{
std::make_shared<ColumnUInt16>(1), std::make_shared<DataTypeDate>(), attribute.value().name});
std::make_shared<ColumnUInt16>(1), std::make_shared<DataTypeDate>(), attribute->name});
for (const auto & attribute : dict_struct.attributes)
{

View File

@ -170,7 +170,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const
{
if (key_types.size() != key.value().size())
if (key_types.size() != key->size())
throw Exception{
"Key structure does not match, expected " + getKeyDescription(),
ErrorCodes::TYPE_MISMATCH};

View File

@ -54,35 +54,35 @@ std::string ExternalQueryBuilder::composeLoadAllQuery() const
if (dict_struct.id)
{
if (!dict_struct.id.value().expression.empty())
if (!dict_struct.id->expression.empty())
{
writeParenthesisedString(dict_struct.id.value().expression, out);
writeParenthesisedString(dict_struct.id->expression, out);
writeString(" AS ", out);
}
writeQuoted(dict_struct.id.value().name, out);
writeQuoted(dict_struct.id->name, out);
if (dict_struct.range_min && dict_struct.range_max)
{
writeString(", ", out);
if (!dict_struct.range_min.value().expression.empty())
if (!dict_struct.range_min->expression.empty())
{
writeParenthesisedString(dict_struct.range_min.value().expression, out);
writeParenthesisedString(dict_struct.range_min->expression, out);
writeString(" AS ", out);
}
writeQuoted(dict_struct.range_min.value().name, out);
writeQuoted(dict_struct.range_min->name, out);
writeString(", ", out);
if (!dict_struct.range_max.value().expression.empty())
if (!dict_struct.range_max->expression.empty())
{
writeParenthesisedString(dict_struct.range_max.value().expression, out);
writeParenthesisedString(dict_struct.range_max->expression, out);
writeString(" AS ", out);
}
writeQuoted(dict_struct.range_max.value().name, out);
writeQuoted(dict_struct.range_max->name, out);
}
}
else if (dict_struct.key)
@ -146,13 +146,13 @@ std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector<UInt64>
WriteBufferFromOwnString out;
writeString("SELECT ", out);
if (!dict_struct.id.value().expression.empty())
if (!dict_struct.id->expression.empty())
{
writeParenthesisedString(dict_struct.id.value().expression, out);
writeParenthesisedString(dict_struct.id->expression, out);
writeString(" AS ", out);
}
writeQuoted(dict_struct.id.value().name, out);
writeQuoted(dict_struct.id->name, out);
for (const auto & attr : dict_struct.attributes)
{
@ -183,7 +183,7 @@ std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector<UInt64>
writeString(" AND ", out);
}
writeQuoted(dict_struct.id.value().name, out);
writeQuoted(dict_struct.id->name, out);
writeString(" IN (", out);
auto first = true;

View File

@ -212,7 +212,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector<UInt64> &
for (const UInt64 id : ids)
ids_array->add(DB::toString(id), Int32(id));
cursor->query().selector().addNewDocument(dict_struct.id.value().name)
cursor->query().selector().addNewDocument(dict_struct.id->name)
.add("$in", ids_array);
return std::make_shared<MongoDBBlockInputStream>(

View File

@ -246,7 +246,7 @@ void TrieDictionary::loadData()
stream->readPrefix();
/// created upfront to avoid excess allocations
const auto keys_size = dict_struct.key.value().size();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
const auto attributes_size = attributes.size();

View File

@ -102,8 +102,8 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
#endif
out_raw.emplace(*response_body_ostr);
deflating_buf.emplace(out_raw.value(), compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &deflating_buf.value();
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf;
}
else
{
@ -112,7 +112,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
#endif
out_raw.emplace(*response_body_ostr, working_buffer.size(), working_buffer.begin());
out = &out_raw.value();
out = &*out_raw;
}
}

View File

@ -425,7 +425,7 @@ void executeQuery(
const auto & out_file = typeid_cast<const ASTLiteral &>(*ast_query_with_output->out_file).value.safeGet<std::string>();
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
out_buf = &out_file_buf.value();
out_buf = &*out_file_buf;
}
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)

View File

@ -967,7 +967,7 @@ private:
const auto & out_file_node = typeid_cast<const ASTLiteral &>(*query_with_output->out_file);
const auto & out_file = out_file_node.value.safeGet<std::string>();
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
out_buf = &out_file_buf.value();
out_buf = &*out_file_buf;
// We are writing to file, so default format is the same as in non-interactive mode.
if (is_interactive && is_default_format)

View File

@ -304,7 +304,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
const auto & range = ranges_to_read[next_range_idx++];
task->current_range_reader = reader->readRange(range.begin, range.end);
}
MergeTreeRangeReader & range_reader = task->current_range_reader.value();
MergeTreeRangeReader & range_reader = *task->current_range_reader;
size_t current_range_rows_read = 0;
auto pre_filter_begin_pos = pre_filter_pos;

View File

@ -1745,7 +1745,7 @@ void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String &
partition_id = getPartitionIDFromQuery(partition_ast, context);
if (prefix)
LOG_DEBUG(log, "Freezing parts with prefix " + prefix.value());
LOG_DEBUG(log, "Freezing parts with prefix " + *prefix);
else
LOG_DEBUG(log, "Freezing parts with partition ID " + partition_id);
@ -1768,7 +1768,7 @@ void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String &
{
if (prefix)
{
if (!startsWith(part->info.partition_id, prefix.value()))
if (!startsWith(part->info.partition_id, *prefix))
continue;
}
else if (part->info.partition_id != partition_id)

View File

@ -221,7 +221,7 @@ BlockInputStreams StorageMerge::read(
if (!processed_stage_in_source_tables)
processed_stage_in_source_tables.emplace(processed_stage_in_source_table);
else if (processed_stage_in_source_table != processed_stage_in_source_tables.value())
else if (processed_stage_in_source_table != *processed_stage_in_source_tables)
throw Exception("Source tables for Merge table are processing data up to different stages",
ErrorCodes::INCOMPATIBLE_SOURCE_TABLES);
@ -250,7 +250,7 @@ BlockInputStreams StorageMerge::read(
if (!processed_stage_in_source_tables)
throw Exception("Logical error: unknown processed stage in source tables",
ErrorCodes::LOGICAL_ERROR);
else if (processed_stage_in_source_table != processed_stage_in_source_tables.value())
else if (processed_stage_in_source_table != *processed_stage_in_source_tables)
throw Exception("Source tables for Merge table are processing data up to different stages",
ErrorCodes::INCOMPATIBLE_SOURCE_TABLES);
@ -281,12 +281,12 @@ BlockInputStreams StorageMerge::read(
}
if (processed_stage_in_source_tables)
processed_stage = processed_stage_in_source_tables.value();
processed_stage = *processed_stage_in_source_tables;
res = narrowBlockInputStreams(res, num_streams);
/// Added to avoid different block structure from different sources
if (!processed_stage_in_source_tables || processed_stage_in_source_tables.value() == QueryProcessingStage::FetchColumns)
if (!processed_stage_in_source_tables || *processed_stage_in_source_tables == QueryProcessingStage::FetchColumns)
{
for (auto & stream : res)
stream = std::make_shared<FilterColumnsBlockInputStream>(stream, column_names, true);