Merge remote-tracking branch 'upstream/master' into #31363_format_template_configure_in_settings

This commit is contained in:
Shaun Struwig 2024-02-01 10:03:31 +00:00
commit e1121ea70d
14 changed files with 35 additions and 26 deletions

View File

@ -136,12 +136,12 @@ namespace
{ {
void assertDigest( void assertDigest(
const KeeperStorage::Digest & first, const KeeperStorage::Digest & expected,
const KeeperStorage::Digest & second, const KeeperStorage::Digest & actual,
const Coordination::ZooKeeperRequest & request, const Coordination::ZooKeeperRequest & request,
bool committing) bool committing)
{ {
if (!KeeperStorage::checkDigest(first, second)) if (!KeeperStorage::checkDigest(expected, actual))
{ {
LOG_FATAL( LOG_FATAL(
getLogger("KeeperStateMachine"), getLogger("KeeperStateMachine"),
@ -149,9 +149,9 @@ void assertDigest(
"{}). Keeper will terminate to avoid inconsistencies.\nExtra information about the request:\n{}", "{}). Keeper will terminate to avoid inconsistencies.\nExtra information about the request:\n{}",
committing ? "committing" : "preprocessing", committing ? "committing" : "preprocessing",
request.getOpNum(), request.getOpNum(),
first.value, expected.value,
second.value, actual.value,
first.version, expected.version,
request.toString()); request.toString());
std::terminate(); std::terminate();
} }

View File

@ -174,7 +174,6 @@ uint64_t calculateDigest(std::string_view path, std::string_view data, const Kee
hash.update(data); hash.update(data);
hash.update(stat.czxid);
hash.update(stat.czxid); hash.update(stat.czxid);
hash.update(stat.mzxid); hash.update(stat.mzxid);
hash.update(stat.ctime); hash.update(stat.ctime);
@ -183,7 +182,6 @@ uint64_t calculateDigest(std::string_view path, std::string_view data, const Kee
hash.update(stat.cversion); hash.update(stat.cversion);
hash.update(stat.aversion); hash.update(stat.aversion);
hash.update(stat.ephemeralOwner); hash.update(stat.ephemeralOwner);
hash.update(data.length());
hash.update(stat.numChildren); hash.update(stat.numChildren);
hash.update(stat.pzxid); hash.update(stat.pzxid);
@ -2531,6 +2529,17 @@ void KeeperStorage::recalculateStats()
container.recalculateDataSize(); container.recalculateDataSize();
} }
bool KeeperStorage::checkDigest(const Digest & first, const Digest & second)
{
if (first.version != second.version)
return true;
if (first.version == DigestVersion::NO_DIGEST)
return true;
return first.value == second.value;
}
String KeeperStorage::generateDigest(const String & userdata) String KeeperStorage::generateDigest(const String & userdata)
{ {
std::vector<String> user_password; std::vector<String> user_password;

View File

@ -95,10 +95,11 @@ public:
{ {
NO_DIGEST = 0, NO_DIGEST = 0,
V1 = 1, V1 = 1,
V2 = 2 // added system nodes that modify the digest on startup so digest from V0 is invalid V2 = 2, // added system nodes that modify the digest on startup so digest from V0 is invalid
V3 = 3 // fixed bug with casting, removed duplicate czxid usage
}; };
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V2; static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V3;
struct ResponseForSession struct ResponseForSession
{ {
@ -113,16 +114,7 @@ public:
uint64_t value{0}; uint64_t value{0};
}; };
static bool checkDigest(const Digest & first, const Digest & second) static bool checkDigest(const Digest & first, const Digest & second);
{
if (first.version != second.version)
return true;
if (first.version == DigestVersion::NO_DIGEST)
return true;
return first.value == second.value;
}
static String generateDigest(const String & userdata); static String generateDigest(const String & userdata);

View File

@ -6,6 +6,7 @@
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <IO/BufferWithOwnMemory.h> #include <IO/BufferWithOwnMemory.h>
#include <IO/PeekableReadBuffer.h>
#include <IO/readFloatText.h> #include <IO/readFloatText.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <base/find_symbols.h> #include <base/find_symbols.h>

View File

@ -38,7 +38,6 @@
#include <IO/CompressionMethod.h> #include <IO/CompressionMethod.h>
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromMemory.h> #include <IO/ReadBufferFromMemory.h>
#include <IO/PeekableReadBuffer.h>
#include <IO/VarInt.h> #include <IO/VarInt.h>
#include <pcg_random.hpp> #include <pcg_random.hpp>
@ -51,6 +50,7 @@ namespace DB
template <typename Allocator> template <typename Allocator>
struct Memory; struct Memory;
class PeekableReadBuffer;
namespace ErrorCodes namespace ErrorCodes
{ {

View File

@ -7,6 +7,7 @@
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h> #include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Processors/Formats/ISchemaReader.h> #include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <IO/PeekableReadBuffer.h>
namespace DB namespace DB

View File

@ -2,6 +2,7 @@
#include <Formats/JSONUtils.h> #include <Formats/JSONUtils.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
#include <Formats/EscapingRuleUtils.h> #include <Formats/EscapingRuleUtils.h>
#include <IO/PeekableReadBuffer.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
namespace DB namespace DB

View File

@ -4,6 +4,7 @@
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h> #include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Processors/Formats/ISchemaReader.h> #include <Processors/Formats/ISchemaReader.h>
#include <IO/PeekableReadBuffer.h>
namespace DB namespace DB

View File

@ -7,6 +7,7 @@
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <IO/PeekableReadBuffer.h>
#include <Formats/EscapingRuleUtils.h> #include <Formats/EscapingRuleUtils.h>

View File

@ -25,6 +25,7 @@
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/Archives/createArchiveReader.h> #include <IO/Archives/createArchiveReader.h>
#include <IO/Archives/IArchiveReader.h> #include <IO/Archives/IArchiveReader.h>
#include <IO/PeekableReadBuffer.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h> #include <Formats/ReadSchemaUtils.h>

View File

@ -1,5 +1,6 @@
#include <algorithm> #include <algorithm>
#include <memory> #include <memory>
#include <stack>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>
#include <Core/TypeId.h> #include <Core/TypeId.h>

View File

@ -4,6 +4,7 @@
<server_id>1</server_id> <server_id>1</server_id>
<create_snapshot_on_exit>1</create_snapshot_on_exit> <create_snapshot_on_exit>1</create_snapshot_on_exit>
<digest_enabled>1</digest_enabled>
<coordination_settings> <coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms> <operation_timeout_ms>10000</operation_timeout_ms>

View File

@ -1 +1 @@
2024-01-01 Hello World 1

View File

@ -1,6 +1,6 @@
CREATE table if not exists table_with_dot_column (date Date, regular_column String, `other_column.2` String) ENGINE = MergeTree() ORDER BY date; CREATE TABLE IF NOT EXISTS table_with_dot_column (date Date, regular_column String, `other_column.2` String) ENGINE = MergeTree() ORDER BY date;
INSERT INTO table_with_dot_column select '2020-01-01', 'Hello', 'World'; INSERT INTO table_with_dot_column SELECT '2020-01-01', 'Hello', 'World';
INSERT INTO table_with_dot_column select '2024-01-01', 'Hello', 'World'; INSERT INTO table_with_dot_column SELECT toDate(now() + 48*3600), 'Hello', 'World';
CREATE ROW POLICY IF NOT EXISTS row_policy ON table_with_dot_column USING toDate(date) >= today() - 30 TO ALL; CREATE ROW POLICY IF NOT EXISTS row_policy ON table_with_dot_column USING toDate(date) >= today() - 30 TO ALL;
SELECT * FROM table_with_dot_column; SELECT count(*) FROM table_with_dot_column;
DROP TABLE table_with_dot_column; DROP TABLE table_with_dot_column;