mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Non significant changes according to clang's -Weverything, part 1 [#CLICKHOUSE-2]
This commit is contained in:
parent
a6693cffc0
commit
734cebdf09
@ -34,7 +34,7 @@ private:
|
||||
/** Create an empty column of strings of fixed-length `n` */
|
||||
ColumnFixedString(size_t n_) : n(n_) {}
|
||||
|
||||
ColumnFixedString(const ColumnFixedString & src) : chars(src.chars.begin(), src.chars.end()), n(src.n) {};
|
||||
ColumnFixedString(const ColumnFixedString & src) : chars(src.chars.begin(), src.chars.end()), n(src.n) {}
|
||||
|
||||
public:
|
||||
std::string getName() const override { return "FixedString(" + std::to_string(n) + ")"; }
|
||||
@ -120,7 +120,7 @@ public:
|
||||
void reserve(size_t size) override
|
||||
{
|
||||
chars.reserve(n * size);
|
||||
};
|
||||
}
|
||||
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
||||
|
@ -138,7 +138,7 @@ private:
|
||||
ColumnVector() {}
|
||||
ColumnVector(const size_t n) : data(n) {}
|
||||
ColumnVector(const size_t n, const value_type x) : data(n, x) {}
|
||||
ColumnVector(const ColumnVector & src) : data(src.data.begin(), src.data.end()) {};
|
||||
ColumnVector(const ColumnVector & src) : data(src.data.begin(), src.data.end()) {}
|
||||
|
||||
/// Sugar constructor.
|
||||
ColumnVector(std::initializer_list<T> il) : data{il} {}
|
||||
|
@ -37,7 +37,7 @@ private:
|
||||
|
||||
public:
|
||||
/// Name of a Column. It is used in info messages.
|
||||
virtual std::string getName() const { return getFamilyName(); };
|
||||
virtual std::string getName() const { return getFamilyName(); }
|
||||
|
||||
/// Name of a Column kind, without parameters (example: FixedString, Array).
|
||||
virtual const char * getFamilyName() const = 0;
|
||||
@ -241,7 +241,7 @@ public:
|
||||
|
||||
/// Reserves memory for specified amount of elements. If reservation isn't possible, does nothing.
|
||||
/// It affects performance only (not correctness).
|
||||
virtual void reserve(size_t /*n*/) {};
|
||||
virtual void reserve(size_t /*n*/) {}
|
||||
|
||||
/// Size of column data in memory (may be approximate) - for profiling. Zero, if could not be determined.
|
||||
virtual size_t byteSize() const = 0;
|
||||
|
@ -108,7 +108,7 @@ public:
|
||||
* This pointer is set when memory consumption is monitored in current thread.
|
||||
* So, you just need to pass it to all the threads that handle one request.
|
||||
*/
|
||||
#if __APPLE__ && __clang__
|
||||
#if defined(__APPLE__) && defined(__clang__)
|
||||
extern __thread MemoryTracker * current_memory_tracker;
|
||||
#else
|
||||
extern thread_local MemoryTracker * current_memory_tracker;
|
||||
|
@ -264,15 +264,15 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(cons
|
||||
if (!is_trivial)
|
||||
{
|
||||
/// Compile the re2 regular expression.
|
||||
typename RegexType::Options options;
|
||||
typename RegexType::Options regexp_options;
|
||||
|
||||
if (is_case_insensitive)
|
||||
options.set_case_sensitive(false);
|
||||
regexp_options.set_case_sensitive(false);
|
||||
|
||||
if (is_dot_nl)
|
||||
options.set_dot_nl(true);
|
||||
regexp_options.set_dot_nl(true);
|
||||
|
||||
re2 = std::make_unique<RegexType>(regexp_, options);
|
||||
re2 = std::make_unique<RegexType>(regexp_, regexp_options);
|
||||
if (!re2->ok())
|
||||
throw Poco::Exception("OptimizedRegularExpression: cannot compile re2: " + regexp_ + ", error: " + re2->error());
|
||||
|
||||
|
@ -15,7 +15,7 @@ namespace StopWatchDetail
|
||||
{
|
||||
struct timespec ts;
|
||||
clock_gettime(clock_type, &ts);
|
||||
return ts.tv_sec * 1000000000ULL + ts.tv_nsec;
|
||||
return UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec);
|
||||
}
|
||||
}
|
||||
|
||||
@ -65,7 +65,7 @@ public:
|
||||
*/
|
||||
bool compareAndRestart(double seconds)
|
||||
{
|
||||
UInt64 threshold = seconds * 1000000000ULL;
|
||||
UInt64 threshold = static_cast<UInt64>(seconds * 1000000000.0);
|
||||
UInt64 current_ns = nanoseconds();
|
||||
UInt64 current_start_ns = start_ns;
|
||||
|
||||
@ -109,7 +109,7 @@ public:
|
||||
*/
|
||||
Lock compareAndRestartDeferred(double seconds)
|
||||
{
|
||||
UInt64 threshold = seconds * 1000000000ULL;
|
||||
UInt64 threshold = UInt64(seconds * 1000000000.0);
|
||||
UInt64 current_ns = nanoseconds();
|
||||
UInt64 current_start_ns = start_ns;
|
||||
|
||||
|
@ -200,14 +200,14 @@ public:
|
||||
using TWithoutRef = std::remove_reference_t<T>;
|
||||
TWithoutRef * MAY_ALIAS ptr = reinterpret_cast<TWithoutRef*>(&storage);
|
||||
return *ptr;
|
||||
};
|
||||
}
|
||||
|
||||
template <typename T> const T & get() const
|
||||
{
|
||||
using TWithoutRef = std::remove_reference_t<T>;
|
||||
const TWithoutRef * MAY_ALIAS ptr = reinterpret_cast<const TWithoutRef*>(&storage);
|
||||
return *ptr;
|
||||
};
|
||||
}
|
||||
|
||||
template <typename T> bool tryGet(T & result)
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
|
||||
bool read(MutableColumns & columns) override;
|
||||
void readPrefix() override;
|
||||
bool allowSyncAfterError() const override { return true; };
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
|
||||
std::string getDiagnosticInfo() override;
|
||||
|
@ -131,7 +131,7 @@ public:
|
||||
|
||||
String getName() const override { return "GraphiteRollupSorted"; }
|
||||
|
||||
~GraphiteRollupSortedBlockInputStream()
|
||||
~GraphiteRollupSortedBlockInputStream() override
|
||||
{
|
||||
if (aggregate_state_created)
|
||||
current_pattern->function->destroy(place_for_aggregate_state.data());
|
||||
|
@ -21,7 +21,7 @@ public:
|
||||
JSONEachRowRowInputStream(ReadBuffer & istr_, const Block & header_, bool skip_unknown_);
|
||||
|
||||
bool read(MutableColumns & columns) override;
|
||||
bool allowSyncAfterError() const override { return true; };
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
|
||||
private:
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
TSKVRowInputStream(ReadBuffer & istr_, const Block & header_, bool skip_unknown_);
|
||||
|
||||
bool read(MutableColumns & columns) override;
|
||||
bool allowSyncAfterError() const override { return true; };
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
|
||||
private:
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
|
||||
bool read(MutableColumns & columns) override;
|
||||
void readPrefix() override;
|
||||
bool allowSyncAfterError() const override { return true; };
|
||||
bool allowSyncAfterError() const override { return true; }
|
||||
void syncAfterError() override;
|
||||
|
||||
std::string getDiagnosticInfo() override;
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
|
||||
bool canBeInsideNullable() const override { return false; }
|
||||
|
||||
DataTypePtr getReturnType() const { return function->getReturnType(); };
|
||||
DataTypePtr getReturnType() const { return function->getReturnType(); }
|
||||
DataTypes getArgumentsDataTypes() const { return argument_types; }
|
||||
|
||||
/// NOTE These two functions for serializing single values are incompatible with the functions below.
|
||||
|
@ -89,7 +89,7 @@ public:
|
||||
bool haveSubtypes() const override { return true; }
|
||||
bool cannotBeStoredInTables() const override { return nested->cannotBeStoredInTables(); }
|
||||
bool textCanContainOnlyValidUTF8() const override { return nested->textCanContainOnlyValidUTF8(); }
|
||||
bool isComparable() const override { return nested->isComparable(); };
|
||||
bool isComparable() const override { return nested->isComparable(); }
|
||||
bool canBeComparedWithCollation() const override { return nested->canBeComparedWithCollation(); }
|
||||
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
bool isCategorial() const override { return true; }
|
||||
bool isEnum() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool isComparable() const override { return true; };
|
||||
bool isComparable() const override { return true; }
|
||||
};
|
||||
|
||||
|
||||
|
@ -74,9 +74,9 @@ public:
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool isComparable() const override { return true; };
|
||||
bool isComparable() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool isFixedString() const override { return true; };
|
||||
bool isFixedString() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return n; }
|
||||
bool isCategorial() const override { return true; }
|
||||
|
@ -71,7 +71,7 @@ public:
|
||||
bool cannotBeStoredInTables() const override { return nested_data_type->cannotBeStoredInTables(); }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return nested_data_type->shouldAlignRightInPrettyFormats(); }
|
||||
bool textCanContainOnlyValidUTF8() const override { return nested_data_type->textCanContainOnlyValidUTF8(); }
|
||||
bool isComparable() const override { return nested_data_type->isComparable(); };
|
||||
bool isComparable() const override { return nested_data_type->isComparable(); }
|
||||
bool canBeComparedWithCollation() const override { return nested_data_type->canBeComparedWithCollation(); }
|
||||
bool canBeUsedAsVersion() const override { return false; }
|
||||
bool isSummable() const override { return nested_data_type->isSummable(); }
|
||||
|
@ -43,7 +43,7 @@ public:
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return true; }
|
||||
bool textCanContainOnlyValidUTF8() const override { return true; }
|
||||
bool isComparable() const override { return true; };
|
||||
bool isComparable() const override { return true; }
|
||||
bool isValueRepresentedByNumber() const override { return true; }
|
||||
bool isValueRepresentedByInteger() const override;
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
|
@ -54,10 +54,10 @@ public:
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool isComparable() const override { return true; };
|
||||
bool isComparable() const override { return true; }
|
||||
bool canBeComparedWithCollation() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool isString() const override { return true; };
|
||||
bool isString() const override { return true; }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
@ -241,71 +241,71 @@ public:
|
||||
/** Can appear in table definition.
|
||||
* Counterexamples: Interval, Nothing.
|
||||
*/
|
||||
virtual bool cannotBeStoredInTables() const { return false; };
|
||||
virtual bool cannotBeStoredInTables() const { return false; }
|
||||
|
||||
/** In text formats that render "pretty" tables,
|
||||
* is it better to align value right in table cell.
|
||||
* Examples: numbers, even nullable.
|
||||
*/
|
||||
virtual bool shouldAlignRightInPrettyFormats() const { return false; };
|
||||
virtual bool shouldAlignRightInPrettyFormats() const { return false; }
|
||||
|
||||
/** Does formatted value in any text format can contain anything but valid UTF8 sequences.
|
||||
* Example: String (because it can contain arbitary bytes).
|
||||
* Counterexamples: numbers, Date, DateTime.
|
||||
* For Enum, it depends.
|
||||
*/
|
||||
virtual bool textCanContainOnlyValidUTF8() const { return false; };
|
||||
virtual bool textCanContainOnlyValidUTF8() const { return false; }
|
||||
|
||||
/** Is it possible to compare for less/greater, to calculate min/max?
|
||||
* Not necessarily totally comparable. For example, floats are comparable despite the fact that NaNs compares to nothing.
|
||||
* The same for nullable of comparable types: they are comparable (but not totally-comparable).
|
||||
*/
|
||||
virtual bool isComparable() const { return false; };
|
||||
virtual bool isComparable() const { return false; }
|
||||
|
||||
/** Does it make sense to use this type with COLLATE modifier in ORDER BY.
|
||||
* Example: String, but not FixedString.
|
||||
*/
|
||||
virtual bool canBeComparedWithCollation() const { return false; };
|
||||
virtual bool canBeComparedWithCollation() const { return false; }
|
||||
|
||||
/** If the type is totally comparable (Ints, Date, DateTime, not nullable, not floats)
|
||||
* and "simple" enough (not String, FixedString) to be used as version number
|
||||
* (to select rows with maximum version).
|
||||
*/
|
||||
virtual bool canBeUsedAsVersion() const { return false; };
|
||||
virtual bool canBeUsedAsVersion() const { return false; }
|
||||
|
||||
/** Values of data type can be summed (possibly with overflow, within the same data type).
|
||||
* Example: numbers, even nullable. Not Date/DateTime. Not Enum.
|
||||
* Enums can be passed to aggregate function 'sum', but the result is Int64, not Enum, so they are not summable.
|
||||
*/
|
||||
virtual bool isSummable() const { return false; };
|
||||
virtual bool isSummable() const { return false; }
|
||||
|
||||
/** Can be used in operations like bit and, bit shift, bit not, etc.
|
||||
*/
|
||||
virtual bool canBeUsedInBitOperations() const { return false; };
|
||||
virtual bool canBeUsedInBitOperations() const { return false; }
|
||||
|
||||
/** Can be used in boolean context (WHERE, HAVING).
|
||||
* UInt8, maybe nullable.
|
||||
*/
|
||||
virtual bool canBeUsedInBooleanContext() const { return false; };
|
||||
virtual bool canBeUsedInBooleanContext() const { return false; }
|
||||
|
||||
/** Integers, floats, not Nullable. Not Enums. Not Date/DateTime.
|
||||
*/
|
||||
virtual bool isNumber() const { return false; };
|
||||
virtual bool isNumber() const { return false; }
|
||||
|
||||
/** Integers. Not Nullable. Not Enums. Not Date/DateTime.
|
||||
*/
|
||||
virtual bool isInteger() const { return false; };
|
||||
virtual bool isUnsignedInteger() const { return false; };
|
||||
virtual bool isInteger() const { return false; }
|
||||
virtual bool isUnsignedInteger() const { return false; }
|
||||
|
||||
virtual bool isDateOrDateTime() const { return false; };
|
||||
virtual bool isDateOrDateTime() const { return false; }
|
||||
|
||||
/** Numbers, Enums, Date, DateTime. Not nullable.
|
||||
*/
|
||||
virtual bool isValueRepresentedByNumber() const { return false; };
|
||||
virtual bool isValueRepresentedByNumber() const { return false; }
|
||||
|
||||
/** Integers, Enums, Date, DateTime. Not nullable.
|
||||
*/
|
||||
virtual bool isValueRepresentedByInteger() const { return false; };
|
||||
virtual bool isValueRepresentedByInteger() const { return false; }
|
||||
|
||||
/** Values are unambiguously identified by contents of contiguous memory region,
|
||||
* that can be obtained by IColumn::getDataAt method.
|
||||
@ -314,22 +314,22 @@ public:
|
||||
* (because Array(String) values became ambiguous if you concatenate Strings).
|
||||
* Counterexamples: Nullable, Tuple.
|
||||
*/
|
||||
virtual bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const { return false; };
|
||||
virtual bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const { return false; }
|
||||
|
||||
virtual bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const
|
||||
{
|
||||
return isValueRepresentedByNumber() || isFixedString();
|
||||
};
|
||||
|
||||
virtual bool isString() const { return false; };
|
||||
virtual bool isFixedString() const { return false; };
|
||||
virtual bool isStringOrFixedString() const { return isString() || isFixedString(); };
|
||||
virtual bool isString() const { return false; }
|
||||
virtual bool isFixedString() const { return false; }
|
||||
virtual bool isStringOrFixedString() const { return isString() || isFixedString(); }
|
||||
|
||||
/** Example: numbers, Date, DateTime, FixedString, Enum... Nullable and Tuple of such types.
|
||||
* Counterexamples: String, Array.
|
||||
* It's Ok to return false for AggregateFunction despite the fact that some of them have fixed size state.
|
||||
*/
|
||||
virtual bool haveMaximumSizeOfValue() const { return false; };
|
||||
virtual bool haveMaximumSizeOfValue() const { return false; }
|
||||
|
||||
/** Size in amount of bytes in memory. Throws an exception if not haveMaximumSizeOfValue.
|
||||
*/
|
||||
@ -341,9 +341,9 @@ public:
|
||||
|
||||
/** Integers (not floats), Enum, String, FixedString.
|
||||
*/
|
||||
virtual bool isCategorial() const { return false; };
|
||||
virtual bool isCategorial() const { return false; }
|
||||
|
||||
virtual bool isEnum() const { return false; };
|
||||
virtual bool isEnum() const { return false; }
|
||||
|
||||
virtual bool isNullable() const { return false; }
|
||||
|
||||
@ -353,7 +353,7 @@ public:
|
||||
|
||||
/** If this data type cannot be wrapped in Nullable data type.
|
||||
*/
|
||||
virtual bool canBeInsideNullable() const { return false; };
|
||||
virtual bool canBeInsideNullable() const { return false; }
|
||||
|
||||
|
||||
/// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
|
||||
|
@ -36,7 +36,7 @@ std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
|
||||
|
||||
/// Copies list of tables and iterates through such snapshot.
|
||||
class DatabaseSnapshotIterator : public IDatabaseIterator
|
||||
class DatabaseSnapshotIterator final : public IDatabaseIterator
|
||||
{
|
||||
private:
|
||||
Tables tables;
|
||||
|
@ -43,7 +43,7 @@ public:
|
||||
virtual IRegionsHierarchyDataSourcePtr getDefaultHierarchySource() const = 0;
|
||||
virtual IRegionsHierarchyDataSourcePtr getHierarchySource(const std::string & name) const = 0;
|
||||
|
||||
virtual ~IRegionsHierarchiesDataProvider() {};
|
||||
virtual ~IRegionsHierarchiesDataProvider() {}
|
||||
};
|
||||
|
||||
using IRegionsHierarchiesDataProviderPtr = std::shared_ptr<IRegionsHierarchiesDataProvider>;
|
||||
|
@ -45,7 +45,6 @@ class IRegionsNamesDataProvider
|
||||
public:
|
||||
virtual ILanguageRegionsNamesDataSourcePtr getLanguageRegionsNamesSource(
|
||||
const std::string& language) const = 0;
|
||||
;
|
||||
|
||||
virtual ~IRegionsNamesDataProvider() {}
|
||||
};
|
||||
|
@ -38,7 +38,7 @@ public:
|
||||
|
||||
inline Position begin() const { return begin_pos; }
|
||||
inline Position end() const { return end_pos; }
|
||||
inline size_t size() const { return end_pos - begin_pos; }
|
||||
inline size_t size() const { return size_t(end_pos - begin_pos); }
|
||||
inline void resize(size_t size) { end_pos = begin_pos + size; }
|
||||
|
||||
inline void swap(Buffer & other)
|
||||
@ -72,10 +72,10 @@ public:
|
||||
inline Buffer & buffer() { return working_buffer; }
|
||||
|
||||
/// get (for reading and modifying) the position in the buffer
|
||||
inline Position & position() { return pos; };
|
||||
inline Position & position() { return pos; }
|
||||
|
||||
/// offset in bytes of the cursor from the beginning of the buffer
|
||||
inline size_t offset() const { return pos - working_buffer.begin(); }
|
||||
inline size_t offset() const { return size_t(pos - working_buffer.begin()); }
|
||||
|
||||
/** How many bytes have been read/written, counting those that are still in the buffer. */
|
||||
size_t count() const
|
||||
|
@ -159,7 +159,7 @@ private:
|
||||
* Return `false` in case of the end, `true` otherwise.
|
||||
* Throw an exception if something is wrong.
|
||||
*/
|
||||
virtual bool nextImpl() { return false; };
|
||||
virtual bool nextImpl() { return false; }
|
||||
};
|
||||
|
||||
|
||||
|
@ -435,8 +435,8 @@ bool tryReadJSONStringInto(Vector & s, ReadBuffer & buf)
|
||||
/// This could be used as template parameter for functions above, if you want to just skip data.
|
||||
struct NullSink
|
||||
{
|
||||
void append(const char *, size_t) {};
|
||||
void push_back(char) {};
|
||||
void append(const char *, size_t) {}
|
||||
void push_back(char) {}
|
||||
};
|
||||
|
||||
void parseUUID(const UInt8 * src36, UInt8 * dst16);
|
||||
|
@ -94,7 +94,7 @@ private:
|
||||
/** Write the data in the buffer (from the beginning of the buffer to the current position).
|
||||
* Throw an exception if something is wrong.
|
||||
*/
|
||||
virtual void nextImpl() { throw Exception("Cannot write after end of buffer.", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); };
|
||||
virtual void nextImpl() { throw Exception("Cannot write after end of buffer.", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); }
|
||||
};
|
||||
|
||||
|
||||
|
@ -116,7 +116,7 @@ namespace detail
|
||||
|
||||
while (x >= 100)
|
||||
{
|
||||
const UInt32 i = (x % 100) * 2;
|
||||
const auto i = (x % 100) * 2;
|
||||
x /= 100;
|
||||
dst[next] = digits[i + 1];
|
||||
dst[next - 1] = digits[i];
|
||||
@ -129,7 +129,7 @@ namespace detail
|
||||
}
|
||||
else
|
||||
{
|
||||
const UInt32 i = x * 2;
|
||||
const auto i = x * 2;
|
||||
dst[next] = digits[i + 1];
|
||||
dst[next - 1] = digits[i];
|
||||
}
|
||||
|
@ -53,9 +53,9 @@ struct ExternalLoaderConfigSettings
|
||||
|
||||
/** Manages user-defined objects.
|
||||
* Monitors configuration file and automatically reloads objects in a separate thread.
|
||||
* The monitoring thread wakes up every @check_period_sec seconds and checks
|
||||
* The monitoring thread wakes up every 'check_period_sec' seconds and checks
|
||||
* modification time of objects' configuration file. If said time is greater than
|
||||
* @config_last_modified, the objects are created from scratch using configuration file,
|
||||
* 'config_last_modified', the objects are created from scratch using configuration file,
|
||||
* possibly overriding currently existing objects with the same name (previous versions of
|
||||
* overridden objects will live as long as there are any users retaining them).
|
||||
*
|
||||
|
@ -94,13 +94,16 @@ inline bool operator==(StringRef_CompareAlwaysTrue, StringRef_CompareAlwaysTrue)
|
||||
}
|
||||
|
||||
|
||||
#define mix(h) ({ \
|
||||
(h) ^= (h) >> 23; \
|
||||
(h) *= 0x2127599bf4325c37ULL; \
|
||||
(h) ^= (h) >> 47; })
|
||||
|
||||
struct FastHash64
|
||||
{
|
||||
static inline uint64_t mix(uint64_t h)
|
||||
{
|
||||
h ^= h >> 23;
|
||||
h *= 0x2127599bf4325c37ULL;
|
||||
h ^= h >> 47;
|
||||
return h;
|
||||
}
|
||||
|
||||
size_t operator() (StringRef x) const
|
||||
{
|
||||
const char * buf = x.data;
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if _MSC_VER
|
||||
#if defined(_MSC_VER)
|
||||
#define likely(x) (x)
|
||||
#define unlikely(x) (x)
|
||||
#else
|
||||
|
@ -17,12 +17,12 @@ private:
|
||||
|
||||
public:
|
||||
template <class Enable = typename std::is_copy_constructible<T>::type>
|
||||
explicit StrongTypedef(const T & t_) : t(t_) {};
|
||||
explicit StrongTypedef(const T & t_) : t(t_) {}
|
||||
template <class Enable = typename std::is_move_constructible<T>::type>
|
||||
explicit StrongTypedef(T && t_) : t(std::move(t_)) {};
|
||||
explicit StrongTypedef(T && t_) : t(std::move(t_)) {}
|
||||
|
||||
template <class Enable = typename std::is_default_constructible<T>::type>
|
||||
StrongTypedef(): t() {};
|
||||
StrongTypedef(): t() {}
|
||||
|
||||
StrongTypedef(const Self &) = default;
|
||||
StrongTypedef(Self &&) = default;
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
}
|
||||
|
||||
protected:
|
||||
singleton() {};
|
||||
singleton() {}
|
||||
|
||||
private:
|
||||
singleton(const singleton &);
|
||||
|
Loading…
Reference in New Issue
Block a user