Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
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
5 people committed Dec 9, 2022
5 parents 4b810d9 + 062845b + 8ae3466 + 14c89e1 + fe27492 commit 072ec8d
Show file tree
Hide file tree
Showing 27 changed files with 909 additions and 171 deletions.
10 changes: 5 additions & 5 deletions pkg/bench/rttanalysis/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
Expand Up @@ -53,9 +53,9 @@ exp,benchmark
11,GrantRole/grant_1_role
15,GrantRole/grant_2_roles
3,ORMQueries/activerecord_type_introspection_query
3,ORMQueries/django_table_introspection_1_table
3,ORMQueries/django_table_introspection_4_tables
3,ORMQueries/django_table_introspection_8_tables
6,ORMQueries/django_table_introspection_1_table
6,ORMQueries/django_table_introspection_4_tables
6,ORMQueries/django_table_introspection_8_tables
2,ORMQueries/has_column_privilege_using_attnum
2,ORMQueries/has_column_privilege_using_column_name
1,ORMQueries/has_schema_privilege_1
Expand All @@ -73,8 +73,8 @@ exp,benchmark
4,ORMQueries/information_schema._pg_index_position
3,ORMQueries/pg_attribute
3,ORMQueries/pg_class
9,ORMQueries/pg_is_other_temp_schema
17,ORMQueries/pg_is_other_temp_schema_multiple_times
7,ORMQueries/pg_is_other_temp_schema
7,ORMQueries/pg_is_other_temp_schema_multiple_times
4,ORMQueries/pg_my_temp_schema
4,ORMQueries/pg_my_temp_schema_multiple_times
4,ORMQueries/pg_namespace
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/activerecord_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ var activeRecordIgnoreList = blocklist{
"LengthValidationTest#test_validates_size_of_association_using_within": "flaky - sometimes complains that a relation does not exist",
"PostgresqlInfinityTest#test_where_clause_with_infinite_range_on_a_datetime_column": "flaky - sometimes complains that a relation does not exist",
"PostgresqlIntervalTest#test_interval_type": "flaky",
"PostgresqlTimestampFixtureTest#test_bc_timestamp": "flaky - sometimes datetime format does not match",
"PostgresqlTimestampWithAwareTypesTest#test_timestamp_with_zone_values_with_rails_time_zone_support_and_time_zone_set": "flaky - sometimes complains given Time instead of ActiveSupport::TimeWithZone",
"PostgresqlTimestampWithTimeZoneTest#test_timestamp_with_zone_values_with_rails_time_zone_support_and_timestamptz_and_time_zone_set": "flaky - sometimes complains given Time instead of ActiveSupport::TimeWithZone",
"RelationTest#test_finding_last_with_arel_order": "flaky - sometimes complains that a relation does not exist",
Expand Down
6 changes: 5 additions & 1 deletion pkg/kv/kvserver/allocator/storepool/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "storepool",
srcs = [
"override_store_pool.go",
"store_pool.go",
"test_helpers.go",
],
Expand Down Expand Up @@ -35,7 +36,10 @@ go_library(

go_test(
name = "storepool_test",
srcs = ["store_pool_test.go"],
srcs = [
"override_store_pool_test.go",
"store_pool_test.go",
],
args = ["-test.timeout=295s"],
embed = [":storepool"],
deps = [
Expand Down
172 changes: 172 additions & 0 deletions pkg/kv/kvserver/allocator/storepool/override_store_pool.go
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,
) {
}
Loading

0 comments on commit 072ec8d

Please sign in to comment.