Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
60758: sql: Added new tables and columns in pg_catalog r=rafiss a=RichardJCai

Taking over #59498 since @mnovelodou is out and we want to get this in before stability.

Previously, added tables and columns existed in postgres but not cockroach db
This was inadequate because this may cause errors at tools that expects these
columns to exists
To address this, this patch contains the result of diff tool to add what is
missing

Release note (sql change): Added missing tables and columns at pg_catalog
New columns in table pg_attribute:
        atthasmissing
New columns in table pg_class:
        relrowsecurity
        relforcerowsecurity
        relispartition
        relispopulated
        relreplident
        relrewrite
New columns in table pg_collation:
        collprovider
        collversion
        collisdeterministic
New columns in table pg_index:
        indnkeyatts
New columns in table pg_proc:
        prokind
        prosupport
New columns in table pg_aggregate:
        aggmfinalmodify
        aggfinalmodify
New Tables:
        pg_replication_origin_roname_index
        pg_config
        pg_largeobject_loid_pn_index
        pg_ts_config
        pg_operator_oid_index
        pg_conversion_oid_index
        pg_depend_depender_index
        pg_stat_user_functions
        pg_index_indrelid_index
        pg_group
        pg_db_role_setting_databaseid_rol_index
        pg_statio_user_tables
        pg_partitioned_table_partrelid_index
        pg_publication_oid_index
        pg_publication
        pg_publication_tables
        pg_stat_xact_user_tables
        pg_ts_config_map_index
        pg_replication_origin_roiident_index
        pg_stat_progress_vacuum
        pg_file_settings
        pg_statio_all_indexes
        pg_shseclabel_object_index
        pg_enum_oid_index
        pg_statistic_relid_att_inh_index
        pg_largeobject_metadata_oid_index
        pg_stat_archiver
        pg_statio_sys_sequences
        pg_collation_oid_index
        pg_subscription
        pg_namespace_oid_index
        pg_amop
        pg_stat_database_conflicts
        pg_tablespace_oid_index
        pg_class_oid_index
        pg_range_rngtypid_index
        pg_description_o_c_o_index
        pg_opclass_oid_index
        pg_sequence_seqrelid_index
        pg_trigger_oid_index
        pg_timezone_abbrevs
        pg_ts_parser_oid_index
        pg_transform
        pg_extension_oid_index
        pg_statio_user_sequences
        pg_shdepend_reference_index
        pg_foreign_data_wrapper_oid_index
        pg_ts_config_oid_index
        pg_ts_dict_oid_index
        pg_init_privs_o_c_o_index
        pg_user_mappings
        pg_default_acl_role_nsp_obj_index
        pg_stat_slru
        pg_constraint_contypid_index
        pg_stat_progress_create_index
        pg_transform_type_lang_index
        pg_authid_oid_index
        pg_shmem_allocations
        pg_statio_sys_tables
        pg_statio_all_sequences
        pg_policy_oid_index
        pg_shdepend_depender_index
        pg_attribute_relid_attnum_index
        pg_event_trigger_oid_index
        pg_amproc
        pg_cast_oid_index
        pg_constraint_conparentid_index
        pg_statio_sys_indexes
        pg_conversion_default_index
        pg_statistic_ext
        pg_shadow
        pg_trigger_tgconstraint_index
        pg_ts_template
        pg_cast_source_target_index
        pg_type_oid_index
        pg_amproc_fam_proc_index
        pg_aggregate_fnoid_index
        pg_stat_progress_basebackup
        pg_default_acl_oid_index
        pg_foreign_server_oid_index
        pg_statistic_ext_data_stxoid_index
        pg_transform_oid_index
        pg_language_oid_index
        pg_db_role_setting
        pg_amop_opr_fam_index
        pg_user_mapping_oid_index
        pg_hba_file_rules
        pg_am_oid_index
        pg_statio_all_tables
        pg_statio_user_indexes
        pg_stat_progress_analyze
        pg_replication_origin
        pg_depend_reference_index
        pg_stat_bgwriter
        pg_attrdef_adrelid_adnum_index
        pg_stat_sys_indexes
        pg_statistic_ext_oid_index
        pg_foreign_table_relid_index
        pg_user_mapping_user_server_index
        pg_seclabel_object_index
        pg_subscription_rel_srrelid_srsubid_index
        pg_publication_rel_prrelid_prpubid_index
        pg_inherits_parent_index
        pg_ts_dict
        pg_stat_user_tables
        pg_stat_progress_cluster
        pg_stat_xact_all_tables
        pg_stat_database
        pg_shdescription_o_c_index
        pg_publication_rel_oid_index
        pg_largeobject
        pg_publication_rel
        pg_rewrite_oid_index
        pg_stat_all_indexes
        pg_amop_fam_strat_index
        pg_proc_oid_index
        pg_database_oid_index
        pg_sequences
        pg_subscription_oid_index
        pg_enum_typid_sortorder_index
        pg_ts_parser
        pg_ts_config_map
        pg_attrdef_oid_index
        pg_timezone_names
        pg_ts_template_oid_index
        pg_statistic_ext_relid_index
        pg_index_indexrelid_index
        pg_amop_oid_index
        pg_amproc_oid_index
        pg_rules
        pg_opfamily
        pg_stat_xact_sys_tables
        pg_policies
        pg_constraint_oid_index
        pg_stat_sys_tables
        pg_stat_xact_user_functions
        pg_available_extension_versions
        pg_stat_all_tables
        pg_auth_members_role_member_index
        pg_auth_members_member_role_index
        pg_inherits_relid_seqno_index
        pg_opfamily_oid_index
        pg_class_tblspc_relfilenode_index
        pg_cursors
        pg_stat_gssapi
        pg_stat_ssl
        pg_stat_user_indexes

Fixes #58001

60765: kv: don't serve non-locking, read-write requests on followers r=nvanbenschoten a=nvanbenschoten

Discovered while investigating a test failure in #59566.

In 278a21b, we shifted from talking about read and write requests to
locking and non-locking requests when deciding whether a request could
be served on a follower. This prevented locking scans and gets from
being served on followers. However, it began letting lone HeartbeatTxn
and EndTxn requests past the old `!IsReadOnly()` check. Luckily, these
were still prevented from being served on followers because they are
only sent in read-write transactions, which were also prevented from
performing follower reads.

Yesterday, in 0ac8ab9, we lifted this second limitation, allowing
read-write transactions to perform follower reads for non-locking
batches. However, this no longer prevented HeartbeatTxn and EndTxn
requests from being routed and served on follower replicas. This
resulted in a pretty disastrous situation where in very rare cases, a
follower was proposing a write under a lease that it did not own.
Luckily, new assertions added in #59566 caught this.

This commit fixes this oversight be re-introducing "read-only" as a
condition for serving follower reads.

Release note: None

60781: cluster: use WaitConditionNextExit r=rickystewart a=tbg

There is a chance that this will address an issue that causes spurious
failures on CI, where it looks like we're janking the file system out
from under a running process. For context, see:

#58955

Release note: None


Co-authored-by: richardjcai <caioftherichard@gmail.com>
Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
  • Loading branch information
4 people committed Feb 19, 2021
4 parents 0e7cfdb + 612b83d + 9fc00c3 + 6543749 commit 98f1479
Show file tree
Hide file tree
Showing 20 changed files with 6,494 additions and 2,055 deletions.
4 changes: 2 additions & 2 deletions pkg/acceptance/cluster/dockercluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -241,7 +241,7 @@ func (l *DockerCluster) OneShot(
if err := l.oneshot.Start(ctx); err != nil {
return err
}
return l.oneshot.Wait(ctx, container.WaitConditionNotRunning)
return l.oneshot.Wait(ctx, container.WaitConditionNextExit)
}

// stopOnPanic is invoked as a deferred function in Start in order to attempt
Expand Down Expand Up @@ -377,7 +377,7 @@ func (l *DockerCluster) initCluster(ctx context.Context) {
// and it'll get in the way of future runs.
l.vols = c
maybePanic(c.Start(ctx))
maybePanic(c.Wait(ctx, container.WaitConditionNotRunning))
maybePanic(c.Wait(ctx, container.WaitConditionNextExit))
}

// cockroachEntryPoint returns the value to be used as
Expand Down
8 changes: 1 addition & 7 deletions pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,12 +92,6 @@ func evalFollowerReadOffset(clusterID uuid.UUID, st *cluster.Settings) (time.Dur
return getFollowerReadLag(st), nil
}

// batchCanBeEvaluatedOnFollower determines if a batch consists exclusively of
// requests that can be evaluated on a follower replica.
func batchCanBeEvaluatedOnFollower(ba roachpb.BatchRequest) bool {
return ba.Txn != nil && !ba.IsLocking() && ba.IsAllTransactional()
}

// closedTimestampLikelySufficient determines if a request with a given required
// frontier timestamp is likely to be below a follower's closed timestamp and
// serviceable as a follower read were the request to be sent to a follower
Expand Down Expand Up @@ -131,7 +125,7 @@ func canSendToFollower(
ba roachpb.BatchRequest,
) bool {
return checkFollowerReadsEnabled(clusterID, st) &&
batchCanBeEvaluatedOnFollower(ba) &&
kvserver.BatchCanBeEvaluatedOnFollower(ba) &&
closedTimestampLikelySufficient(st, clock, ctPolicy, ba.Txn.RequiredFrontier())
}

Expand Down
22 changes: 22 additions & 0 deletions pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,16 @@ func TestCanSendToFollower(t *testing.T) {
ba: batch(txn(stale), &roachpb.PutRequest{}),
exp: false,
},
{
name: "stale heartbeat txn",
ba: batch(txn(stale), &roachpb.HeartbeatTxnRequest{}),
exp: false,
},
{
name: "stale end txn",
ba: batch(txn(stale), &roachpb.EndTxnRequest{}),
exp: false,
},
{
name: "stale non-txn request",
ba: batch(txn(stale), &roachpb.QueryTxnRequest{}),
Expand Down Expand Up @@ -170,6 +180,18 @@ func TestCanSendToFollower(t *testing.T) {
ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS,
exp: false,
},
{
name: "stale heartbeat txn, global reads policy",
ba: batch(txn(stale), &roachpb.HeartbeatTxnRequest{}),
ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS,
exp: false,
},
{
name: "stale end txn, global reads policy",
ba: batch(txn(stale), &roachpb.EndTxnRequest{}),
ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS,
exp: false,
},
{
name: "stale non-txn request, global reads policy",
ba: batch(txn(stale), &roachpb.QueryTxnRequest{}),
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -1243,6 +1243,7 @@ func (r *Replica) checkExecutionCanProceed(
if !r.canServeFollowerReadRLocked(ctx, ba, err) {
return st, err
}
err = nil // ignoring error
}
}

Expand Down
21 changes: 20 additions & 1 deletion pkg/kv/kvserver/replica_follower_read.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,25 @@ var FollowerReadsEnabled = settings.RegisterBoolSetting(
true,
).WithPublic()

// BatchCanBeEvaluatedOnFollower determines if a batch consists exclusively of
// requests that can be evaluated on a follower replica, given a sufficiently
// advanced closed timestamp.
func BatchCanBeEvaluatedOnFollower(ba roachpb.BatchRequest) bool {
// Explanation of conditions:
// 1. the batch needs to be part of a transaction, because non-transactional
// batches often rely on the server setting their timestamp. If a follower
// with a lagging clock sets their timestamp then they might miss past
// writes served at higher timestamps.
// 2. each request in the batch needs to be "transactional", because those are
// the only ones that have clearly defined semantics when served under the
// closed timestamp.
// 3. the batch needs to be read-only, because a follower replica cannot
// propose writes to Raft.
// 4. the batch needs to be non-locking, because unreplicated locks are only
// held on the leaseholder.
return ba.Txn != nil && ba.IsAllTransactional() && ba.IsReadOnly() && !ba.IsLocking()
}

// canServeFollowerReadRLocked tests, when a range lease could not be acquired,
// whether the batch can be served as a follower read despite the error. Only
// non-locking, read-only requests can be served as follower reads. The batch
Expand All @@ -44,7 +63,7 @@ func (r *Replica) canServeFollowerReadRLocked(
var lErr *roachpb.NotLeaseHolderError
eligible := errors.As(err, &lErr) &&
lErr.LeaseHolder != nil && lErr.Lease.Type() == roachpb.LeaseEpoch &&
(ba.Txn != nil && !ba.IsLocking() && ba.IsAllTransactional()) && // followerreadsccl.batchCanBeEvaluatedOnFollower
BatchCanBeEvaluatedOnFollower(*ba) &&
FollowerReadsEnabled.Get(&r.store.cfg.Settings.SV)

if !eligible {
Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,10 @@ func (r *Replica) evalAndPropose(

// Attach information about the proposer to the command.
proposal.command.ProposerLeaseSequence = st.Lease.Sequence
// Perform a sanity check that the lease is owned by this replica.
if !st.Lease.OwnedBy(r.store.StoreID()) && !ba.IsLeaseRequest() {
log.Fatalf(ctx, "cannot propose %s on follower with remotely owned lease %s", ba, st.Lease)
}

// Once a command is written to the raft log, it must be loaded into memory
// and replayed on all replicas. If a command is too big, stop it here. If
Expand Down
158 changes: 152 additions & 6 deletions pkg/sql/catalog/catconstants/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,58 +106,204 @@ const (
InformationSchemaViewsTableID
InformationSchemaUserPrivilegesID
PgCatalogID
PgCatalogAggregateFnoidIndexTableID
PgCatalogAggregateTableID
PgCatalogAmOidIndexTableID
PgCatalogAmTableID
PgCatalogAmopFamStratIndexTableID
PgCatalogAmopOidIndexTableID
PgCatalogAmopOprFamIndexTableID
PgCatalogAmopTableID
PgCatalogAmprocFamProcIndexTableID
PgCatalogAmprocOidIndexTableID
PgCatalogAmprocTableID
PgCatalogAttrDefTableID
PgCatalogAttrdefAdrelidAdnumIndexTableID
PgCatalogAttrdefOidIndexTableID
PgCatalogAttributeRelidAttnumIndexTableID
PgCatalogAttributeTableID
PgCatalogAuthIDTableID
PgCatalogAuthMembersMemberRoleIndexTableID
PgCatalogAuthMembersRoleMemberIndexTableID
PgCatalogAuthMembersTableID
PgCatalogAuthidOidIndexTableID
PgCatalogAvailableExtensionVersionsTableID
PgCatalogAvailableExtensionsTableID
PgCatalogCastOidIndexTableID
PgCatalogCastSourceTargetIndexTableID
PgCatalogCastTableID
PgCatalogClassOidIndexTableID
PgCatalogClassTableID
PgCatalogClassTblspcRelfilenodeIndexTableID
PgCatalogCollationOidIndexTableID
PgCatalogCollationTableID
PgCatalogConfigTableID
PgCatalogConstraintConparentidIndexTableID
PgCatalogConstraintContypidIndexTableID
PgCatalogConstraintOidIndexTableID
PgCatalogConstraintTableID
PgCatalogConversionDefaultIndexTableID
PgCatalogConversionOidIndexTableID
PgCatalogConversionTableID
PgCatalogCursorsTableID
PgCatalogDatabaseOidIndexTableID
PgCatalogDatabaseTableID
PgCatalogDbRoleSettingDatabaseidRolIndexTableID
PgCatalogDbRoleSettingTableID
PgCatalogDefaultACLOidIndexTableID
PgCatalogDefaultACLRoleNspObjIndexTableID
PgCatalogDefaultACLTableID
PgCatalogDependDependerIndexTableID
PgCatalogDependReferenceIndexTableID
PgCatalogDependTableID
PgCatalogDescriptionOCOIndexTableID
PgCatalogDescriptionTableID
PgCatalogSharedDescriptionTableID
PgCatalogEnumOidIndexTableID
PgCatalogEnumTableID
PgCatalogEnumTypidSortorderIndexTableID
PgCatalogEventTriggerOidIndexTableID
PgCatalogEventTriggerTableID
PgCatalogExtensionOidIndexTableID
PgCatalogExtensionTableID
PgCatalogFileSettingsTableID
PgCatalogForeignDataWrapperOidIndexTableID
PgCatalogForeignDataWrapperTableID
PgCatalogForeignServerOidIndexTableID
PgCatalogForeignServerTableID
PgCatalogForeignTableRelidIndexTableID
PgCatalogForeignTableTableID
PgCatalogGroupTableID
PgCatalogHbaFileRulesTableID
PgCatalogIndexIndexrelidIndexTableID
PgCatalogIndexIndrelidIndexTableID
PgCatalogIndexTableID
PgCatalogIndexesTableID
PgCatalogInheritsParentIndexTableID
PgCatalogInheritsRelidSeqnoIndexTableID
PgCatalogInheritsTableID
PgCatalogInitPrivsOCOIndexTableID
PgCatalogLanguageOidIndexTableID
PgCatalogLanguageTableID
PgCatalogLargeobjectLoidPnIndexTableID
PgCatalogLargeobjectMetadataOidIndexTableID
PgCatalogLargeobjectTableID
PgCatalogLocksTableID
PgCatalogMatViewsTableID
PgCatalogNamespaceOidIndexTableID
PgCatalogNamespaceTableID
PgCatalogOpclassOidIndexTableID
PgCatalogOpclassTableID
PgCatalogOperatorOidIndexTableID
PgCatalogOperatorTableID
PgCatalogOpfamilyOidIndexTableID
PgCatalogOpfamilyTableID
PgCatalogPartitionedTablePartrelidIndexTableID
PgCatalogPoliciesTableID
PgCatalogPolicyOidIndexTableID
PgCatalogPreparedStatementsTableID
PgCatalogPreparedXactsTableID
PgCatalogProcOidIndexTableID
PgCatalogProcTableID
PgCatalogPublicationOidIndexTableID
PgCatalogPublicationRelOidIndexTableID
PgCatalogPublicationRelPrrelidPrpubidIndexTableID
PgCatalogPublicationRelTableID
PgCatalogPublicationTableID
PgCatalogPublicationTablesTableID
PgCatalogRangeRngtypidIndexTableID
PgCatalogRangeTableID
PgCatalogReplicationOriginRoiidentIndexTableID
PgCatalogReplicationOriginRonameIndexTableID
PgCatalogReplicationOriginTableID
PgCatalogRewriteOidIndexTableID
PgCatalogRewriteTableID
PgCatalogRolesTableID
PgCatalogRulesTableID
PgCatalogSecLabelsTableID
PgCatalogSeclabelObjectIndexTableID
PgCatalogSecurityLabelTableID
PgCatalogSequenceSeqrelidIndexTableID
PgCatalogSequencesTableID
PgCatalogSettingsTableID
PgCatalogShadowTableID
PgCatalogSharedDescriptionTableID
PgCatalogSharedSecurityLabelTableID
PgCatalogShdependDependerIndexTableID
PgCatalogShdependReferenceIndexTableID
PgCatalogShdependTableID
PgCatalogUserTableID
PgCatalogUserMappingTableID
PgCatalogShdescriptionOCIndexTableID
PgCatalogShmemAllocationsTableID
PgCatalogShseclabelObjectIndexTableID
PgCatalogStatActivityTableID
PgCatalogStatAllIndexesTableID
PgCatalogStatAllTablesTableID
PgCatalogStatArchiverTableID
PgCatalogStatBgwriterTableID
PgCatalogStatDatabaseConflictsTableID
PgCatalogStatDatabaseTableID
PgCatalogStatGssapiTableID
PgCatalogStatProgressAnalyzeTableID
PgCatalogStatProgressBasebackupTableID
PgCatalogStatProgressClusterTableID
PgCatalogStatProgressCreateIndexTableID
PgCatalogStatProgressVacuumTableID
PgCatalogStatSlruTableID
PgCatalogStatSslTableID
PgCatalogStatSysIndexesTableID
PgCatalogStatSysTablesTableID
PgCatalogStatUserFunctionsTableID
PgCatalogStatUserIndexesTableID
PgCatalogStatUserTablesTableID
PgCatalogStatXactAllTablesTableID
PgCatalogStatXactSysTablesTableID
PgCatalogStatXactUserFunctionsTableID
PgCatalogStatXactUserTablesTableID
PgCatalogStatioAllIndexesTableID
PgCatalogStatioAllSequencesTableID
PgCatalogStatioAllTablesTableID
PgCatalogStatioSysIndexesTableID
PgCatalogStatioSysSequencesTableID
PgCatalogStatioSysTablesTableID
PgCatalogStatioUserIndexesTableID
PgCatalogStatioUserSequencesTableID
PgCatalogStatioUserTablesTableID
PgCatalogStatisticExtDataStxoidIndexTableID
PgCatalogStatisticExtOidIndexTableID
PgCatalogStatisticExtRelidIndexTableID
PgCatalogStatisticExtTableID
PgCatalogStatisticRelidAttInhIndexTableID
PgCatalogSubscriptionOidIndexTableID
PgCatalogSubscriptionRelSrrelidSrsubidIndexTableID
PgCatalogSubscriptionTableID
PgCatalogTablesTableID
PgCatalogTablespaceOidIndexTableID
PgCatalogTablespaceTableID
PgCatalogTimezoneAbbrevsTableID
PgCatalogTimezoneNamesTableID
PgCatalogTransformOidIndexTableID
PgCatalogTransformTableID
PgCatalogTransformTypeLangIndexTableID
PgCatalogTriggerOidIndexTableID
PgCatalogTriggerTableID
PgCatalogTriggerTgconstraintIndexTableID
PgCatalogTsConfigMapIndexTableID
PgCatalogTsConfigMapTableID
PgCatalogTsConfigOidIndexTableID
PgCatalogTsConfigTableID
PgCatalogTsDictOidIndexTableID
PgCatalogTsDictTableID
PgCatalogTsParserOidIndexTableID
PgCatalogTsParserTableID
PgCatalogTsTemplateOidIndexTableID
PgCatalogTsTemplateTableID
PgCatalogTypeOidIndexTableID
PgCatalogTypeTableID
PgCatalogUserMappingOidIndexTableID
PgCatalogUserMappingTableID
PgCatalogUserMappingUserServerIndexTableID
PgCatalogUserMappingsTableID
PgCatalogUserTableID
PgCatalogViewsTableID
PgCatalogStatActivityTableID
PgCatalogSecurityLabelTableID
PgCatalogSharedSecurityLabelTableID
PgExtensionSchemaID
PgExtensionGeographyColumnsTableID
PgExtensionGeometryColumnsTableID
Expand Down
Loading

0 comments on commit 98f1479

Please sign in to comment.