Merge pull request #2463 from yandex/pvs-studio

Fixes according to PVS-Studio report.
This commit is contained in:
alexey-milovidov 2018-06-04 22:59:47 +03:00 committed by GitHub
commit 88efe5ad56
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
34 changed files with 10027 additions and 108 deletions

View File

@ -165,12 +165,13 @@ public:
set.resize(reserved);
// Specialized here because there's no deserialiser for StringRef
size_t count = 0;
readVarUInt(count, buf);
for (size_t i = 0; i < count; ++i)
size_t size = 0;
readVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
{
auto ref = readStringBinaryInto(*arena, buf);
UInt64 count, error;
UInt64 count;
UInt64 error;
readVarUInt(count, buf);
readVarUInt(error, buf);
set.insert(ref, count, error);

View File

@ -3,6 +3,8 @@
#include <city.h>
#include <type_traits>
#include <ext/bit_cast.h>
#include <AggregateFunctions/UniquesHashSet.h>
#include <IO/WriteHelpers.h>
@ -185,9 +187,7 @@ template <> struct AggregateFunctionUniqTraits<Float32>
{
static UInt64 hash(Float32 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return res;
return ext::bit_cast<UInt64>(x);
}
};
@ -195,9 +195,7 @@ template <> struct AggregateFunctionUniqTraits<Float64>
{
static UInt64 hash(Float64 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
return res;
return ext::bit_cast<UInt64>(x);
}
};
@ -220,8 +218,7 @@ template <> struct AggregateFunctionUniqCombinedTraits<Float32>
{
static UInt32 hash(Float32 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
UInt64 res = ext::bit_cast<UInt64>(x);
return static_cast<UInt32>(intHash64(res));
}
};
@ -230,8 +227,7 @@ template <> struct AggregateFunctionUniqCombinedTraits<Float64>
{
static UInt32 hash(Float64 x)
{
UInt64 res = 0;
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&x), sizeof(x));
UInt64 res = ext::bit_cast<UInt64>(x);
return static_cast<UInt32>(intHash64(res));
}
};

View File

@ -749,7 +749,7 @@ public:
{
medium.getMany(levels, levels_permutation, size, result);
}
else if (kind == Kind::Large)
else /*if (kind == Kind::Large)*/
{
large->getMany(levels, levels_permutation, size, result);
}

View File

@ -123,11 +123,11 @@ void * Allocator<clear_memory_>::realloc(void * buf, size_t old_size, size_t new
{
CurrentMemoryTracker::realloc(old_size, new_size);
buf = ::realloc(buf, new_size);
if (nullptr == buf)
void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
DB::throwFromErrno("Allocator: Cannot realloc from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
buf = new_buf;
if (clear_memory && new_size > old_size)
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
}

View File

@ -169,9 +169,9 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root,
bool remove = false;
if (with_node->nodeType() == Node::ELEMENT_NODE)
{
Element * with_element = dynamic_cast<Element *>(with_node);
remove = with_element->hasAttribute("remove");
bool replace = with_element->hasAttribute("replace");
Element & with_element = dynamic_cast<Element &>(*with_node);
remove = with_element.hasAttribute("remove");
bool replace = with_element.hasAttribute("replace");
if (remove && replace)
throw Poco::Exception("both remove and replace attributes set for element <" + with_node->nodeName() + ">");
@ -189,7 +189,7 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root,
}
else if (replace)
{
with_element->removeAttribute("replace");
with_element.removeAttribute("replace");
NodePtr new_node = config->importNode(with_node, true);
config_root->replaceChild(new_node, config_node);
}
@ -294,17 +294,17 @@ void ConfigProcessor::doIncludesRecursive(
}
else
{
Element * element = dynamic_cast<Element *>(node);
Element & element = dynamic_cast<Element &>(*node);
element->removeAttribute("incl");
element->removeAttribute("from_zk");
element.removeAttribute("incl");
element.removeAttribute("from_zk");
if (replace)
{
while (Node * child = node->firstChild())
node->removeChild(child);
element->removeAttribute("replace");
element.removeAttribute("replace");
}
const NodeListPtr children = node_to_include->childNodes();
@ -317,7 +317,7 @@ void ConfigProcessor::doIncludesRecursive(
const NamedNodeMapPtr from_attrs = node_to_include->attributes();
for (size_t i = 0, size = from_attrs->length(); i < size; ++i)
{
element->setAttributeNode(dynamic_cast<Attr *>(config->importNode(from_attrs->item(i), true)));
element.setAttributeNode(dynamic_cast<Attr *>(config->importNode(from_attrs->item(i), true)));
}
included_something = true;

View File

@ -21,18 +21,13 @@ using RWLockFIFOPtr = std::shared_ptr<RWLockFIFO>;
class RWLockFIFO : public std::enable_shared_from_this<RWLockFIFO>
{
public:
static RWLockFIFOPtr create()
{
return RWLockFIFOPtr(new RWLockFIFO);
}
enum Type
{
Read,
Write
};
private:
/// Client is that who wants to acquire the lock.
struct Client
{
@ -48,6 +43,12 @@ public:
Type type;
};
public:
static RWLockFIFOPtr create()
{
return RWLockFIFOPtr(new RWLockFIFO);
}
/// Just use LockHandler::reset() to release the lock
class LockHandlerImpl;
@ -69,7 +70,6 @@ public:
Clients getClientsInTheQueue() const;
private:
RWLockFIFO() = default;
struct Group;

View File

@ -54,7 +54,7 @@ struct RadixSortFloatTransform
static KeyBits forward(KeyBits x)
{
return x ^ (-((x >> (sizeof(KeyBits) * 8 - 1) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)))));
return x ^ (-(x >> (sizeof(KeyBits) * 8 - 1) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1))));
}
static KeyBits backward(KeyBits x)

View File

@ -359,7 +359,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(MemoryTracker
* Main thread knows, that there will be result for 'output_order' place.
* Main thread must return results exactly in 'output_order', so that is important.
*/
parallel_merge_data->merged_blocks[output_order];
parallel_merge_data->merged_blocks[output_order]; //-V607
}
}

View File

@ -91,8 +91,8 @@ private:
IndexForNativeFormat::Blocks::const_iterator index_block_end;
IndexOfBlockForNativeFormat::Columns::const_iterator index_column_it;
/// If an index is specified, then `istr` must be CompressedReadBufferFromFile.
CompressedReadBufferFromFile * istr_concrete;
/// If an index is specified, then `istr` must be CompressedReadBufferFromFile. Unused otherwise.
CompressedReadBufferFromFile * istr_concrete = nullptr;
PODArray<double> avg_value_size_hints;

View File

@ -2585,7 +2585,7 @@ String FunctionArraySlice::getName() const
DataTypePtr FunctionArraySlice::getReturnTypeImpl(const DataTypes & arguments) const
{
size_t number_of_arguments = arguments.size();
const size_t number_of_arguments = arguments.size();
if (number_of_arguments < 2 || number_of_arguments > 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -3320,7 +3320,7 @@ String FunctionArrayResize::getName() const
DataTypePtr FunctionArrayResize::getReturnTypeImpl(const DataTypes & arguments) const
{
size_t number_of_arguments = arguments.size();
const size_t number_of_arguments = arguments.size();
if (number_of_arguments < 2 || number_of_arguments > 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -3343,9 +3343,9 @@ DataTypePtr FunctionArrayResize::getReturnTypeImpl(const DataTypes & arguments)
"Argument " + toString(1) + " for function " + getName() + " must be integer but it has type "
+ arguments[1]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (number_of_arguments)
if (number_of_arguments == 2)
return arguments[0];
else
else /* if (number_of_arguments == 3) */
return std::make_shared<DataTypeArray>(getLeastSupertype({array_type->getNestedType(), arguments[2]}));
}

View File

@ -1038,7 +1038,6 @@ public:
return std::make_shared<DataTypeNumber<typename IndexConv::ResultType>>();
}
/// Perform function on the given block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
/// If one or both arguments passed to this function are nullable,
@ -1056,8 +1055,7 @@ public:
/// values.
bool is_nullable;
const ColumnArray * col_array = nullptr;
col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
if (col_array)
is_nullable = col_array->getData().isColumnNullable();
else

View File

@ -911,7 +911,7 @@ public:
}
if (!array_type)
throw Exception("Expected array type, found " + array_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception("Expected array type, found " + array_type_ptr->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!offsets_column)
{

View File

@ -1238,7 +1238,7 @@ struct IsNaNImpl
template <typename T>
static bool execute(const T t)
{
return t != t;
return t != t; //-V501
}
};

View File

@ -38,7 +38,7 @@ private:
/// Passed into file_in.
ReadBufferFromFileBase::ProfileCallback profile_callback;
clockid_t clock_type;
clockid_t clock_type {};
public:
CachedCompressedReadBuffer(

View File

@ -45,7 +45,7 @@ void CompressedWriteBuffer::nextImpl()
compressed_buffer.resize(header_size + LZ4_COMPRESSBOUND(uncompressed_size));
#pragma GCC diagnostic pop
compressed_buffer[0] = static_cast<UInt8>(CompressionMethodByte::LZ4);
compressed_buffer[0] = static_cast<char>(CompressionMethodByte::LZ4);
if (compression_settings.method == CompressionMethod::LZ4)
compressed_size = header_size + LZ4_compress_default(
@ -76,7 +76,7 @@ void CompressedWriteBuffer::nextImpl()
compressed_buffer.resize(header_size + ZSTD_compressBound(uncompressed_size));
compressed_buffer[0] = static_cast<UInt8>(CompressionMethodByte::ZSTD);
compressed_buffer[0] = static_cast<char>(CompressionMethodByte::ZSTD);
size_t res = ZSTD_compress(
&compressed_buffer[header_size],
@ -109,7 +109,7 @@ void CompressedWriteBuffer::nextImpl()
compressed_buffer.resize(compressed_size);
compressed_buffer[0] = static_cast<UInt8>(CompressionMethodByte::NONE);
compressed_buffer[0] = static_cast<char>(CompressionMethodByte::NONE);
unalignedStore(&compressed_buffer[1], compressed_size_32);
unalignedStore(&compressed_buffer[5], uncompressed_size_32);

View File

@ -210,7 +210,7 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date
UInt8 hour_or_day_of_month = 0;
if (num_digits == 2)
readDecimalNumber<2>(hour_or_day_of_month, digits);
else if (num_digits == 1)
else if (num_digits == 1) //-V547
readDecimalNumber<1>(hour_or_day_of_month, digits);
else
return on_error("Cannot read DateTime: logical error, unexpected branch in code", ErrorCodes::LOGICAL_ERROR);

View File

@ -2051,8 +2051,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
};
std::unique_ptr<ThreadPool> thread_pool;
if (max_threads > 1 && total_input_rows > 100000 /// TODO Make a custom threshold.
&& has_two_level)
if (max_threads > 1 && total_input_rows > 100000) /// TODO Make a custom threshold.
thread_pool = std::make_unique<ThreadPool>(max_threads);
for (const auto & bucket_blocks : bucket_to_blocks)

View File

@ -144,7 +144,7 @@ void InJoinSubqueriesPreprocessor::process(ASTSelectQuery * query) const
ASTTableExpression * table_expression = static_cast<ASTTableExpression *>(tables_element.table_expression.get());
/// If not ordinary table, skip it.
if (!table_expression || !table_expression->database_and_table_name)
if (!table_expression->database_and_table_name)
return;
/// If not really distributed table, skip it.

View File

@ -1098,7 +1098,7 @@ private:
if (!position)
position = decltype(position)(
static_cast<void *>(new typename Map::const_iterator(map.begin())),
static_cast<void *>(new typename Map::const_iterator(map.begin())), //-V572
[](void * ptr) { delete reinterpret_cast<typename Map::const_iterator *>(ptr); });
auto & it = *reinterpret_cast<typename Map::const_iterator *>(position.get());

View File

@ -69,7 +69,7 @@ public:
SetElements & getSetElements() { return *set_elements.get(); }
private:
size_t keys_size;
size_t keys_size = 0;
Sizes key_sizes;
SetVariants data;

View File

@ -1695,8 +1695,7 @@ protected:
output = io_insert.out;
}
using ExistsFuture = std::future<zkutil::ExistsResponse>;
std::unique_ptr<ExistsFuture> future_is_dirty_checker;
std::future<zkutil::ExistsResponse> future_is_dirty_checker;
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
constexpr size_t check_period_milliseconds = 500;
@ -1707,25 +1706,14 @@ protected:
if (zookeeper->expired())
throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
if (!future_is_dirty_checker)
future_is_dirty_checker = std::make_unique<ExistsFuture>(zookeeper->asyncExists(is_dirty_flag_path));
if (!future_is_dirty_checker.valid())
future_is_dirty_checker = zookeeper->asyncExists(is_dirty_flag_path);
/// check_period_milliseconds should less than average insert time of single block
/// Otherwise, the insertion will slow a little bit
if (watch.elapsedMilliseconds() >= check_period_milliseconds)
{
zkutil::ExistsResponse status;
try
{
status = future_is_dirty_checker->get();
future_is_dirty_checker.reset();
}
catch (const zkutil::KeeperException & e)
{
future_is_dirty_checker.reset();
throw;
}
zkutil::ExistsResponse status = future_is_dirty_checker.get();
if (status.error != ZooKeeperImpl::ZooKeeper::ZNONODE)
throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
@ -1747,7 +1735,7 @@ protected:
copyData(*input, *output, cancel_check, update_stats);
// Just in case
if (future_is_dirty_checker != nullptr)
if (future_is_dirty_checker.valid())
future_is_dirty_checker.get();
if (inject_fault)

View File

@ -535,7 +535,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
const Context & context,
size_t & out_key_column_num,
DataTypePtr & out_key_res_column_type,
RPNElement::MonotonicFunctionsChain & out_functions_chain)
MonotonicFunctionsChain & out_functions_chain)
{
std::vector<const ASTFunction *> chain_not_tested_for_monotonicity;
DataTypePtr key_column_type;
@ -637,7 +637,7 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
DataTypePtr key_expr_type; /// Type of expression containing key column
size_t key_arg_pos; /// Position of argument with key column (non-const argument)
size_t key_column_num; /// Number of a key column (inside sort_descr array)
RPNElement::MonotonicFunctionsChain chain;
MonotonicFunctionsChain chain;
bool is_set_const = false;
bool is_constant_transformed = false;
@ -934,7 +934,7 @@ bool KeyCondition::mayBeTrueInRange(
std::optional<Range> KeyCondition::applyMonotonicFunctionsChainToRange(
Range key_range,
RPNElement::MonotonicFunctionsChain & functions,
MonotonicFunctionsChain & functions,
DataTypePtr current_type
)
{

View File

@ -257,6 +257,22 @@ public:
String toString() const;
/** A chain of possibly monotone functions.
* If the key column is wrapped in functions that can be monotonous in some value ranges
* (for example: -toFloat64(toDayOfWeek(date))), then here the functions will be located: toDayOfWeek, toFloat64, negate.
*/
using MonotonicFunctionsChain = std::vector<FunctionBasePtr>;
static Block getBlockWithConstants(
const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns);
static std::optional<Range> applyMonotonicFunctionsChainToRange(
Range key_range,
MonotonicFunctionsChain & functions,
DataTypePtr current_type);
private:
/// The expression is stored as Reverse Polish Notation.
struct RPNElement
{
@ -289,35 +305,24 @@ public:
/// For FUNCTION_IN_RANGE and FUNCTION_NOT_IN_RANGE.
Range range;
size_t key_column;
size_t key_column = 0;
/// For FUNCTION_IN_SET, FUNCTION_NOT_IN_SET
ASTPtr in_function;
using MergeTreeSetIndexPtr = std::shared_ptr<MergeTreeSetIndex>;
MergeTreeSetIndexPtr set_index;
/** A chain of possibly monotone functions.
* If the key column is wrapped in functions that can be monotonous in some value ranges
* (for example: -toFloat64(toDayOfWeek(date))), then here the functions will be located: toDayOfWeek, toFloat64, negate.
*/
using MonotonicFunctionsChain = std::vector<FunctionBasePtr>;
mutable MonotonicFunctionsChain monotonic_functions_chain; /// The function execution does not violate the constancy.
};
static Block getBlockWithConstants(
const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns);
using AtomMap = std::unordered_map<std::string, bool(*)(RPNElement & out, const Field & value, const ASTPtr & node)>;
static const AtomMap atom_map;
static std::optional<Range> applyMonotonicFunctionsChainToRange(
Range key_range,
RPNElement::MonotonicFunctionsChain & functions,
DataTypePtr current_type);
private:
using RPN = std::vector<RPNElement>;
using ColumnIndices = std::map<String, size_t>;
using AtomMap = std::unordered_map<std::string, bool(*)(RPNElement & out, const Field & value, const ASTPtr & node)>;
public:
static const AtomMap atom_map;
private:
bool mayBeTrueInRange(
size_t used_key_size,
const Field * left_key,
@ -342,7 +347,7 @@ private:
const Context & context,
size_t & out_key_column_num,
DataTypePtr & out_key_res_column_type,
RPNElement::MonotonicFunctionsChain & out_functions_chain);
MonotonicFunctionsChain & out_functions_chain);
bool isKeyPossiblyWrappedByMonotonicFunctionsImpl(
const ASTPtr & node,

View File

@ -108,7 +108,7 @@ struct MergeTreeBlockSizePredictor
inline void updateFilteredRowsRation(size_t rows_was_read, size_t rows_was_filtered, double decay = DECAY())
{
double alpha = std::pow(1. - decay, rows_was_read);
double current_ration = rows_was_filtered / std::max<double>(1, rows_was_read);
double current_ration = rows_was_filtered / std::max(1.0, static_cast<double>(rows_was_read));
filtered_rows_ratio = current_ration < filtered_rows_ratio
? current_ration
: alpha * filtered_rows_ratio + (1.0 - alpha) * current_ration;

View File

@ -2340,7 +2340,7 @@ MergeTreeData * MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePt
return ast ? queryToString(ast) : "";
};
if (query_to_string(secondary_sort_expr_ast) != query_to_string(secondary_sort_expr_ast))
if (query_to_string(secondary_sort_expr_ast) != query_to_string(src_data->secondary_sort_expr_ast))
throw Exception("Tables have different ordering", ErrorCodes::BAD_ARGUMENTS);
if (query_to_string(partition_expr_ast) != query_to_string(src_data->partition_expr_ast))

View File

@ -467,7 +467,7 @@ void MergeTreeDataPart::loadIndex()
String index_path = getFullPath() + "primary.idx";
ReadBufferFromFile index_file = openForReading(index_path);
for (size_t i = 0; i < marks_count; ++i)
for (size_t i = 0; i < marks_count; ++i) //-V756
for (size_t j = 0; j < key_size; ++j)
storage.primary_key_data_types[j]->deserializeBinary(*loaded_index[j].get(), index_file);

View File

@ -150,7 +150,7 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPart(const String & par
/// No one has such a part and the merge is impossible.
String not_found_msg;
if (found_part_with_the_same_min_block)
if (found_part_with_the_same_max_block)
not_found_msg = "a smaller part with the same max block.";
else if (found_part_with_the_same_min_block)
not_found_msg = "a smaller part with the same min block.";

View File

@ -1796,7 +1796,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
return true;
}
if (!parts_to_add.empty() && parts_to_add.size() < all_parts.size())
if (parts_to_add.size() < all_parts.size())
{
LOG_WARNING(log, "Some (but not all) parts from REPLACE PARTITION command already exist. REPLACE PARTITION will not be atomic.");
}

View File

@ -0,0 +1,15 @@
https://www.viva64.com/ru/m/0036/
# Analyze project with 4 threads. It takes about six minutes.
pvs-studio-analyzer analyze -o pvs-studio.log -e contrib -j 4
# Generate a report with "general" diagnostics of severity 1 and 2, in "tasks" format (simple text file):
plog-converter -a GA:1,2 -t tasklist -o project.tasks pvs-studio.log
# Generate an HTML report:
plog-converter -a GA:1,2 -t fullhtml -o ./pvs-studio-html-report pvs-studio.log
# Open ./pvs-studio-html-report/index.html in your browser.

View File

@ -11,3 +11,4 @@
[[],[],[1,2],[3,4]]
[[1,2],[3,4],[5,6],[5,6]]
[[5,6],[5,6],[1,2],[3,4]]
[1,2,3,423.56,423.56]

View File

@ -12,3 +12,5 @@ select arrayResize([[1, 2], [3, 4]], -4);
select arrayResize([[1, 2], [3, 4]], 4, [5, 6]);
select arrayResize([[1, 2], [3, 4]], -4, [5, 6]);
-- different types of array elements and default value to fill
select arrayResize([1, 2, 3], 5, 423.56);

View File

@ -62,9 +62,6 @@ public:
JSON(const JSON & rhs) : ptr_begin(rhs.ptr_begin), ptr_end(rhs.ptr_end), level(rhs.level) {}
/// Для вставки в контейнеры (создаёт некорректный объект)
JSON() : ptr_begin(nullptr), ptr_end(ptr_begin + 1) {}
const char * data() const { return ptr_begin; }
const char * dataEnd() const { return ptr_end; }

View File

@ -11,8 +11,8 @@ struct NullType {};
const NullType null = {};
/** Класс для NULL-able типов.
* Использование:
/** Store NULL-able types from MySQL.
* Usage example:
* mysqlxx::Null<int> x = mysqlxx::null;
* std::cout << (x.isNull() ? "Ok." : "Fail.") << std::endl;
* x = 10;
@ -21,8 +21,8 @@ template <typename T>
class Null
{
public:
T data;
bool is_null;
T data {};
bool is_null = true;
Null() : is_null(true) {}
Null(const Null<T> &) = default;

9917
suppress_base.json Normal file

File diff suppressed because it is too large Load Diff