mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Fix codespell warnings. Split style checks. Update style checks docker
This commit is contained in:
parent
4561527d4d
commit
ce1e2b18a3
@ -38,7 +38,7 @@
|
||||
* = log(6.3*5.3) + lgamma(5.3)
|
||||
* = log(6.3*5.3*4.3*3.3*2.3) + lgamma(2.3)
|
||||
* 2. Polynomial approximation of lgamma around its
|
||||
* minimun ymin=1.461632144968362245 to maintain monotonicity.
|
||||
* minimum ymin=1.461632144968362245 to maintain monotonicity.
|
||||
* On [ymin-0.23, ymin+0.27] (i.e., [1.23164,1.73163]), use
|
||||
* Let z = x-ymin;
|
||||
* lgamma(x) = -1.214862905358496078218 + z^2*poly(z)
|
||||
|
@ -202,7 +202,7 @@ long double powl(long double x, long double y)
|
||||
volatile long double z=0;
|
||||
long double w=0, W=0, Wa=0, Wb=0, ya=0, yb=0, u=0;
|
||||
|
||||
/* make sure no invalid exception is raised by nan comparision */
|
||||
/* make sure no invalid exception is raised by nan comparison */
|
||||
if (isnan(x)) {
|
||||
if (!isnan(y) && y == 0.0)
|
||||
return 1.0;
|
||||
|
@ -129,7 +129,7 @@ using namespace pcg_extras;
|
||||
*
|
||||
* default_multiplier<uint32_t>::multiplier()
|
||||
*
|
||||
* gives you the default multipler for 32-bit integers. We use the name
|
||||
* gives you the default multiplier for 32-bit integers. We use the name
|
||||
* of the constant and not a generic word like value to allow these classes
|
||||
* to be used as mixins.
|
||||
*/
|
||||
|
@ -4,5 +4,8 @@ FROM ubuntu:20.04
|
||||
RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes shellcheck libxml2-utils git python3-pip && pip3 install codespell
|
||||
|
||||
|
||||
CMD cd /ClickHouse/utils/check-style && ./check-style -n | tee /test_output/style_output.txt && \
|
||||
CMD cd /ClickHouse/utils/check-style && \
|
||||
./check-style -n | tee /test_output/style_output.txt && \
|
||||
./check-typos | tee /test_output/typos_output.txt && \
|
||||
./check-whitespaces -n | tee /test_output/whitespaces_output.txt && \
|
||||
./check-duplicate-includes.sh | tee /test_output/duplicate_output.txt
|
||||
|
@ -107,7 +107,7 @@ public:
|
||||
/// TODO Do positions need to be 1-based for this function?
|
||||
size_t position = columns[1]->getUInt(row_num);
|
||||
|
||||
/// If position is larger than size to which array will be cutted - simply ignore value.
|
||||
/// If position is larger than size to which array will be cut - simply ignore value.
|
||||
if (length_to_resize && position >= length_to_resize)
|
||||
return;
|
||||
|
||||
|
@ -97,7 +97,7 @@ public:
|
||||
/// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/
|
||||
void savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir);
|
||||
|
||||
/// Set path of main config.xml. It will be cutted from all configs placed to preprocessed_configs/
|
||||
/// Set path of main config.xml. It will be cut from all configs placed to preprocessed_configs/
|
||||
static void setConfigPath(const std::string & config_path);
|
||||
|
||||
public:
|
||||
|
@ -1008,7 +1008,7 @@ public:
|
||||
* then deleting a erased_key_position will break search for it, so we need to move next_element
|
||||
* to erased_key_position. Now we have empty place at next_element, so we apply the identical
|
||||
* procedure for it.
|
||||
* If an empty element is encoutered then means that there is no more next elements for which we can
|
||||
* If an empty element is encountered then means that there is no more next elements for which we can
|
||||
* break the search so we can exit.
|
||||
*/
|
||||
|
||||
|
@ -44,7 +44,7 @@ namespace QueryProcessingStage
|
||||
: "Unknown stage";
|
||||
}
|
||||
|
||||
/// This methid is used for the program options,
|
||||
/// This method is used for the program options,
|
||||
/// hence it accept under_score notation for stage:
|
||||
/// - complete
|
||||
/// - fetch_columns
|
||||
|
@ -185,7 +185,7 @@ using Decimal64 = Decimal<Int64>;
|
||||
using Decimal128 = Decimal<Int128>;
|
||||
using Decimal256 = Decimal<Int256>;
|
||||
|
||||
// Distinguishable type to allow function resultion/deduction based on value type,
|
||||
// Distinguishable type to allow function resolution/deduction based on value type,
|
||||
// but also relatively easy to convert to/from Decimal64.
|
||||
class DateTime64 : public Decimal64
|
||||
{
|
||||
|
@ -204,7 +204,7 @@ void DataTypeMap::serializeText(const IColumn & column, size_t row_num, WriteBuf
|
||||
|
||||
void DataTypeMap::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
// need_safe_get_int_key is set for Interger to prevent to readIntTextUnsafe
|
||||
// need_safe_get_int_key is set for Integer to prevent to readIntTextUnsafe
|
||||
bool need_safe_get_int_key = isInteger(key_type);
|
||||
|
||||
deserializeTextImpl(column, istr, need_safe_get_int_key,
|
||||
@ -226,7 +226,7 @@ void DataTypeMap::serializeTextJSON(const IColumn & column, size_t row_num, Writ
|
||||
|
||||
void DataTypeMap::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
// need_safe_get_int_key is set for Interger to prevent to readIntTextUnsafe
|
||||
// need_safe_get_int_key is set for Integer to prevent to readIntTextUnsafe
|
||||
bool need_safe_get_int_key = isInteger(key_type);
|
||||
|
||||
deserializeTextImpl(column, istr, need_safe_get_int_key,
|
||||
|
@ -67,7 +67,7 @@ public:
|
||||
* If the data type require single stream (it's true for most of data types), the stream will have empty path.
|
||||
* Otherwise, the path can have components like "array elements", "array sizes", etc.
|
||||
*
|
||||
* For multidimensional arrays, path can have arbiraty length.
|
||||
* For multidimensional arrays, path can have arbitrary length.
|
||||
* As an example, for 2-dimensional arrays of numbers we have at least three streams:
|
||||
* - array sizes; (sizes of top level arrays)
|
||||
* - array elements / array sizes; (sizes of second level (nested) arrays)
|
||||
|
@ -229,7 +229,7 @@ void DatabaseOnDisk::removeDetachedPermanentlyFlag(const String & table_name, co
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while trying to remove permanenty detached flag. Table {}.{} may still be marked as permanently detached, and will not be reattached during server restart.", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
e.addMessage("while trying to remove permanently detached flag. Table {}.{} may still be marked as permanently detached, and will not be reattached during server restart.", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -264,7 +264,7 @@ void DatabaseOnDisk::detachTablePermanently(const String & table_name)
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while trying to set permanenty detached flag. Table {}.{} may be reattached during server restart.", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
e.addMessage("while trying to set permanently detached flag. Table {}.{} may be reattached during server restart.", backQuote(getDatabaseName()), backQuote(table_name));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
@ -147,7 +147,7 @@ void CacheDictionary::getItemsNumberImpl(
|
||||
tryPushToUpdateQueueOrThrow(update_unit_ptr);
|
||||
waitForCurrentUpdateFinish(update_unit_ptr);
|
||||
|
||||
/// Add updated keys to asnwer.
|
||||
/// Add updated keys to answer.
|
||||
|
||||
const size_t attribute_index = getAttributeIndex(attribute.name);
|
||||
|
||||
|
@ -29,7 +29,7 @@ namespace DB
|
||||
* multiMatchAnyIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns index of any match or zero if none;
|
||||
* multiMatchAllIndices(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns an array of matched indices in any order;
|
||||
*
|
||||
* countSubstrings(haystack, needle) -- count number of occurences of needle in haystack.
|
||||
* countSubstrings(haystack, needle) -- count number of occurrences of needle in haystack.
|
||||
* countSubstringsCaseInsensitive(haystack, needle)
|
||||
*
|
||||
* Applies regexp re2 and pulls:
|
||||
|
@ -236,7 +236,7 @@ public:
|
||||
if (const ColumnConst * col_higher_is_better = checkAndGetColumnConst<ColumnUInt8>(arguments[1].column.get()))
|
||||
higher_is_better = col_higher_is_better->getBool(0);
|
||||
else
|
||||
throw Exception("Second argument for function " + getName() + " must be Constatnt boolean", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception("Second argument for function " + getName() + " must be Constant boolean", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arguments[2].column.get()))
|
||||
{
|
||||
|
@ -86,7 +86,7 @@ private:
|
||||
ColumnPtr executeTuple(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const;
|
||||
|
||||
/** For a map the function finds the matched value for a key.
|
||||
* Currently implemented just as linear seach in array.
|
||||
* Currently implemented just as linear search in array.
|
||||
* However, optimizations are possible.
|
||||
*/
|
||||
ColumnPtr executeMap(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const;
|
||||
|
@ -15,7 +15,7 @@ namespace ErrorCodes
|
||||
|
||||
/** Writes data to existing std::vector or similar type. When not enough space, it doubles vector size.
|
||||
*
|
||||
* In destructor, vector is cutted to the size of written data.
|
||||
* In destructor, vector is cut to the size of written data.
|
||||
* You can call 'finalize' to resize earlier.
|
||||
*
|
||||
* The vector should live until this object is destroyed or until the 'finish' method is called.
|
||||
|
@ -268,7 +268,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
const auto & selects = select_query.list_of_selects->children;
|
||||
const auto & union_modes = select_query.list_of_modes;
|
||||
|
||||
/// ASTSelectWithUnionQuery is not normalized now, so it may pass some querys which can be Trivial select querys
|
||||
/// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries
|
||||
is_trivial_insert_select
|
||||
= std::all_of(
|
||||
union_modes.begin(),
|
||||
|
@ -57,7 +57,7 @@ BlockIO InterpreterOptimizeQuery::execute()
|
||||
{
|
||||
// Deduplication is performed only for adjacent rows in a block,
|
||||
// and all rows in block are in the sorting key order within a single partition,
|
||||
// hence deduplication always implicitly takes sorting keys and parition keys in account.
|
||||
// hence deduplication always implicitly takes sorting keys and partition keys in account.
|
||||
// So we just explicitly state that limitation in order to avoid confusion.
|
||||
if (std::find(column_names.begin(), column_names.end(), required_col) == column_names.end())
|
||||
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN,
|
||||
|
@ -132,7 +132,7 @@ struct CustomizeAggregateFunctionsSuffixData
|
||||
}
|
||||
};
|
||||
|
||||
// Used to rewrite aggregate functions with -OrNull suffix in some cases, such as sumIfOrNull, we shoule rewrite to sumOrNullIf
|
||||
// Used to rewrite aggregate functions with -OrNull suffix in some cases, such as sumIfOrNull, we should rewrite to sumOrNullIf
|
||||
struct CustomizeAggregateFunctionsMoveSuffixData
|
||||
{
|
||||
using TypeToVisit = ASTFunction;
|
||||
|
@ -57,7 +57,7 @@ JoinExpr::JoinExpr(JoinExpr::ExprType type, JoinExpr::JoinOpType op, JoinExpr::J
|
||||
|
||||
ASTPtr JoinExpr::convertToOld() const
|
||||
{
|
||||
/** The sole convertable chain of Join's may look like:
|
||||
/** The sole convertible chain of Join's may look like:
|
||||
*
|
||||
* … FROM table1 JOIN table2 ON SMTH JOIN table3 ON SMTH JOIN …
|
||||
*
|
||||
|
@ -13,7 +13,7 @@ Basic principles in code
|
||||
**predetermined order** and with **predetermined type**: some elements may be `nullptr` to preserve positions of other elements.
|
||||
- The order may be defined as a position in vector from the start, the last element, and some pattern of variable number of elements
|
||||
in between. It's convenient to define `enum ChildIndex : Uint8 {…}` with index numbers for each class.
|
||||
- If there is more than one variable pack of elements or the order can't be determenistic, then wrap elements into the lists and store the
|
||||
- If there is more than one variable pack of elements or the order can't be deterministic, then wrap elements into the lists and store the
|
||||
multi-level structure (see `ColumnExpr::ExprType::FUNCTION` for example).
|
||||
- Don't do multi-level structure just for nothing or to mimic the parse tree: the less is depth the better.
|
||||
- The whole grammar separates expressions for databases, tables and columns. That way we already assess the semantics on the parser level.
|
||||
|
@ -14,7 +14,7 @@ bool ParserOptimizeQueryColumnsSpecification::parseImpl(Pos & pos, ASTPtr & node
|
||||
{
|
||||
// Do not allow APPLY and REPLACE transformers.
|
||||
// Since we use Columns Transformers only to get list of columns,
|
||||
// ad we can't actuall modify content of the columns for deduplication.
|
||||
// we can't actually modify content of the columns for deduplication.
|
||||
const auto allowed_transformers = ParserColumnsTransformers::ColumnTransformers{ParserColumnsTransformers::ColumnTransformer::EXCEPT};
|
||||
|
||||
return ParserColumnsMatcher(allowed_transformers).parse(pos, node, expected)
|
||||
|
@ -21,7 +21,7 @@ message NameAndType {
|
||||
string type = 2;
|
||||
}
|
||||
|
||||
// Desribes an external table - a table which will exists only while a query is executing.
|
||||
// Describes an external table - a table which will exists only while a query is executing.
|
||||
message ExternalTable {
|
||||
// Name of the table. If omitted, "_data" is used.
|
||||
string name = 1;
|
||||
|
@ -637,7 +637,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
|
||||
|
||||
if (!columns.empty())
|
||||
{
|
||||
/// If some columns absent in part, then evaulate default values
|
||||
/// If some columns absent in part, then evaluate default values
|
||||
if (should_evaluate_missing_defaults)
|
||||
{
|
||||
auto block = prev_reader->sample_block.cloneWithColumns(read_result.columns);
|
||||
@ -681,7 +681,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
|
||||
merge_tree_reader->fillMissingColumns(read_result.columns, should_evaluate_missing_defaults,
|
||||
read_result.num_rows);
|
||||
|
||||
/// If some columns absent in part, then evaulate default values
|
||||
/// If some columns absent in part, then evaluate default values
|
||||
if (should_evaluate_missing_defaults)
|
||||
merge_tree_reader->evaluateMissingDefaults({}, read_result.columns);
|
||||
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
*/
|
||||
block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context);
|
||||
|
||||
/// Trasform WHERE section to Reverse Polish notation
|
||||
/// Transform WHERE section to Reverse Polish notation
|
||||
const ASTSelectQuery & select = typeid_cast<const ASTSelectQuery &>(*query_info.query);
|
||||
if (select.where())
|
||||
{
|
||||
|
@ -213,7 +213,7 @@ TEST_P(ReplicatedMergeTreeLogEntryDataTest, transcode)
|
||||
|
||||
// Enabling this warning would ruin test brievity without adding anything else in return,
|
||||
// since most of the fields have default constructors or be will be zero-initialized as by standard,
|
||||
// so values are predicatable and stable accross runs.
|
||||
// so values are predicatable and stable across runs.
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wmissing-field-initializers"
|
||||
|
||||
@ -301,7 +301,7 @@ INSTANTIATE_TEST_SUITE_P(Merge, ReplicatedMergeTreeLogEntryDataTest,
|
||||
|
||||
// This is just an example of how to set all fields. Can't be used as is since depending on type,
|
||||
// only some fields are serialized/deserialized, and even if everything works perfectly,
|
||||
// some fileds in deserialized object would be unset (hence differ from expected).
|
||||
// some fields in deserialized object would be unset (hence differ from expected).
|
||||
// INSTANTIATE_TEST_SUITE_P(Full, ReplicatedMergeTreeLogEntryDataTest,
|
||||
// ::testing::ValuesIn(std::initializer_list<ReplicatedMergeTreeLogEntryData>{
|
||||
// {
|
||||
|
@ -56,7 +56,7 @@ Block EmbeddedRocksDBBlockInputStream::readImpl()
|
||||
finished = !iterator->Valid();
|
||||
if (!iterator->status().ok())
|
||||
{
|
||||
throw Exception("Engine " + getName() + " got error while seeking key value datas: " + iterator->status().ToString(),
|
||||
throw Exception("Engine " + getName() + " got error while seeking key value data: " + iterator->status().ToString(),
|
||||
ErrorCodes::ROCKSDB_ERROR);
|
||||
}
|
||||
return sample_block.cloneWithColumns(std::move(columns));
|
||||
|
@ -61,13 +61,13 @@ struct StorageInMemoryMetadata
|
||||
/// Sets constraints
|
||||
void setConstraints(ConstraintsDescription constraints_);
|
||||
|
||||
/// Set partition key for storage (methods bellow, are just wrappers for this struct).
|
||||
/// Set partition key for storage (methods below, are just wrappers for this struct).
|
||||
void setPartitionKey(const KeyDescription & partition_key_);
|
||||
/// Set sorting key for storage (methods bellow, are just wrappers for this struct).
|
||||
/// Set sorting key for storage (methods below, are just wrappers for this struct).
|
||||
void setSortingKey(const KeyDescription & sorting_key_);
|
||||
/// Set primary key for storage (methods bellow, are just wrappers for this struct).
|
||||
/// Set primary key for storage (methods below, are just wrappers for this struct).
|
||||
void setPrimaryKey(const KeyDescription & primary_key_);
|
||||
/// Set sampling key for storage (methods bellow, are just wrappers for this struct).
|
||||
/// Set sampling key for storage (methods below, are just wrappers for this struct).
|
||||
void setSamplingKey(const KeyDescription & sampling_key_);
|
||||
|
||||
/// Set common table TTLs
|
||||
|
@ -1,14 +1,14 @@
|
||||
#!/usr/bin/env bash
|
||||
set +x
|
||||
|
||||
# Check for typos in code.
|
||||
|
||||
ROOT_PATH=$(git rev-parse --show-toplevel)
|
||||
CURDIR=$(dirname "${BASH_SOURCE[0]}")
|
||||
|
||||
codespell \
|
||||
--skip '*generated*,*gperf*,*.bin,*.mrk*,*.idx,checksums.txt,*.dat,*.pyc,*.kate-swp' \
|
||||
--ignore-words "${CURDIR}/codespell-ignore-words.list" \
|
||||
--exclude-file "${CURDIR}/codespell-ignore-lines.list" \
|
||||
--skip '*generated*,*gperf*,*.bin,*.mrk*,*.idx,checksums.txt,*.dat,*.pyc,*.kate-swp,*obfuscateQueries.cpp' \
|
||||
--ignore-words "${ROOT_PATH}/utils/check-style/codespell-ignore-words.list" \
|
||||
--exclude-file "${ROOT_PATH}/utils/check-style/codespell-ignore-lines.list" \
|
||||
--quiet-level 2 \
|
||||
"$ROOT_PATH"/{src,base,programs,utils} \
|
||||
$@ | grep -P '.' \
|
||||
|
9
utils/check-style/check-whitespaces
Executable file
9
utils/check-style/check-whitespaces
Executable file
@ -0,0 +1,9 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
ROOT_PATH=$(git rev-parse --show-toplevel)
|
||||
EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing/|Parsers/New'
|
||||
|
||||
# Double whitespaces
|
||||
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null |
|
||||
grep -vP $EXCLUDE_DIRS |
|
||||
while read i; do $ROOT_PATH/utils/check-style/double-whitespaces.pl < $i || echo -e "^ File $i contains double whitespaces\n"; done
|
@ -1,7 +1,14 @@
|
||||
thenn
|
||||
wthenn
|
||||
fpr
|
||||
creat
|
||||
parsering
|
||||
nd
|
||||
ect
|
||||
pullrequest
|
||||
pullrequests
|
||||
ans
|
||||
splitted
|
||||
thenn
|
||||
ths
|
||||
offsett
|
||||
numer
|
||||
|
Loading…
Reference in New Issue
Block a user