Skip to content

Commit

Permalink
[#23669] YSQL: Fix DDL atomicity assertion failure
Browse files Browse the repository at this point in the history
Summary:
From the ticket, the following error is noticed:

```
E0824 04:05:51.544625 262184 pg_client_session.cc:1109] Sending ReportYsqlDdlTxnStatus call failed: Illegal state (yb/master/ysql_ddl_handler.cc:221): Mismatch in txn_state for transaction 6c3e8bad-d77d-4fc3-b05f-cde9ba7cc985: kAborted vs kCommitted
```

I found two logs that showed contradicting message:

```
./yb-master.ip-172-151-28-119.us-west-2.compute.internal.yugabyte.log.INFO.20240824-035800.38216.gz:I0824 04:05:51.536473 291955 ysql_ddl_handler.cc:188] YsqlDdlTxnCompleteCallback for transaction 6c3e8bad-d77d-4fc3-b05f-cde9ba7cc985 is_committed: true
```

vs

```
./yb-master.ip-172-151-28-119.us-west-2.compute.internal.yugabyte.log.INFO.20240824-035800.38216.gz:I0824 04:05:51.544212 185834 ysql_ddl_handler.cc:188] YsqlDdlTxnCompleteCallback for transaction 6c3e8bad-d77d-4fc3-b05f-cde9ba7cc985 is_committed: false
```

Both are related to the same txn `6c3e8bad-d77d-4fc3-b05f-cde9ba7cc985`, and their
timestamps are close: 04:05:51.536473 vs 04:05:51.54421.

However the first log says the txn has committed (is_committed: true), and the
second log says the txn has aborted (is_committed: false)
So two different code paths deduced different commit/abort status for the same txn.

DDL atomicity has two ways for docdb to follow and become eventually
consistent with the PG transaction status (commit or abort):
(1) PG sends a status report to master tell it whether the txn has committed or
not. This is the fastest since PG drives the DDL txn and it knows firsthand.
(2) Docdb has a background task, periodically polling the transaction status
tablet to find out the status of the give txn. If the transaction status tablet
response with the txn status as terminated (not pending), then the same
background task will begin to read the current PG metadata from sys catalog
table, based upon the nature of the DDL, performs a schema comparison to find
out whether the current PG catalog schema indicated a commit or abort:
(2.1) If the current PG catalog schema matches the current DocDB schema (the new
table schema if the DDL successfully commits), it means the DDL txn has
committed.
(2.2) If the current PG catalog schema matches the previous DocDB schema (the
old table schema prior to the DDL starts), it means the DDL txn has aborted.

The bug is that there are some DDL statements that do not change the table's
schema. For example, `alter table foo alter column id set not null` only bumps
up the table foo's schema version without changing the schema at all. As a
result, the current schema is identical to the previous schema. In other words,
both (2.1) and (2.2) will compare as equal and because we check (2.1) first, we
always conclude that the DDL txn has committed. Had we reversed the order by
checking (2.2) first, then we would always conclude that the DDL txn has
aborted.

I made a change to make the type of is_committed from bool to
std::optional<bool>. When is_committed.has_value() is true, it means either
committed or aborted so we continue with the current logic. When is_committed is
nullopt, it means the DDL does not change the table's schema at all.

A new state TxnState::kNoChange is added in addition to the current kUnknown,
kCommitted, kAborted. It is not possible to tell whether the DDL txn has
committed or aborted by doing schema comparison if the DDL does not change the
table schema. However it is fine because we do not need to know whether the DDL
txn is committed or aborted, we only need to know the DDL txn has terminated.
The end result is that the table's schema does not change. We just need to clear
the DDL atomicity related state from the table's DocDB SysTablesEntryPB
metadata and the table's schema will be correct.
Jira: DB-12580

Test Plan: jenkins

Reviewers: hsunder, fizaa

Reviewed By: fizaa

Subscribers: ybase, yql

Differential Revision: https://phorge.dev.yugabyte.com/D37755
  • Loading branch information
myang2021 committed Sep 10, 2024
1 parent 1099625 commit a6981f2
Show file tree
Hide file tree
Showing 8 changed files with 99 additions and 39 deletions.
8 changes: 8 additions & 0 deletions src/yb/common/schema.h
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,10 @@ class ColumnSchema {
return a.order_ == b.order_;
}

static bool CompMarkedForDeletion(const ColumnSchema &a, const ColumnSchema &b) {
return a.marked_for_deletion_ == b.marked_for_deletion_;
}

// Combined comparators.
static bool CompareType(const ColumnSchema &a, const ColumnSchema &b) {
return CompNullable(a, b) && CompKind(a, b) && CompTypeInfo(a, b);
Expand All @@ -142,6 +146,10 @@ class ColumnSchema {
return CompareType(a, b) && CompName(a, b);
}

static bool CompareDdlAtomicity(const ColumnSchema &a, const ColumnSchema &b) {
return CompareByDefault(a, b) && CompMarkedForDeletion(a, b);
}

// name: column name
// type: column type (e.g. UINT8, INT32, STRING, MAP<INT32, STRING> ...)
// is_nullable: true if a row value can be null
Expand Down
14 changes: 11 additions & 3 deletions src/yb/master/catalog_entity_info.cc
Original file line number Diff line number Diff line change
Expand Up @@ -519,16 +519,24 @@ TableType TableInfo::GetTableType() const {
return LockForRead()->pb.table_type();
}

bool TableInfo::IsBeingDroppedDueToDdlTxn(const std::string& pb_txn_id, bool txn_success) const {
bool TableInfo::IsBeingDroppedDueToDdlTxn(
const std::string& pb_txn_id, std::optional<bool> txn_success) const {
auto l = LockForRead();
if (l->pb_transaction_id() != pb_txn_id) {
return false;
}
if (!txn_success.has_value()) {
// This means the DDL txn only increments the schema version and does not change
// the DocDB schema at all. It cannot be one of the following 2 cases.
DCHECK(!l->is_being_created_by_ysql_ddl_txn());
DCHECK(!l->is_being_deleted_by_ysql_ddl_txn());
return false;
}
// The table can be dropped in 2 cases due to a DDL:
// 1. This table was created by a transaction that subsequently aborted.
// 2. This is a successful transaction that DROPs the table.
return (l->is_being_created_by_ysql_ddl_txn() && !txn_success) ||
(l->is_being_deleted_by_ysql_ddl_txn() && txn_success);
return (l->is_being_created_by_ysql_ddl_txn() && !*txn_success) ||
(l->is_being_deleted_by_ysql_ddl_txn() && *txn_success);
}

Status TableInfo::AddTablet(const TabletInfoPtr& tablet) {
Expand Down
3 changes: 2 additions & 1 deletion src/yb/master/catalog_entity_info.h
Original file line number Diff line number Diff line change
Expand Up @@ -614,7 +614,8 @@ class TableInfo : public RefCountedThreadSafe<TableInfo>,
return GetTableType() == REDIS_TABLE_TYPE;
}

bool IsBeingDroppedDueToDdlTxn(const std::string& txn_id_pb, bool txn_success) const;
bool IsBeingDroppedDueToDdlTxn(
const std::string& txn_id_pb, std::optional<bool> txn_success) const;

// Add a tablet to this table.
Status AddTablet(const TabletInfoPtr& tablet);
Expand Down
9 changes: 6 additions & 3 deletions src/yb/master/catalog_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -4385,7 +4385,7 @@ Status CatalogManager::CreateTableIfNotFound(
void CatalogManager::ScheduleVerifyTablePgLayer(TransactionMetadata txn,
const TableInfoPtr& table,
const LeaderEpoch& epoch) {
auto when_done = [this, table, epoch](Result<bool> exists) {
auto when_done = [this, table, epoch](Result<std::optional<bool>> exists) {
WARN_NOT_OK(VerifyTablePgLayer(table, exists, epoch), "Failed to verify table");
};
TableSchemaVerificationTask::CreateAndStartTask(
Expand All @@ -4394,10 +4394,13 @@ void CatalogManager::ScheduleVerifyTablePgLayer(TransactionMetadata txn,
}

Status CatalogManager::VerifyTablePgLayer(
scoped_refptr<TableInfo> table, Result<bool> exists, const LeaderEpoch& epoch) {
scoped_refptr<TableInfo> table, Result<std::optional<bool>> exists, const LeaderEpoch& epoch) {
if (!exists.ok()) {
return exists.status();
}
auto opt_exists = exists.get();
SCHECK(opt_exists.has_value(), IllegalState,
Substitute("Unexpected opt_exists for $0", table->ToString()));
// Upon Transaction completion, check pg system table using OID to ensure SUCCESS.
auto l = table->LockForWrite();
auto* mutable_table_info = table->mutable_metadata()->mutable_dirty();
Expand All @@ -4409,7 +4412,7 @@ Status CatalogManager::VerifyTablePgLayer(
"Unexpected table state ($0), abandoning transaction GC work for $1",
SysTablesEntryPB_State_Name(metadata.state()), table->ToString()));

if (exists.get()) {
if (*opt_exists) {
// Remove the transaction from the entry since we're done processing it.
metadata.clear_transaction();
RETURN_NOT_OK(sys_catalog_->Upsert(epoch, table));
Expand Down
17 changes: 12 additions & 5 deletions src/yb/master/catalog_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,13 @@ YB_DEFINE_ENUM(YsqlDdlVerificationState,
(kDdlPostProcessing)
(kDdlPostProcessingFailed));

YB_DEFINE_ENUM(TxnState, (kUnknown) (kCommitted) (kAborted));
// kNoChange is used when a PG DDL statement only increments the table's schema version
// without any real DocDB table schema change (e.g., alter table t alter column c set not null).
// In this case we cannot decide whether the PG DDL txn has committed or aborted by doing
// schema comparison of DocDB current schema or previous schema against the PG catalog schema.
// That's fine because whether the PG DDL txn has committed or aborted makes no difference for
// this table's DocDB schema.
YB_DEFINE_ENUM(TxnState, (kUnknown) (kCommitted) (kAborted) (kNoChange));

struct YsqlTableDdlTxnState;

Expand Down Expand Up @@ -352,7 +358,7 @@ class CatalogManager : public tserver::TabletPeerLookupIf,
const TableInfoPtr& table, const LeaderEpoch& epoch);

// Called when transaction associated with table create finishes. Verifies postgres layer present.
Status VerifyTablePgLayer(scoped_refptr<TableInfo> table, Result<bool> exists,
Status VerifyTablePgLayer(scoped_refptr<TableInfo> table, Result<std::optional<bool>> exists,
const LeaderEpoch& epoch);

// Truncate the specified table.
Expand Down Expand Up @@ -474,16 +480,17 @@ class CatalogManager : public tserver::TabletPeerLookupIf,

Status YsqlTableSchemaChecker(TableInfoPtr table,
const std::string& pb_txn_id,
Result<bool> is_committed,
Result<std::optional<bool>> is_committed,
const LeaderEpoch& epoch);

Status YsqlDdlTxnCompleteCallback(const std::string& pb_txn_id,
bool is_committed,
std::optional<bool> is_committed,
const LeaderEpoch& epoch,
const std::string& debug_caller_info);

Status YsqlDdlTxnCompleteCallbackInternal(
TableInfo* table, const TransactionId& txn_id, bool success, const LeaderEpoch& epoch);
TableInfo* table, const TransactionId& txn_id, std::optional<bool> success,
const LeaderEpoch& epoch);

Status HandleSuccessfulYsqlDdlTxn(const YsqlTableDdlTxnState txn_data);

Expand Down
47 changes: 33 additions & 14 deletions src/yb/master/ysql_ddl_handler.cc
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ Status CatalogManager::ScheduleVerifyTransaction(
<< " id: " << table->id() << " schema version: " << l->pb.version()
<< " for transaction " << txn;
const string txn_id_pb = l->pb_transaction_id();
auto when_done = [this, table, txn_id_pb, epoch](Result<bool> is_committed) {
auto when_done = [this, table, txn_id_pb, epoch](Result<std::optional<bool>> is_committed) {
WARN_NOT_OK(YsqlTableSchemaChecker(table, txn_id_pb, is_committed, epoch),
"YsqlTableSchemaChecker failed");
};
Expand All @@ -152,7 +152,7 @@ Status CatalogManager::ScheduleVerifyTransaction(

Status CatalogManager::YsqlTableSchemaChecker(TableInfoPtr table,
const string& pb_txn_id,
Result<bool> is_committed,
Result<std::optional<bool>> is_committed,
const LeaderEpoch& epoch) {
if (!is_committed.ok()) {
auto txn = VERIFY_RESULT(FullyDecodeTransactionId(pb_txn_id));
Expand All @@ -178,7 +178,7 @@ Status CatalogManager::YsqlTableSchemaChecker(TableInfoPtr table,
}

Status CatalogManager::YsqlDdlTxnCompleteCallback(const string& pb_txn_id,
bool is_committed,
std::optional<bool> is_committed,
const LeaderEpoch& epoch,
const std::string& debug_caller_info) {
SCHECK(!pb_txn_id.empty(), IllegalState,
Expand All @@ -188,7 +188,8 @@ Status CatalogManager::YsqlDdlTxnCompleteCallback(const string& pb_txn_id,

auto txn = VERIFY_RESULT(FullyDecodeTransactionId(pb_txn_id));
LOG(INFO) << "YsqlDdlTxnCompleteCallback for transaction "
<< txn << " is_committed: " << (is_committed ? "true" : "false")
<< txn << " is_committed: "
<< (is_committed.has_value() ? (*is_committed ? "true" : "false") : "nullopt")
<< ", debug_caller_info " << debug_caller_info;

vector<TableInfoPtr> tables;
Expand All @@ -202,7 +203,8 @@ Status CatalogManager::YsqlDdlTxnCompleteCallback(const string& pb_txn_id,
}

auto state = verifier_state->state;
auto txn_state = is_committed ? TxnState::kCommitted : TxnState::kAborted;
auto txn_state = is_committed.has_value() ?
(*is_committed ? TxnState::kCommitted : TxnState::kAborted) : TxnState::kNoChange;
if (state == YsqlDdlVerificationState::kDdlPostProcessing) {
// We used to return Status::OK() here on the grounds that the txn is
// already being verified and we assumed verifier_state->tables represent
Expand All @@ -220,10 +222,18 @@ Status CatalogManager::YsqlDdlTxnCompleteCallback(const string& pb_txn_id,
// to process t3. It is fine to reprocess t1 and t2, that will result in a
// no-op, except for delete table operation for which we'll detect and avoid
// reprocessing.
SCHECK_EQ(txn_state, verifier_state->txn_state, IllegalState,
Format("Mismatch in txn_state for transaction $0", txn));
// Some alter table DDL statements only increment table schema version and does not make
// any table schema change, this is represented by TxnState::kNoChange. In this case
// it does not matter whether the PG DDL transaction is committed or aborted.
if (txn_state != verifier_state->txn_state &&
txn_state != TxnState::kNoChange && verifier_state->txn_state != TxnState::kNoChange) {
return STATUS_FORMAT(IllegalState, "Mismatch in txn_state for transaction $0", txn);
}
} else {
verifier_state->txn_state = txn_state;
if (verifier_state->txn_state != TxnState::kCommitted &&
verifier_state->txn_state != TxnState::kAborted) {
verifier_state->txn_state = txn_state;
}
verifier_state->state = YsqlDdlVerificationState::kDdlPostProcessing;
}
tables = verifier_state->tables;
Expand Down Expand Up @@ -301,7 +311,8 @@ struct YsqlTableDdlTxnState {
};

Status CatalogManager::YsqlDdlTxnCompleteCallbackInternal(
TableInfo* table, const TransactionId& txn_id, bool success, const LeaderEpoch& epoch) {
TableInfo* table, const TransactionId& txn_id,
std::optional<bool> success, const LeaderEpoch& epoch) {

TEST_PAUSE_IF_FLAG(TEST_pause_ddl_rollback);

Expand All @@ -315,7 +326,8 @@ Status CatalogManager::YsqlDdlTxnCompleteCallbackInternal(
return Status::OK();
}
LOG_WITH_FUNC(INFO) << id << " for transaction " << txn_id
<< ": Success: " << (success ? "true" : "false")
<< ": Success: "
<< (success.has_value() ? (*success ? "true" : "false") : "nullopt")
<< " ysql_ddl_txn_verifier_state: "
<< l->ysql_ddl_txn_verifier_state().DebugString();

Expand All @@ -332,10 +344,17 @@ Status CatalogManager::YsqlDdlTxnCompleteCallbackInternal(
.ddl_txn_id = txn_id
};

if (success) {
RETURN_NOT_OK(HandleSuccessfulYsqlDdlTxn(txn_data));
if (success.has_value()) {
if (*success) {
RETURN_NOT_OK(HandleSuccessfulYsqlDdlTxn(txn_data));
} else {
RETURN_NOT_OK(HandleAbortedYsqlDdlTxn(txn_data));
}
} else {
RETURN_NOT_OK(HandleAbortedYsqlDdlTxn(txn_data));
// If success is nullopt, it represents a PG DDL statement that only increments the schema
// version of this table without any table schema change. There is nothing to do but to
// cleanup.
RETURN_NOT_OK(ClearYsqlDdlTxnState(txn_data));
}
return Status::OK();
}
Expand Down Expand Up @@ -664,7 +683,7 @@ Status CatalogManager::TriggerDdlVerificationIfNeeded(
<< " for transaction " << txn;

const string txn_id_pb = l->pb_transaction_id();
auto when_done = [this, table, txn_id_pb, epoch](Result<bool> is_committed) {
auto when_done = [this, table, txn_id_pb, epoch](Result<std::optional<bool>> is_committed) {
WARN_NOT_OK(YsqlTableSchemaChecker(table, txn_id_pb, is_committed, epoch),
"YsqlTableSchemaChecker failed");
};
Expand Down
32 changes: 23 additions & 9 deletions src/yb/master/ysql_ddl_verification_task.cc
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ Status PgEntryExistsWithReadTime(
Status PgSchemaCheckerWithReadTime(SysCatalogTable* sys_catalog,
const scoped_refptr<TableInfo>& table,
const ReadHybridTime& read_time,
bool* result,
std::optional<bool>* result,
HybridTime* read_restart_ht);

bool MatchPgDocDBSchemaColumns(const scoped_refptr<TableInfo>& table,
Expand Down Expand Up @@ -205,8 +205,9 @@ Status PgEntryExistsWithReadTime(
return Status::OK();
}

Result<bool> PgSchemaChecker(SysCatalogTable& sys_catalog, const scoped_refptr<TableInfo>& table) {
bool result = false;
Result<std::optional<bool>> PgSchemaChecker(
SysCatalogTable& sys_catalog, const scoped_refptr<TableInfo>& table) {
std::optional<bool> result{std::nullopt};
RETURN_NOT_OK(sys_catalog.ReadWithRestarts(
std::bind(&PgSchemaCheckerWithReadTime, &sys_catalog, table, std::placeholders::_1, &result,
std::placeholders::_2)));
Expand All @@ -216,7 +217,7 @@ Result<bool> PgSchemaChecker(SysCatalogTable& sys_catalog, const scoped_refptr<T
Status PgSchemaCheckerWithReadTime(SysCatalogTable* sys_catalog,
const scoped_refptr<TableInfo>& table,
const ReadHybridTime& read_time,
bool* result,
std::optional<bool>* result,
HybridTime* read_restart_ht) {
PgOid oid = kPgInvalidOid;
string pg_catalog_table_id, name_col;
Expand Down Expand Up @@ -339,6 +340,7 @@ Status PgSchemaCheckerWithReadTime(SysCatalogTable* sys_catalog,

// Table was being altered. Check whether its current DocDB schema matches
// that of PG catalog.
VLOG(3) << "Comparing with the PG schema for alter table";
CHECK(l->ysql_ddl_txn_verifier_state().contains_alter_table_op());
const auto& relname_col = row.GetValue(relname_col_id);
const string& table_name = relname_col->string_value();
Expand All @@ -361,17 +363,28 @@ Status PgSchemaCheckerWithReadTime(SysCatalogTable* sys_catalog,

Schema schema;
RETURN_NOT_OK(table->GetSchema(&schema));
Schema previous_schema;
RETURN_NOT_OK(SchemaFromPB(l->ysql_ddl_txn_verifier_state().previous_schema(), &previous_schema));
// CompareDdlAtomicity takes marked_for_deletion() into comparison. If a column is marked for
// deletion in the current schema and not in the previous schema, then CompareByDefault would
// return true which isn't right for correct handling.
if (schema.Equals(previous_schema, ColumnSchema::CompareDdlAtomicity)) {
VLOG(3) << "The current DocDB schema is the same as the previous DocDB schema";
*result = std::nullopt;
return Status::OK();
}

if (MatchPgDocDBSchemaColumns(table, schema, pg_cols)) {
// The PG catalog schema matches the current DocDB schema. The transaction was a success.
VLOG(3) << "PG schema matches the current DocDB schema";
*result = true;
return Status::OK();
}

Schema previous_schema;
RETURN_NOT_OK(SchemaFromPB(l->ysql_ddl_txn_verifier_state().previous_schema(), &previous_schema));
if (MatchPgDocDBSchemaColumns(table, previous_schema, pg_cols)) {
// The PG catalog schema matches the DocDB schema of the table prior to this transaction. The
// transaction must have aborted.
VLOG(3) << "PG schema matches the previous DocDB schema";
*result = false;
return Status::OK();
}
Expand Down Expand Up @@ -719,7 +732,8 @@ void NamespaceVerificationTask::PerformAbort() {

TableSchemaVerificationTask::TableSchemaVerificationTask(
CatalogManager& catalog_manager, scoped_refptr<TableInfo> table,
const TransactionMetadata& transaction, std::function<void(Result<bool>)> complete_callback,
const TransactionMetadata& transaction,
std::function<void(Result<std::optional<bool>>)> complete_callback,
SysCatalogTable* sys_catalog, std::shared_future<client::YBClient*> client_future,
rpc::Messenger& messenger, const LeaderEpoch& epoch, bool ddl_atomicity_enabled)
: MultiStepTableTaskBase(
Expand All @@ -741,7 +755,7 @@ void TableSchemaVerificationTask::CreateAndStartTask(
CatalogManager& catalog_manager,
scoped_refptr<TableInfo> table,
const TransactionMetadata& transaction,
std::function<void(Result<bool>)> complete_callback,
std::function<void(Result<std::optional<bool>>)> complete_callback,
SysCatalogTable* sys_catalog,
std::shared_future<client::YBClient*> client_future,
rpc::Messenger& messenger,
Expand Down Expand Up @@ -794,7 +808,7 @@ void TableSchemaVerificationTask::FinishPollTransaction(Status status) {
}, "Compare Schema");
}

Status TableSchemaVerificationTask::FinishTask(Result<bool> is_committed) {
Status TableSchemaVerificationTask::FinishTask(Result<std::optional<bool>> is_committed) {
RETURN_NOT_OK(is_committed);

is_committed_ = *is_committed;
Expand Down
8 changes: 4 additions & 4 deletions src/yb/master/ysql_ddl_verification_task.h
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ class TableSchemaVerificationTask : public MultiStepTableTaskBase,
CatalogManager& catalog_manager,
scoped_refptr<TableInfo> table,
const TransactionMetadata& transaction,
std::function<void(Result<bool>)> complete_callback,
std::function<void(Result<std::optional<bool>>)> complete_callback,
SysCatalogTable* sys_catalog,
std::shared_future<client::YBClient*> client_future,
rpc::Messenger& messenger,
Expand All @@ -172,7 +172,7 @@ class TableSchemaVerificationTask : public MultiStepTableTaskBase,
CatalogManager& catalog_manager,
scoped_refptr<TableInfo> table,
const TransactionMetadata& transaction,
std::function<void(Result<bool>)> complete_callback,
std::function<void(Result<std::optional<bool>>)> complete_callback,
SysCatalogTable* sys_catalog,
std::shared_future<client::YBClient*> client_future,
rpc::Messenger& messenger,
Expand All @@ -185,14 +185,14 @@ class TableSchemaVerificationTask : public MultiStepTableTaskBase,
Status ValidateRunnable() override;
Status CheckTableExists(Status s);
Status CompareSchema(Status s);
Status FinishTask(Result<bool> is_committed);
Status FinishTask(Result<std::optional<bool>> is_committed);
void FinishPollTransaction(Status s) override;
void TaskCompleted(const Status& status) override;
void PerformAbort() override;

SysCatalogTable& sys_catalog_;
bool ddl_atomicity_enabled_;
bool is_committed_ = false;
std::optional<bool> is_committed_{std::nullopt};
};

} // namespace master
Expand Down

0 comments on commit a6981f2

Please sign in to comment.