Removed zero-width spaces [#CLICKHOUSE-3].

This commit is contained in:
Alexey Milovidov 2017-03-09 07:18:41 +03:00
parent 17c35615aa
commit f24df4b648
22 changed files with 47 additions and 47 deletions

View File

@ -107,7 +107,7 @@ struct AggregateFunctionGroupArrayDataGeneric
};
/// Puts all values to an array, general case. Implemented inefficiently.
/// Puts all values to an array, general case. Implemented inefficiently.
class AggregateFunctionGroupArrayGeneric final
: public IUnaryAggregateFunction<AggregateFunctionGroupArrayDataGeneric, AggregateFunctionGroupArrayGeneric>
{

View File

@ -35,7 +35,7 @@ struct AggregateFunctionGroupUniqArrayData
};
/// Puts all values to the hash set. Returns an array of unique values. Implemented for numeric types.
/// Puts all values to the hash set. Returns an array of unique values. Implemented for numeric types.
template <typename T>
class AggregateFunctionGroupUniqArray
: public IUnaryAggregateFunction<AggregateFunctionGroupUniqArrayData<T>, AggregateFunctionGroupUniqArray<T>>

View File

@ -11,7 +11,7 @@ namespace DB
/** Not an aggregate function, but an adapter of aggregate functions,
* which any aggregate function `agg(x)` makes an aggregate function of the form `aggIf(x, cond)`.
* The adapted aggregate function takes two arguments - a value and a condition,
* and calculates the nested aggregate function for the values when the condition is satisfied.
* and calculates the nested aggregate function for the values when the condition is satisfied.
* For example, avgIf(x, cond) calculates the average x if `cond`.
*/
class AggregateFunctionIf final : public IAggregateFunction

View File

@ -18,7 +18,7 @@ namespace DB
{
/** The state is an array, into which all values are added.
/** The state is an array, into which all values are added.
* NOTE If there are few different values then this is not optimal.
* For 8 and 16-bit values it might be better to use a lookup table.
*/

View File

@ -66,7 +66,7 @@ struct Centroid
/** :param epsilon: value \delta from the article - error in the range
* quantile 0.5 (default is 0.01, i.e. 1%)
* :param max_unmerged: when accumulating count of new points beyond this
* value centroid compression is triggered
* value centroid compression is triggered
* (default is 2048, the higher the value - the
* more memory is required, but amortization of execution time increases)
*/
@ -182,7 +182,7 @@ public:
Value k = 4 * count * err * params.epsilon;
/** The ratio of the weight of the glued column pair to all values is not greater,
/** The ratio of the weight of the glued column pair to all values is not greater,
* than epsilon multiply by a certain quadratic coefficient, which in the median is 1 (4 * 1/2 * 1/2),
* and at the edges decreases and is approximately equal to the distance to the edge * 4.
*/
@ -199,14 +199,14 @@ public:
sum += l->count;
++l;
/// We skip all the values "eaten" earlier.
/// We skip all the values "eaten" earlier.
if (l != r)
*l = *r;
}
++r;
}
/// At the end of the loop, all values to the right of l were "eaten".
/// At the end of the loop, all values to the right of l were "eaten".
summary.resize(l - summary.begin() + 1);
}

View File

@ -34,10 +34,10 @@ namespace DB
*
* Three different data structures are used:
* - flat array (of all met values) of fixed length, allocated inplace, size 64 bytes; Stores 0..31 values;
* - flat array (of all values encountered), allocated separately, increasing length;
* - flat array (of all values encountered), allocated separately, increasing length;
* - a histogram (that is, value -> number), consisting of two parts
* -- for values from 0 to 1023 - in increments of 1;
* -- for values from 1024 to 30,000 - in increments of 16;
* -- for values from 0 to 1023 - in increments of 1;
* -- for values from 1024 to 30,000 - in increments of 16;
*/
#define TINY_MAX_ELEMS 31
@ -250,17 +250,17 @@ namespace detail
/// Use of UInt64 is very wasteful.
/// But UInt32 is definitely not enough, and it's too hard to invent 6-byte values.
/// Number of values for each value is smaller than `small_threshold`.
/// Number of values for each value is smaller than `small_threshold`.
UInt64 count_small[SMALL_THRESHOLD];
/// The number of values for each value from `small_threshold` to `big_threshold`, rounded to `big_precision`.
/// The number of values for each value from `small_threshold` to `big_threshold`, rounded to `big_precision`.
UInt64 count_big[BIG_SIZE];
/// Get value of quantile by index in array `count_big`.
static inline UInt16 indexInBigToValue(size_t i)
{
return (i * BIG_PRECISION) + SMALL_THRESHOLD
+ (intHash32<0>(i) % BIG_PRECISION - (BIG_PRECISION / 2)); /// A small randomization so that it is not noticeable that all the values are even.
+ (intHash32<0>(i) % BIG_PRECISION - (BIG_PRECISION / 2)); /// A small randomization so that it is not noticeable that all the values are even.
}
/// Lets you scroll through the histogram values, skipping zeros.
@ -420,7 +420,7 @@ namespace detail
return it.isValid() ? it.key() : BIG_THRESHOLD;
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
template <typename ResultType>
void getMany(const double * levels, const size_t * indices, size_t size, ResultType * result) const
@ -740,7 +740,7 @@ public:
}
}
/// Get the size values of the quantiles of the `levels` levels. Record `size` results starting with `result` address.
/// Get the size values of the quantiles of the `levels` levels. Record `size` results starting with `result` address.
template <typename ResultType>
void getMany(const double * levels, const size_t * levels_permutation, size_t size, ResultType * result) const
{

View File

@ -97,7 +97,7 @@ struct AggregateFunctionUniqExactData
static String getName() { return "uniqExact"; }
};
/// For rows, we put the SipHash values (128 bits) into the hash table.
/// For rows, we put the SipHash values (128 bits) into the hash table.
template <>
struct AggregateFunctionUniqExactData<String>
{
@ -324,7 +324,7 @@ struct OneAdder<T, Data, typename std::enable_if<
}
/// Calculates the number of different values approximately or exactly.
/// Calculates the number of different values approximately or exactly.
template <typename T, typename Data>
class AggregateFunctionUniq final : public IUnaryAggregateFunction<Data, AggregateFunctionUniq<T, Data> >
{

View File

@ -11,7 +11,7 @@ namespace DB
{
/** Counts the number of unique values up to no more than specified in the parameter.
/** Counts the number of unique values up to no more than specified in the parameter.
*
* Example: uniqUpTo(3)(UserID)
* - will count the number of unique visitors, return 1, 2, 3 or 4 if visitors > = 4.
@ -22,10 +22,10 @@ namespace DB
template <typename T>
struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
{
/** If count == threshold + 1 - this means that it is "overflowed" (values greater than threshold).
/** If count == threshold + 1 - this means that it is "overflowed" (values greater than threshold).
* In this case (for example, after calling the merge function), the `data` array does not necessarily contain the initialized values
* - example: combine a state in which there are few values, with another state that has overflowed;
* then set count to `threshold + 1`, and values from another state are not copied.
* then set count to `threshold + 1`, and values from another state are not copied.
*/
UInt8 count = 0;
@ -77,7 +77,7 @@ struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
{
writeBinary(count, wb);
/// Write values only if the state is not overflowed. Otherwise, they are not needed, and only the fact that the state is overflowed is important.
/// Write values only if the state is not overflowed. Otherwise, they are not needed, and only the fact that the state is overflowed is important.
if (count <= threshold)
wb.write(reinterpret_cast<const char *>(&data[0]), count * sizeof(data[0]));
}

View File

@ -8,7 +8,7 @@ namespace DB
{
/// For possible values for template parameters, see AggregateFunctionsMinMaxAny.h
/// For possible values for template parameters, see AggregateFunctionsMinMaxAny.h
template <typename ResultData, typename ValueData>
struct AggregateFunctionsArgMinMaxData
{

View File

@ -15,7 +15,7 @@ namespace DB
namespace
{
/// This function returns true if both values are large and comparable.
/// This function returns true if both values are large and comparable.
/// It is used to calculate the mean value by merging two sources.
/// It means that if the sizes of both sources are large and comparable, then we must apply a special
/// formula guaranteeing more stability.

View File

@ -20,7 +20,7 @@ namespace DB
  * - bias_corrected_graph.txt (1st column: the present number of unique values;
  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++)
  * 6. Generate a graph with gnuplot based on this data.
  * 7. Determine the minimum number of unique values at which it is better to correct the error
  * 7. Determine the minimum number of unique values at which it is better to correct the error
  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm.
  * 7. Accordingly, update the constant in the function getThreshold()
  * 8. Assemble ClickHouse.

View File

@ -16,7 +16,7 @@
/** Approximate calculation of anything, as a rule, is constructed according to the following scheme:
  * - some data structure is used to calculate the value of X;
  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria);
  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria);
  * - after processing all elements, the data structure is in some state S;
  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned:
  * at what real value X, the probability of finding the data structure in the obtained state S is maximal.
@ -43,10 +43,10 @@
  * - there are methods for quick reading and writing in binary and text form.
  */
/// The maximum degree of buffer size before the values are discarded
/// The maximum degree of buffer size before the values are discarded
#define UNIQUES_HASH_MAX_SIZE_DEGREE 17
/// The maximum number of elements before the values are discarded
/// The maximum number of elements before the values are discarded
#define UNIQUES_HASH_MAX_SIZE (1 << (UNIQUES_HASH_MAX_SIZE_DEGREE - 1))
/** The number of least significant bits used for thinning. The remaining high-order bits are used to determine the position in the hash table.
@ -121,7 +121,7 @@ private:
return Hash()(key);
}
/// Delete all values whose hashes do not divide by 2 ^ skip_degree
/// Delete all values whose hashes do not divide by 2 ^ skip_degree
void rehash()
{
for (size_t i = 0; i < buf_size(); ++i)

View File

@ -30,21 +30,21 @@ namespace ErrorCodes
* It can be in two variants:
*
* 1. Own its values - that is, be responsible for destroying them.
* The column consists of the values "assigned to it" after the aggregation is performed (see Aggregator, convertToBlocks function),
* or from values created by itself (see `insert` method).
* The column consists of the values "assigned to it" after the aggregation is performed (see Aggregator, convertToBlocks function),
* or from values created by itself (see `insert` method).
* In this case, `src` will be `nullptr`, and the column itself will be destroyed (call `IAggregateFunction::destroy`)
* states of aggregate functions in the destructor.
*
* 2. Do not own its values, but use values taken from another ColumnAggregateFunction column.
* 2. Do not own its values, but use values taken from another ColumnAggregateFunction column.
* For example, this is a column obtained by permutation/filtering or other transformations from another column.
* In this case, `src` will be `shared ptr` to the source column. Destruction of values will be handled by this source column.
* In this case, `src` will be `shared ptr` to the source column. Destruction of values will be handled by this source column.
*
* This solution is somewhat limited:
* - the variant in which the column contains a part of "it's own" and a part of "another's" values is not supported;
* - the variant in which the column contains a part of "it's own" and a part of "another's" values is not supported;
* - the option of having multiple source columns is not supported, which may be necessary for a more optimal merge of the two columns.
*
* These restrictions can be removed if you add an array of flags or even refcount,
* specifying which individual values should be destroyed and which ones should not.
* specifying which individual values should be destroyed and which ones should not.
* Clearly, this method would have a substantially non-zero price.
*/
class ColumnAggregateFunction final : public IColumn, public std::enable_shared_from_this<ColumnAggregateFunction>

View File

@ -345,14 +345,14 @@ private:
size_t ALWAYS_INLINE sizeAt(size_t i) const { return i == 0 ? getOffsets()[0] : (getOffsets()[i] - getOffsets()[i - 1]); }
/// Multiply values if the nested column is ColumnVector<T>.
/// Multiply values if the nested column is ColumnVector<T>.
template <typename T>
ColumnPtr replicateNumber(const Offsets_t & replicate_offsets) const;
/// Multiply the values if the nested column is ColumnString. The code is too complicated.
/// Multiply the values if the nested column is ColumnString. The code is too complicated.
ColumnPtr replicateString(const Offsets_t & replicate_offsets) const;
/** Non-constant arrays of constant values are quite rare.
/** Non-constant arrays of constant values are quite rare.
* Most functions can not work with them, and does not create such columns as a result.
* An exception is the function `replicate`(see FunctionsMiscellaneous.h), which has service meaning for the implementation of lambda functions.
* Only for its sake is the implementation of the `replicate` method for ColumnArray(ColumnConst).

View File

@ -22,7 +22,7 @@ namespace ErrorCodes
extern const int PARAMETER_OUT_OF_BOUND;
}
/** A column of values of "fixed-length string" type.
/** A column of values of "fixed-length string" type.
* If you insert a smaller string, it will be padded with zero bytes.
*/
class ColumnFixedString final : public IColumn

View File

@ -10,7 +10,7 @@ class Set;
using ConstSetPtr = std::shared_ptr<const Set>;
/** A column containing multiple values in the `IN` section.
/** A column containing multiple values in the `IN` section.
* Behaves like a constant-column (because the set is one, not its own for each line).
* This column has a nonstandard value, so it can not be obtained via a normal interface.
*/

View File

@ -25,7 +25,7 @@ namespace ErrorCodes
}
/** A column of values of type `String`.
/** A column of values of type `String`.
*/
class ColumnString final : public IColumn
{

View File

@ -28,7 +28,7 @@ namespace ErrorCodes
/** Stuff for comparing numbers.
* Integer values are compared as usual.
* Integer values are compared as usual.
* Floating-point numbers are compared this way that NaNs always end up at the end
* (if you don't do this, the sort would not work at all).
*/
@ -39,7 +39,7 @@ struct CompareHelper
static bool greater(T a, T b) { return a > b; }
/** Compares two numbers. Returns a number less than zero, equal to zero, or greater than zero if a < b, a == b, a > b, respectively.
* If one of the values is NaN, then
* If one of the values is NaN, then
* - if nan_direction_hint == -1 - NaN are considered less than all numbers;
* - if nan_direction_hint == 1 - NaN are considered to be larger than all numbers;
* Essentially: nan_direction_hint == -1 says that the comparison is for sorting in descending order.

View File

@ -11,7 +11,7 @@
#include "ResultSet.h"
/// Information where and how to add values when reading.
/// Information where and how to add values when reading.
struct Binding
{
SQLSMALLINT target_type;

View File

@ -22,7 +22,7 @@ SQLGetInfo(HDBC connection_handle,
LOG("GetInfo with info_type: " << info_type << ", out_value_max_length: " << out_value_max_length);
/** How are all these values selected?
/** How are all these values selected?
* Part of them provides true information about the capabilities of the DBMS.
* But in most cases, the possibilities are declared "in reserve" to see,
* what requests will be sent and what any software will do, meaning these features.

View File

@ -29,7 +29,7 @@ RETCODE doWith(SQLHANDLE handle_opaque, F && f)
}
/// Parse a string of the form `key1=value1;key2=value2` ... TODO Parsing values in curly brackets.
/// Parse a string of the form `key1=value1;key2=value2` ... TODO Parsing values in curly brackets.
static const char * nextKeyValuePair(const char * data, const char * end, StringRef & out_key, StringRef & out_value)
{
if (data >= end)

View File

@ -336,7 +336,7 @@ void TCPHandler::processOrdinaryQuery()
}
}
/** If data has run out, we will send the profiling data and total values to
/** If data has run out, we will send the profiling data and total values to
* the last zero block to be able to use
* this information in the suffix output of stream.
* If the request was interrupted, then `sendTotals` and other methods could not be called,