-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
90649: sql: support indexes in regclass cast and pg_table_is_visible r=ajwerner a=rafiss fixes #88097 ### sql: remove deprecated function from DatabaseCatalog The ParseQualifiedTableName function was deprecated and unused. ### sql: support casts from index name to regclass Release note (sql change): Casts from index name to regclass are now supported. Previously, only table names could be cast to regclass. ### sql: add index on pg_namespace(oid) This will allow us to efficiently join to this table. ### sql: fix pg_table_is_visible to handle indexes This uses the internal executor now to do the introspection using pg_class where everything can be looked up all at once. There's an increase in number of round trips, but it's a constant factor. Release note (bug fix): Fixed the pg_table_is_visible builtin function so it correctly reports visibility of indexes based on the current search_path. 91965: allocator: add support for store pool liveness overrides r=AlexTalks a=AlexTalks While previously the allocator only evaluated using liveness obtained from gossip, this change introduces a new `OverrideStorePool` struct which can be used to override the liveness of a node for the purposes of evaluating allocator actions and targets. This `OverrideStorePool` is backed by an existing actual `StorePool`, which retains the majority of its logic. Depends on #91461. Part of #91570. Release note: None 93146: sql/sem: add check for interval for asof.DatumToHLC() r=rafiss a=ZhouXing19 Currently, if the given interval for `AS OF SYSTEM TIME interval` is a small postive duration, the query can incorrectly pass. It's because when we call `clock.Now()`, it has passed the threashold ('statement timestamp + duration'). Now we add a check for the duration value. It has to be negative, with the absolute value greater than a nanosecond. fixes #91021 link epic CRDB-17785 Release note (bug fix): add restriction for duration value for AS OF SYSTEM TIME statement. 93340: roachtest: add flaky test to activerecord ignore list r=ZhouXing19 a=andyyang890 Fixes #93189 Release note: None Co-authored-by: Rafi Shamim <[email protected]> Co-authored-by: Alex Sarkesian <[email protected]> Co-authored-by: Jane Xing <[email protected]> Co-authored-by: Andy Yang <[email protected]>
- Loading branch information
Showing
27 changed files
with
909 additions
and
171 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
172 changes: 172 additions & 0 deletions
172
pkg/kv/kvserver/allocator/storepool/override_store_pool.go
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,172 @@ | ||
// Copyright 2022 The Cockroach Authors. | ||
// | ||
// Use of this software is governed by the Business Source License | ||
// included in the file licenses/BSL.txt. | ||
// | ||
// As of the Change Date specified in that file, in accordance with | ||
// the Business Source License, use of this software will be governed | ||
// by the Apache License, Version 2.0, included in the file | ||
// licenses/APL.txt. | ||
|
||
package storepool | ||
|
||
import ( | ||
"context" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" | ||
"github.com/cockroachdb/cockroach/pkg/roachpb" | ||
"github.com/cockroachdb/cockroach/pkg/util" | ||
"github.com/cockroachdb/cockroach/pkg/util/hlc" | ||
) | ||
|
||
// OverrideStorePool is an implementation of AllocatorStorePool that allows | ||
// the ability to override a node's liveness status for the purposes of | ||
// evaluation for the allocator, otherwise delegating to an actual StorePool | ||
// for all its logic, including management and lookup of store descriptors. | ||
// | ||
// The OverrideStorePool is meant to provide a read-only overlay to an | ||
// StorePool, and as such, read-only methods are dispatched to the underlying | ||
// StorePool using the configured NodeLivenessFunc override. Methods that | ||
// mutate the state of the StorePool such as UpdateLocalStoreAfterRebalance | ||
// are instead no-ops. | ||
// | ||
// NB: Despite the fact that StorePool.DetailsMu is held in write mode in | ||
// some of the dispatched functions, these do not mutate the state of the | ||
// underlying StorePool. | ||
type OverrideStorePool struct { | ||
sp *StorePool | ||
|
||
overrideNodeLivenessFn NodeLivenessFunc | ||
} | ||
|
||
var _ AllocatorStorePool = &OverrideStorePool{} | ||
|
||
func NewOverrideStorePool(storePool *StorePool, nl NodeLivenessFunc) *OverrideStorePool { | ||
return &OverrideStorePool{ | ||
sp: storePool, | ||
overrideNodeLivenessFn: nl, | ||
} | ||
} | ||
|
||
func (o *OverrideStorePool) String() string { | ||
return o.sp.statusString(o.overrideNodeLivenessFn) | ||
} | ||
|
||
// IsStoreReadyForRoutineReplicaTransfer implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) IsStoreReadyForRoutineReplicaTransfer( | ||
ctx context.Context, targetStoreID roachpb.StoreID, | ||
) bool { | ||
return o.sp.isStoreReadyForRoutineReplicaTransferInternal(ctx, targetStoreID, o.overrideNodeLivenessFn) | ||
} | ||
|
||
// DecommissioningReplicas implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) DecommissioningReplicas( | ||
repls []roachpb.ReplicaDescriptor, | ||
) []roachpb.ReplicaDescriptor { | ||
return o.sp.decommissioningReplicasWithLiveness(repls, o.overrideNodeLivenessFn) | ||
} | ||
|
||
// GetStoreList implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) GetStoreList( | ||
filter StoreFilter, | ||
) (StoreList, int, ThrottledStoreReasons) { | ||
o.sp.DetailsMu.Lock() | ||
defer o.sp.DetailsMu.Unlock() | ||
|
||
var storeIDs roachpb.StoreIDSlice | ||
for storeID := range o.sp.DetailsMu.StoreDetails { | ||
storeIDs = append(storeIDs, storeID) | ||
} | ||
return o.sp.getStoreListFromIDsLocked(storeIDs, o.overrideNodeLivenessFn, filter) | ||
} | ||
|
||
// GetStoreListFromIDs implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) GetStoreListFromIDs( | ||
storeIDs roachpb.StoreIDSlice, filter StoreFilter, | ||
) (StoreList, int, ThrottledStoreReasons) { | ||
o.sp.DetailsMu.Lock() | ||
defer o.sp.DetailsMu.Unlock() | ||
return o.sp.getStoreListFromIDsLocked(storeIDs, o.overrideNodeLivenessFn, filter) | ||
} | ||
|
||
// LiveAndDeadReplicas implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) LiveAndDeadReplicas( | ||
repls []roachpb.ReplicaDescriptor, includeSuspectAndDrainingStores bool, | ||
) (liveReplicas, deadReplicas []roachpb.ReplicaDescriptor) { | ||
return o.sp.liveAndDeadReplicasWithLiveness(repls, o.overrideNodeLivenessFn, includeSuspectAndDrainingStores) | ||
} | ||
|
||
// ClusterNodeCount implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) ClusterNodeCount() int { | ||
return o.sp.ClusterNodeCount() | ||
} | ||
|
||
// IsDeterministic implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) IsDeterministic() bool { | ||
return o.sp.deterministic | ||
} | ||
|
||
// Clock implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) Clock() *hlc.Clock { | ||
return o.sp.clock | ||
} | ||
|
||
// GetLocalitiesByNode implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) GetLocalitiesByNode( | ||
replicas []roachpb.ReplicaDescriptor, | ||
) map[roachpb.NodeID]roachpb.Locality { | ||
return o.sp.GetLocalitiesByNode(replicas) | ||
} | ||
|
||
// GetLocalitiesByStore implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) GetLocalitiesByStore( | ||
replicas []roachpb.ReplicaDescriptor, | ||
) map[roachpb.StoreID]roachpb.Locality { | ||
return o.sp.GetLocalitiesByStore(replicas) | ||
} | ||
|
||
// GetStores implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) GetStores() map[roachpb.StoreID]roachpb.StoreDescriptor { | ||
return o.sp.GetStores() | ||
} | ||
|
||
// GetStoreDescriptor implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) GetStoreDescriptor( | ||
storeID roachpb.StoreID, | ||
) (roachpb.StoreDescriptor, bool) { | ||
return o.sp.GetStoreDescriptor(storeID) | ||
} | ||
|
||
// GossipNodeIDAddress implements the AllocatorStorePool interface. | ||
func (o *OverrideStorePool) GossipNodeIDAddress( | ||
nodeID roachpb.NodeID, | ||
) (*util.UnresolvedAddr, error) { | ||
return o.sp.GossipNodeIDAddress(nodeID) | ||
} | ||
|
||
// UpdateLocalStoreAfterRebalance implements the AllocatorStorePool interface. | ||
// This override method is a no-op, as | ||
// StorePool.UpdateLocalStoreAfterRebalance(..) is not a read-only method and | ||
// mutates the state of the held store details. | ||
func (o *OverrideStorePool) UpdateLocalStoreAfterRebalance( | ||
_ roachpb.StoreID, _ allocator.RangeUsageInfo, _ roachpb.ReplicaChangeType, | ||
) { | ||
} | ||
|
||
// UpdateLocalStoresAfterLeaseTransfer implements the AllocatorStorePool interface. | ||
// This override method is a no-op, as | ||
// StorePool.UpdateLocalStoresAfterLeaseTransfer(..) is not a read-only method and | ||
// mutates the state of the held store details. | ||
func (o *OverrideStorePool) UpdateLocalStoresAfterLeaseTransfer( | ||
_ roachpb.StoreID, _ roachpb.StoreID, _ float64, | ||
) { | ||
} | ||
|
||
// UpdateLocalStoreAfterRelocate implements the AllocatorStorePool interface. | ||
// This override method is a no-op, as | ||
// StorePool.UpdateLocalStoreAfterRelocate(..) is not a read-only method and | ||
// mutates the state of the held store details. | ||
func (o *OverrideStorePool) UpdateLocalStoreAfterRelocate( | ||
_, _ []roachpb.ReplicationTarget, _, _ []roachpb.ReplicaDescriptor, _ roachpb.StoreID, _ float64, | ||
) { | ||
} |
Oops, something went wrong.