ClickHouse/dbms/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp

296 lines
11 KiB
C++
Raw Normal View History

#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ExpressionListParsers.h>
#include <IO/Operators.h>
namespace DB
{
namespace ErrorCodes
{
extern const int METADATA_MISMATCH;
}
static String formattedAST(const ASTPtr & ast)
{
if (!ast)
return "";
std::stringstream ss;
formatAST(*ast, ss, false, true);
return ss.str();
}
ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data)
{
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos];
2019-08-26 14:24:29 +00:00
const auto data_settings = data.getSettings();
sampling_expression = formattedAST(data.sample_by_ast);
2019-08-13 10:29:31 +00:00
index_granularity = data_settings->index_granularity;
merging_params_mode = static_cast<int>(data.merging_params.mode);
sign_column = data.merging_params.sign_column;
2019-12-27 14:36:59 +00:00
/// This code may looks strange, but previously we had only one entity: PRIMARY KEY (or ORDER BY, it doesn't matter)
/// Now we have two different entities ORDER BY and it's optional prefix -- PRIMARY KEY.
2019-12-27 16:34:50 +00:00
/// In most cases user doesn't specify PRIMARY KEY and semantically it's equal to ORDER BY.
2019-12-27 14:36:59 +00:00
/// So rules in zookeeper metadata is following:
/// - When we have only ORDER BY, than store it in "primary key:" row of /metadata
/// - When we have both, than store PRIMARY KEY in "primary key:" row and ORDER BY in "sorting key:" row of /metadata
if (!data.primary_key_ast)
primary_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.order_by_ast));
else
{
primary_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.primary_key_ast));
sorting_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.order_by_ast));
}
data_format_version = data.format_version;
if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
partition_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.partition_by_ast));
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
ttl_table = formattedAST(data.ttl_table_ast);
2019-05-02 16:07:23 +00:00
skip_indices = data.getIndices().toString();
if (data.canUseAdaptiveGranularity())
2019-08-13 10:29:31 +00:00
index_granularity_bytes = data_settings->index_granularity_bytes;
else
index_granularity_bytes = 0;
constraints = data.getConstraints().toString();
}
void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const
{
out << "metadata format version: 1" << "\n"
<< "date column: " << date_column << "\n"
<< "sampling expression: " << sampling_expression << "\n"
<< "index granularity: " << index_granularity << "\n"
<< "mode: " << merging_params_mode << "\n"
<< "sign column: " << sign_column << "\n"
<< "primary key: " << primary_key << "\n";
if (data_format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{
out << "data format version: " << data_format_version.toUnderType() << "\n"
<< "partition key: " << partition_key << "\n";
}
if (!sorting_key.empty())
out << "sorting key: " << sorting_key << "\n";
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
if (!ttl_table.empty())
out << "ttl: " << ttl_table << "\n";
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
if (!skip_indices.empty())
out << "indices: " << skip_indices << "\n";
if (index_granularity_bytes != 0)
out << "granularity bytes: " << index_granularity_bytes << "\n";
if (!constraints.empty())
out << "constraints: " << constraints << "\n";
}
String ReplicatedMergeTreeTableMetadata::toString() const
{
WriteBufferFromOwnString out;
write(out);
return out.str();
}
void ReplicatedMergeTreeTableMetadata::read(ReadBuffer & in)
{
in >> "metadata format version: 1\n";
in >> "date column: " >> date_column >> "\n";
in >> "sampling expression: " >> sampling_expression >> "\n";
in >> "index granularity: " >> index_granularity >> "\n";
in >> "mode: " >> merging_params_mode >> "\n";
in >> "sign column: " >> sign_column >> "\n";
in >> "primary key: " >> primary_key >> "\n";
if (in.eof())
data_format_version = 0;
2019-06-19 12:30:56 +00:00
else if (checkString("data format version: ", in))
in >> data_format_version.toUnderType() >> "\n";
if (data_format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
in >> "partition key: " >> partition_key >> "\n";
if (checkString("sorting key: ", in))
in >> sorting_key >> "\n";
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
if (checkString("ttl: ", in))
in >> ttl_table >> "\n";
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
if (checkString("indices: ", in))
in >> skip_indices >> "\n";
if (checkString("granularity bytes: ", in))
{
in >> index_granularity_bytes >> "\n";
index_granularity_bytes_found_in_zk = true;
}
else
index_granularity_bytes = 0;
if (checkString("constraints: ", in))
in >> constraints >> "\n";
}
ReplicatedMergeTreeTableMetadata ReplicatedMergeTreeTableMetadata::parse(const String & s)
{
ReplicatedMergeTreeTableMetadata metadata;
ReadBufferFromString buf(s);
metadata.read(buf);
return metadata;
}
2020-02-14 10:17:04 +00:00
void ReplicatedMergeTreeTableMetadata::checkImmutableFieldsEquals(const ReplicatedMergeTreeTableMetadata & from_zk) const
{
if (data_format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{
if (date_column != from_zk.date_column)
throw Exception("Existing table metadata in ZooKeeper differs in date index column."
" Stored in ZooKeeper: " + from_zk.date_column + ", local: " + date_column,
ErrorCodes::METADATA_MISMATCH);
}
else if (!from_zk.date_column.empty())
2020-02-14 10:17:04 +00:00
{
throw Exception(
"Existing table metadata in ZooKeeper differs in date index column."
" Stored in ZooKeeper: " + from_zk.date_column + ", local is custom-partitioned.",
ErrorCodes::METADATA_MISMATCH);
2020-02-14 10:17:04 +00:00
}
if (sampling_expression != from_zk.sampling_expression)
throw Exception("Existing table metadata in ZooKeeper differs in sample expression."
" Stored in ZooKeeper: " + from_zk.sampling_expression + ", local: " + sampling_expression,
ErrorCodes::METADATA_MISMATCH);
if (index_granularity != from_zk.index_granularity)
throw Exception("Existing table metadata in ZooKeeper differs in index granularity."
" Stored in ZooKeeper: " + DB::toString(from_zk.index_granularity) + ", local: " + DB::toString(index_granularity),
ErrorCodes::METADATA_MISMATCH);
if (merging_params_mode != from_zk.merging_params_mode)
throw Exception("Existing table metadata in ZooKeeper differs in mode of merge operation."
" Stored in ZooKeeper: " + DB::toString(from_zk.merging_params_mode) + ", local: "
+ DB::toString(merging_params_mode),
ErrorCodes::METADATA_MISMATCH);
if (sign_column != from_zk.sign_column)
throw Exception("Existing table metadata in ZooKeeper differs in sign column."
" Stored in ZooKeeper: " + from_zk.sign_column + ", local: " + sign_column,
ErrorCodes::METADATA_MISMATCH);
/// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes
/// in formatAST code.
if (primary_key != from_zk.primary_key)
throw Exception("Existing table metadata in ZooKeeper differs in primary key."
" Stored in ZooKeeper: " + from_zk.primary_key + ", local: " + primary_key,
ErrorCodes::METADATA_MISMATCH);
if (data_format_version != from_zk.data_format_version)
throw Exception("Existing table metadata in ZooKeeper differs in data format version."
" Stored in ZooKeeper: " + DB::toString(from_zk.data_format_version.toUnderType()) +
", local: " + DB::toString(data_format_version.toUnderType()),
ErrorCodes::METADATA_MISMATCH);
if (partition_key != from_zk.partition_key)
throw Exception(
"Existing table metadata in ZooKeeper differs in partition key expression."
" Stored in ZooKeeper: " + from_zk.partition_key + ", local: " + partition_key,
ErrorCodes::METADATA_MISMATCH);
2020-02-14 10:17:04 +00:00
}
void ReplicatedMergeTreeTableMetadata::checkEquals(const ReplicatedMergeTreeTableMetadata & from_zk) const
{
checkImmutableFieldsEquals(from_zk);
if (sorting_key != from_zk.sorting_key)
{
2020-02-14 10:17:04 +00:00
throw Exception(
"Existing table metadata in ZooKeeper differs in sorting key expression."
" Stored in ZooKeeper: " + from_zk.sorting_key + ", local: " + sorting_key,
ErrorCodes::METADATA_MISMATCH);
}
if (ttl_table != from_zk.ttl_table)
{
2020-02-14 10:17:04 +00:00
throw Exception(
"Existing table metadata in ZooKeeper differs in TTL."
" Stored in ZooKeeper: " + from_zk.ttl_table +
", local: " + ttl_table,
ErrorCodes::METADATA_MISMATCH);
}
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
if (skip_indices != from_zk.skip_indices)
{
2020-02-14 10:17:04 +00:00
throw Exception(
"Existing table metadata in ZooKeeper differs in skip indexes."
" Stored in ZooKeeper: " + from_zk.skip_indices +
", local: " + skip_indices,
ErrorCodes::METADATA_MISMATCH);
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
}
if (constraints != from_zk.constraints)
{
2020-02-14 10:17:04 +00:00
throw Exception(
"Existing table metadata in ZooKeeper differs in constraints."
" Stored in ZooKeeper: " + from_zk.constraints +
", local: " + constraints,
ErrorCodes::METADATA_MISMATCH);
}
if (from_zk.index_granularity_bytes_found_in_zk && index_granularity_bytes != from_zk.index_granularity_bytes)
throw Exception("Existing table metadata in ZooKeeper differs in index granularity bytes."
" Stored in ZooKeeper: " + DB::toString(from_zk.index_granularity_bytes) +
", local: " + DB::toString(index_granularity_bytes),
ErrorCodes::METADATA_MISMATCH);
2020-02-14 10:17:04 +00:00
}
ReplicatedMergeTreeTableMetadata::Diff
ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk) const
{
checkImmutableFieldsEquals(from_zk);
Diff diff;
if (sorting_key != from_zk.sorting_key)
{
diff.sorting_key_changed = true;
diff.new_sorting_key = from_zk.sorting_key;
}
if (ttl_table != from_zk.ttl_table)
{
diff.ttl_table_changed = true;
diff.new_ttl_table = from_zk.ttl_table;
}
if (skip_indices != from_zk.skip_indices)
{
diff.skip_indices_changed = true;
diff.new_skip_indices = from_zk.skip_indices;
}
if (constraints != from_zk.constraints)
{
diff.constraints_changed = true;
diff.new_constraints = from_zk.constraints;
}
return diff;
}
}