mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
parent
739ba29466
commit
ecd4c31387
@ -42,13 +42,13 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int
|
||||
char buffer[buf_size];
|
||||
WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer);
|
||||
|
||||
StringRef query_id;
|
||||
std::string_view query_id;
|
||||
UInt64 thread_id;
|
||||
|
||||
if (CurrentThread::isInitialized())
|
||||
{
|
||||
query_id = StringRef(CurrentThread::getQueryId());
|
||||
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);
|
||||
query_id = CurrentThread::getQueryId();
|
||||
query_id = std::string_view(query_id.data(), std::min(query_id.size(), QUERY_ID_MAX_LEN));
|
||||
|
||||
thread_id = CurrentThread::get().thread_id;
|
||||
}
|
||||
@ -59,8 +59,8 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int
|
||||
|
||||
writeChar(false, out); /// true if requested to stop the collecting thread.
|
||||
|
||||
writeBinary(static_cast<uint8_t>(query_id.size), out);
|
||||
out.write(query_id.data, query_id.size);
|
||||
writeBinary(static_cast<uint8_t>(query_id.size()), out);
|
||||
out.write(query_id.data(), query_id.size());
|
||||
|
||||
size_t stack_trace_size = stack_trace.getSize();
|
||||
size_t stack_trace_offset = stack_trace.getOffset();
|
||||
|
@ -18,7 +18,7 @@ void SerializationEnum<Type>::serializeText(const IColumn & column, size_t row_n
|
||||
template <typename Type>
|
||||
void SerializationEnum<Type>::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
writeEscapedString(this->getNameForValue(assert_cast<const ColumnType &>(column).getData()[row_num]), ostr);
|
||||
writeEscapedString(this->getNameForValue(assert_cast<const ColumnType &>(column).getData()[row_num]).toView(), ostr);
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
@ -69,13 +69,13 @@ void SerializationEnum<Type>::deserializeWholeText(IColumn & column, ReadBuffer
|
||||
template <typename Type>
|
||||
void SerializationEnum<Type>::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeJSONString(this->getNameForValue(assert_cast<const ColumnType &>(column).getData()[row_num]), ostr, settings);
|
||||
writeJSONString(this->getNameForValue(assert_cast<const ColumnType &>(column).getData()[row_num]).toView(), ostr, settings);
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
void SerializationEnum<Type>::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
writeXMLStringForTextElement(this->getNameForValue(assert_cast<const ColumnType &>(column).getData()[row_num]), ostr);
|
||||
writeXMLStringForTextElement(this->getNameForValue(assert_cast<const ColumnType &>(column).getData()[row_num]).toView(), ostr);
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
|
@ -213,7 +213,7 @@ void SerializationString::serializeText(const IColumn & column, size_t row_num,
|
||||
|
||||
void SerializationString::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
writeEscapedString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
||||
writeEscapedString(assert_cast<const ColumnString &>(column).getDataAt(row_num).toView(), ostr);
|
||||
}
|
||||
|
||||
|
||||
@ -266,7 +266,7 @@ void SerializationString::deserializeTextQuoted(IColumn & column, ReadBuffer & i
|
||||
|
||||
void SerializationString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeJSONString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr, settings);
|
||||
writeJSONString(assert_cast<const ColumnString &>(column).getDataAt(row_num).toView(), ostr, settings);
|
||||
}
|
||||
|
||||
|
||||
@ -278,7 +278,7 @@ void SerializationString::deserializeTextJSON(IColumn & column, ReadBuffer & ist
|
||||
|
||||
void SerializationString::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
writeXMLStringForTextElement(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
||||
writeXMLStringForTextElement(assert_cast<const ColumnString &>(column).getDataAt(row_num).toView(), ostr);
|
||||
}
|
||||
|
||||
|
||||
|
@ -87,10 +87,10 @@ public:
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
StringRef current_row = column_haystack->getDataAt(i);
|
||||
std::string_view current_row = column_haystack->getDataAt(i).toView();
|
||||
|
||||
if (re2->Match(re2_st::StringPiece(current_row.data, current_row.size),
|
||||
0, current_row.size, re2_st::RE2::UNANCHORED, matched_groups.data(), matched_groups.size()))
|
||||
if (re2->Match(re2_st::StringPiece(current_row.data(), current_row.size()),
|
||||
0, current_row.size(), re2_st::RE2::UNANCHORED, matched_groups.data(), matched_groups.size()))
|
||||
{
|
||||
// 1 is to exclude group #0 which is whole re match.
|
||||
for (size_t group = 1; group <= groups_count; ++group)
|
||||
|
@ -94,19 +94,19 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
StringRef maximum_unit_str;
|
||||
std::string_view maximum_unit_str;
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnPtr & maximum_unit_column = arguments[1].column;
|
||||
const ColumnConst * maximum_unit_const_col = checkAndGetColumnConstStringOrFixedString(maximum_unit_column.get());
|
||||
if (maximum_unit_const_col)
|
||||
maximum_unit_str = maximum_unit_const_col->getDataColumn().getDataAt(0);
|
||||
maximum_unit_str = maximum_unit_const_col->getDataColumn().getDataAt(0).toView();
|
||||
}
|
||||
|
||||
Unit max_unit;
|
||||
|
||||
/// Default means "use all available units".
|
||||
if (maximum_unit_str.size == 0 || maximum_unit_str == "years")
|
||||
if (maximum_unit_str.empty() || maximum_unit_str == "years")
|
||||
max_unit = Years;
|
||||
else if (maximum_unit_str == "months")
|
||||
max_unit = Months;
|
||||
@ -122,7 +122,7 @@ public:
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unexpected value of maximum unit argument ({}) for function {}, the only allowed values are:"
|
||||
" 'seconds', 'minutes', 'hours', 'days', 'months', 'years'.",
|
||||
maximum_unit_str.toString(), getName());
|
||||
std::string(maximum_unit_str), getName());
|
||||
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
|
@ -64,8 +64,8 @@ public:
|
||||
|
||||
for (size_t i = 0; i < count; ++i)
|
||||
{
|
||||
StringRef encoded_string = encoded->getDataAt(i);
|
||||
geohashDecode(encoded_string.data, encoded_string.size, &lon_data[i], &lat_data[i]);
|
||||
std::string_view encoded_string = encoded->getDataAt(i).toView();
|
||||
geohashDecode(encoded_string.data(), encoded_string.size(), &lon_data[i], &lat_data[i]);
|
||||
}
|
||||
|
||||
MutableColumns result;
|
||||
|
@ -27,7 +27,7 @@ class IPAddressVariant
|
||||
{
|
||||
public:
|
||||
|
||||
explicit IPAddressVariant(StringRef address_str)
|
||||
explicit IPAddressVariant(std::string_view address_str)
|
||||
{
|
||||
/// IP address parser functions require that the input is
|
||||
/// NULL-terminated so we need to copy it.
|
||||
@ -85,7 +85,7 @@ IPAddressCIDR parseIPWithCIDR(std::string_view cidr_str)
|
||||
throw DB::Exception("The text does not contain '/': " + std::string(cidr_str), DB::ErrorCodes::CANNOT_PARSE_TEXT);
|
||||
|
||||
std::string_view addr_str = cidr_str.substr(0, pos_slash);
|
||||
IPAddressVariant addr(StringRef{addr_str.data(), addr_str.size()});
|
||||
IPAddressVariant addr(addr_str);
|
||||
|
||||
uint8_t prefix = 0;
|
||||
auto prefix_str = cidr_str.substr(pos_slash+1);
|
||||
@ -188,7 +188,7 @@ namespace DB
|
||||
const auto & col_addr = col_addr_const.getDataColumn();
|
||||
const auto & col_cidr = col_cidr_const.getDataColumn();
|
||||
|
||||
const auto addr = IPAddressVariant(col_addr.getDataAt(0));
|
||||
const auto addr = IPAddressVariant(col_addr.getDataAt(0).toView());
|
||||
const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(0).toView());
|
||||
|
||||
ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(1);
|
||||
@ -204,7 +204,7 @@ namespace DB
|
||||
{
|
||||
const auto & col_addr = col_addr_const.getDataColumn();
|
||||
|
||||
const auto addr = IPAddressVariant(col_addr.getDataAt (0));
|
||||
const auto addr = IPAddressVariant(col_addr.getDataAt(0).toView());
|
||||
|
||||
ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count);
|
||||
ColumnUInt8::Container & vec_res = col_res->getData();
|
||||
@ -228,7 +228,7 @@ namespace DB
|
||||
ColumnUInt8::Container & vec_res = col_res->getData();
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
const auto addr = IPAddressVariant(col_addr.getDataAt(i));
|
||||
const auto addr = IPAddressVariant(col_addr.getDataAt(i).toView());
|
||||
vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0;
|
||||
}
|
||||
return col_res;
|
||||
@ -242,7 +242,7 @@ namespace DB
|
||||
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
const auto addr = IPAddressVariant(col_addr.getDataAt(i));
|
||||
const auto addr = IPAddressVariant(col_addr.getDataAt(i).toView());
|
||||
const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i).toView());
|
||||
|
||||
vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0;
|
||||
|
@ -54,9 +54,7 @@ template <typename T> WriteBuffer & operator<< (QuoteManipWriteBuffer buf,
|
||||
template <typename T> WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const T & x) { writeDoubleQuoted(x, buf.get()); return buf; }
|
||||
template <typename T> WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const T & x) { writeBinary(x, buf.get()); return buf; }
|
||||
|
||||
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const String & x) { writeEscapedString(x, buf); return buf; }
|
||||
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, std::string_view x) { writeEscapedString(x, buf); return buf; }
|
||||
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, StringRef x) { writeEscapedString(x, buf); return buf; }
|
||||
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const char * x) { writeEscapedString(x, strlen(x), buf); return buf; }
|
||||
|
||||
inline WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'\''>(x, x + strlen(x), buf.get()); return buf; }
|
||||
|
@ -360,19 +360,9 @@ void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & b
|
||||
}
|
||||
|
||||
|
||||
inline void writeJSONString(StringRef s, WriteBuffer & buf, const FormatSettings & settings)
|
||||
{
|
||||
writeJSONString(s.data, s.data + s.size, buf, settings);
|
||||
}
|
||||
|
||||
inline void writeJSONString(std::string_view s, WriteBuffer & buf, const FormatSettings & settings)
|
||||
{
|
||||
writeJSONString(StringRef{s}, buf, settings);
|
||||
}
|
||||
|
||||
inline void writeJSONString(const String & s, WriteBuffer & buf, const FormatSettings & settings)
|
||||
{
|
||||
writeJSONString(StringRef{s}, buf, settings);
|
||||
writeJSONString(s.data(), s.data() + s.size(), buf, settings);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -417,7 +407,7 @@ void writeJSONNumber(T x, WriteBuffer & ostr, const FormatSettings & settings)
|
||||
|
||||
|
||||
template <char c>
|
||||
void writeAnyEscapedString(const String & s, WriteBuffer & buf)
|
||||
void writeAnyEscapedString(std::string_view s, WriteBuffer & buf)
|
||||
{
|
||||
writeAnyEscapedString<c>(s.data(), s.data() + s.size(), buf);
|
||||
}
|
||||
@ -428,18 +418,6 @@ inline void writeEscapedString(const char * str, size_t size, WriteBuffer & buf)
|
||||
writeAnyEscapedString<'\''>(str, str + size, buf);
|
||||
}
|
||||
|
||||
|
||||
inline void writeEscapedString(const String & s, WriteBuffer & buf)
|
||||
{
|
||||
writeEscapedString(s.data(), s.size(), buf);
|
||||
}
|
||||
|
||||
|
||||
inline void writeEscapedString(StringRef ref, WriteBuffer & buf)
|
||||
{
|
||||
writeEscapedString(ref.data, ref.size, buf);
|
||||
}
|
||||
|
||||
inline void writeEscapedString(std::string_view ref, WriteBuffer & buf)
|
||||
{
|
||||
writeEscapedString(ref.data(), ref.size(), buf);
|
||||
@ -455,16 +433,9 @@ void writeAnyQuotedString(const char * begin, const char * end, WriteBuffer & bu
|
||||
|
||||
|
||||
template <char quote_character>
|
||||
void writeAnyQuotedString(const String & s, WriteBuffer & buf)
|
||||
void writeAnyQuotedString(std::string_view ref, WriteBuffer & buf)
|
||||
{
|
||||
writeAnyQuotedString<quote_character>(s.data(), s.data() + s.size(), buf);
|
||||
}
|
||||
|
||||
|
||||
template <char quote_character>
|
||||
void writeAnyQuotedString(StringRef ref, WriteBuffer & buf)
|
||||
{
|
||||
writeAnyQuotedString<quote_character>(ref.data, ref.data + ref.size, buf);
|
||||
writeAnyQuotedString<quote_character>(ref.data(), ref.data() + ref.size(), buf);
|
||||
}
|
||||
|
||||
|
||||
@ -475,7 +446,7 @@ inline void writeQuotedString(const String & s, WriteBuffer & buf)
|
||||
|
||||
inline void writeQuotedString(StringRef ref, WriteBuffer & buf)
|
||||
{
|
||||
writeAnyQuotedString<'\''>(ref, buf);
|
||||
writeAnyQuotedString<'\''>(ref.toView(), buf);
|
||||
}
|
||||
|
||||
inline void writeQuotedString(std::string_view ref, WriteBuffer & buf)
|
||||
@ -490,7 +461,7 @@ inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf)
|
||||
|
||||
inline void writeDoubleQuotedString(StringRef s, WriteBuffer & buf)
|
||||
{
|
||||
writeAnyQuotedString<'"'>(s, buf);
|
||||
writeAnyQuotedString<'"'>(s.toView(), buf);
|
||||
}
|
||||
|
||||
inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf)
|
||||
@ -501,7 +472,7 @@ inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf)
|
||||
/// Outputs a string in backquotes.
|
||||
inline void writeBackQuotedString(StringRef s, WriteBuffer & buf)
|
||||
{
|
||||
writeAnyQuotedString<'`'>(s, buf);
|
||||
writeAnyQuotedString<'`'>(s.toView(), buf);
|
||||
}
|
||||
|
||||
/// Outputs a string in backquotes for MySQL.
|
||||
@ -611,16 +582,11 @@ inline void writeXMLStringForTextElementOrAttributeValue(const char * begin, con
|
||||
}
|
||||
}
|
||||
|
||||
inline void writeXMLStringForTextElementOrAttributeValue(const String & s, WriteBuffer & buf)
|
||||
inline void writeXMLStringForTextElementOrAttributeValue(std::string_view s, WriteBuffer & buf)
|
||||
{
|
||||
writeXMLStringForTextElementOrAttributeValue(s.data(), s.data() + s.size(), buf);
|
||||
}
|
||||
|
||||
inline void writeXMLStringForTextElementOrAttributeValue(StringRef s, WriteBuffer & buf)
|
||||
{
|
||||
writeXMLStringForTextElementOrAttributeValue(s.data, s.data + s.size, buf);
|
||||
}
|
||||
|
||||
/// Writing a string to a text node in XML (not into an attribute - otherwise you need more escaping).
|
||||
inline void writeXMLStringForTextElement(const char * begin, const char * end, WriteBuffer & buf)
|
||||
{
|
||||
@ -652,16 +618,11 @@ inline void writeXMLStringForTextElement(const char * begin, const char * end, W
|
||||
}
|
||||
}
|
||||
|
||||
inline void writeXMLStringForTextElement(const String & s, WriteBuffer & buf)
|
||||
inline void writeXMLStringForTextElement(std::string_view s, WriteBuffer & buf)
|
||||
{
|
||||
writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf);
|
||||
}
|
||||
|
||||
inline void writeXMLStringForTextElement(StringRef s, WriteBuffer & buf)
|
||||
{
|
||||
writeXMLStringForTextElement(s.data, s.data + s.size, buf);
|
||||
}
|
||||
|
||||
template <typename IteratorSrc, typename IteratorDst>
|
||||
void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes);
|
||||
void formatUUID(const UInt8 * src16, UInt8 * dst36);
|
||||
|
@ -32,11 +32,6 @@ TEST(OperatorsManipTest, EscapingTest)
|
||||
checkString(sv1, escape, "Hello \\'world\\'");
|
||||
std::string_view sv2 = s2;
|
||||
checkString(sv2, escape, "Hello \\\\world\\\\"); // NOLINT
|
||||
|
||||
StringRef sr1 = s1;
|
||||
checkString(sr1, escape, "Hello \\'world\\'");
|
||||
StringRef sr2 = s2;
|
||||
checkString(sr2, escape, "Hello \\\\world\\\\"); // NOLINT
|
||||
}
|
||||
|
||||
TEST(OperatorsManipTest, QuouteTest)
|
||||
|
@ -71,11 +71,11 @@ static const Graphite::Pattern undef_pattern =
|
||||
.type = undef_pattern.TypeUndef,
|
||||
};
|
||||
|
||||
inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, StringRef path)
|
||||
inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, std::string_view path)
|
||||
{
|
||||
if (params.patterns_typed)
|
||||
{
|
||||
std::string_view path_view = path.toView();
|
||||
std::string_view path_view = path;
|
||||
if (path_view.find("?"sv) == path_view.npos)
|
||||
return params.patterns_plain;
|
||||
else
|
||||
@ -89,7 +89,7 @@ inline static const Patterns & selectPatternsForMetricType(const Graphite::Param
|
||||
|
||||
Graphite::RollupRule selectPatternForPath(
|
||||
const Graphite::Params & params,
|
||||
StringRef path)
|
||||
std::string_view path)
|
||||
{
|
||||
const Graphite::Pattern * first_match = &undef_pattern;
|
||||
|
||||
@ -119,7 +119,7 @@ Graphite::RollupRule selectPatternForPath(
|
||||
}
|
||||
else
|
||||
{
|
||||
if (pattern.regexp->match(path.data, path.size))
|
||||
if (pattern.regexp->match(path.data(), path.size()))
|
||||
{
|
||||
/// General pattern with matched path
|
||||
if (pattern.type == pattern.TypeAll)
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/StringRef.h>
|
||||
#include <Common/OptimizedRegularExpression.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
@ -147,7 +146,7 @@ struct Params
|
||||
|
||||
using RollupRule = std::pair<const RetentionPattern *, const AggregationPattern *>;
|
||||
|
||||
Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, StringRef path);
|
||||
Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, std::string_view path);
|
||||
|
||||
void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params);
|
||||
|
||||
|
@ -120,7 +120,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge()
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
|
||||
StringRef next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->getRow());
|
||||
std::string_view next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->getRow()).toView();
|
||||
bool new_path = is_first || next_path != current_group_path;
|
||||
|
||||
is_first = false;
|
||||
@ -190,7 +190,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge()
|
||||
current_subgroup_newest_row.set(current, sources[current.impl->order].chunk);
|
||||
|
||||
/// Small hack: group and subgroups have the same path, so we can set current_group_path here instead of startNextGroup
|
||||
/// But since we keep in memory current_subgroup_newest_row's block, we could use StringRef for current_group_path and don't
|
||||
/// But since we keep in memory current_subgroup_newest_row's block, we could use string_view for current_group_path and don't
|
||||
/// make deep copy of the path.
|
||||
current_group_path = next_path;
|
||||
}
|
||||
|
@ -92,7 +92,7 @@ private:
|
||||
*/
|
||||
|
||||
/// Path name of current bucket
|
||||
StringRef current_group_path;
|
||||
std::string_view current_group_path;
|
||||
|
||||
static constexpr size_t max_row_refs = 2; /// current_subgroup_newest_row, current_row.
|
||||
/// Last row with maximum version for current primary key (time bucket).
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <Common/ActionBlocker.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <base/StringRef.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Storages/Distributed/DirectoryMonitor.h>
|
||||
|
@ -20,9 +20,9 @@ using namespace DB;
|
||||
|
||||
static SharedContextHolder shared_context = Context::createShared();
|
||||
|
||||
std::vector<StringRef> loadMetrics(const std::string & metrics_file)
|
||||
std::vector<std::string_view> loadMetrics(const std::string & metrics_file)
|
||||
{
|
||||
std::vector<StringRef> metrics;
|
||||
std::vector<std::string_view> metrics;
|
||||
|
||||
FILE * stream;
|
||||
char * line = nullptr;
|
||||
@ -47,7 +47,7 @@ std::vector<StringRef> loadMetrics(const std::string & metrics_file)
|
||||
}
|
||||
if (l > 0)
|
||||
{
|
||||
metrics.push_back(StringRef(strdup(line), l));
|
||||
metrics.emplace_back(std::string_view(strdup(line), l));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -80,7 +80,7 @@ void bench(const std::string & config_path, const std::string & metrics_file, si
|
||||
Graphite::Params params;
|
||||
setGraphitePatternsFromConfig(context, "graphite_rollup", params);
|
||||
|
||||
std::vector<StringRef> metrics = loadMetrics(metrics_file);
|
||||
std::vector<std::string_view> metrics = loadMetrics(metrics_file);
|
||||
|
||||
std::vector<double> durations(metrics.size());
|
||||
size_t j, i;
|
||||
@ -99,15 +99,15 @@ void bench(const std::string & config_path, const std::string & metrics_file, si
|
||||
|
||||
if (j == 0 && verbose)
|
||||
{
|
||||
std::cout << metrics[i].data << ": rule with regexp '" << rule.second->regexp_str << "' found\n";
|
||||
std::cout << metrics[i].data() << ": rule with regexp '" << rule.second->regexp_str << "' found\n";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (i = 0; i < metrics.size(); i++)
|
||||
{
|
||||
std::cout << metrics[i].data << " " << durations[i] / n << " ns\n";
|
||||
free(const_cast<void *>(static_cast<const void *>(metrics[i].data)));
|
||||
std::cout << metrics[i].data() << " " << durations[i] / n << " ns\n";
|
||||
free(const_cast<void *>(static_cast<const void *>(metrics[i].data())));
|
||||
}
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user