diff --git a/c-deps/libroach/protos/roachpb/data.pb.cc b/c-deps/libroach/protos/roachpb/data.pb.cc index 0bdcb76663c9..b3fb65581e29 100644 --- a/c-deps/libroach/protos/roachpb/data.pb.cc +++ b/c-deps/libroach/protos/roachpb/data.pb.cc @@ -20,11 +20,11 @@ namespace protobuf_roachpb_2fdata_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_SequencedWrite; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_Span; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_ChangeReplicasTrigger; -extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_MergeTrigger; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_ModifiedSpanTrigger; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_ObservedTimestamp; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_SplitTrigger; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_Value; +extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<2> scc_info_MergeTrigger; extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fdata_2eproto ::google::protobuf::internal::SCCInfo<4> scc_info_Transaction; } // namespace protobuf_roachpb_2fdata_2eproto namespace protobuf_roachpb_2fmetadata_2eproto { @@ -34,6 +34,9 @@ extern PROTOBUF_INTERNAL_EXPORT_protobuf_roachpb_2fmetadata_2eproto ::google::pr namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto ::google::protobuf::internal::SCCInfo<1> scc_info_TxnMeta; } // namespace protobuf_storage_2fengine_2fenginepb_2fmvcc3_2eproto +namespace protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto { +extern PROTOBUF_INTERNAL_EXPORT_protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_MVCCStats; +} // namespace protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto namespace protobuf_util_2fhlc_2ftimestamp_2eproto { extern PROTOBUF_INTERNAL_EXPORT_protobuf_util_2fhlc_2ftimestamp_2eproto ::google::protobuf::internal::SCCInfo<0> scc_info_Timestamp; } // namespace protobuf_util_2fhlc_2ftimestamp_2eproto @@ -206,9 +209,10 @@ static void InitDefaultsMergeTrigger() { ::cockroach::roachpb::MergeTrigger::InitAsDefaultInstance(); } -::google::protobuf::internal::SCCInfo<1> scc_info_MergeTrigger = - {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsMergeTrigger}, { - &protobuf_roachpb_2fmetadata_2eproto::scc_info_RangeDescriptor.base,}}; +::google::protobuf::internal::SCCInfo<2> scc_info_MergeTrigger = + {{ATOMIC_VAR_INIT(::google::protobuf::internal::SCCInfoBase::kUninitialized), 2, InitDefaultsMergeTrigger}, { + &protobuf_roachpb_2fmetadata_2eproto::scc_info_RangeDescriptor.base, + &protobuf_storage_2fengine_2fenginepb_2fmvcc_2eproto::scc_info_MVCCStats.base,}}; static void InitDefaultsChangeReplicasTrigger() { GOOGLE_PROTOBUF_VERIFY_VERSION; @@ -1601,6 +1605,8 @@ void MergeTrigger::InitAsDefaultInstance() { ::cockroach::roachpb::RangeDescriptor::internal_default_instance()); ::cockroach::roachpb::_MergeTrigger_default_instance_._instance.get_mutable()->right_desc_ = const_cast< ::cockroach::roachpb::RangeDescriptor*>( ::cockroach::roachpb::RangeDescriptor::internal_default_instance()); + ::cockroach::roachpb::_MergeTrigger_default_instance_._instance.get_mutable()->right_mvcc_stats_ = const_cast< ::cockroach::storage::engine::enginepb::MVCCStats*>( + ::cockroach::storage::engine::enginepb::MVCCStats::internal_default_instance()); } void MergeTrigger::clear_left_desc() { if (GetArenaNoVirtual() == NULL && left_desc_ != NULL) { @@ -1614,10 +1620,16 @@ void MergeTrigger::clear_right_desc() { } right_desc_ = NULL; } +void MergeTrigger::clear_right_mvcc_stats() { + if (GetArenaNoVirtual() == NULL && right_mvcc_stats_ != NULL) { + delete right_mvcc_stats_; + } + right_mvcc_stats_ = NULL; +} #if !defined(_MSC_VER) || _MSC_VER >= 1900 const int MergeTrigger::kLeftDescFieldNumber; const int MergeTrigger::kRightDescFieldNumber; -const int MergeTrigger::kRightDataFieldNumber; +const int MergeTrigger::kRightMvccStatsFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 MergeTrigger::MergeTrigger() @@ -1631,10 +1643,6 @@ MergeTrigger::MergeTrigger(const MergeTrigger& from) : ::google::protobuf::MessageLite(), _internal_metadata_(NULL) { _internal_metadata_.MergeFrom(from._internal_metadata_); - right_data_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.right_data().size() > 0) { - right_data_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.right_data_); - } if (from.has_left_desc()) { left_desc_ = new ::cockroach::roachpb::RangeDescriptor(*from.left_desc_); } else { @@ -1645,14 +1653,18 @@ MergeTrigger::MergeTrigger(const MergeTrigger& from) } else { right_desc_ = NULL; } + if (from.has_right_mvcc_stats()) { + right_mvcc_stats_ = new ::cockroach::storage::engine::enginepb::MVCCStats(*from.right_mvcc_stats_); + } else { + right_mvcc_stats_ = NULL; + } // @@protoc_insertion_point(copy_constructor:cockroach.roachpb.MergeTrigger) } void MergeTrigger::SharedCtor() { - right_data_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); ::memset(&left_desc_, 0, static_cast( - reinterpret_cast(&right_desc_) - - reinterpret_cast(&left_desc_)) + sizeof(right_desc_)); + reinterpret_cast(&right_mvcc_stats_) - + reinterpret_cast(&left_desc_)) + sizeof(right_mvcc_stats_)); } MergeTrigger::~MergeTrigger() { @@ -1661,9 +1673,9 @@ MergeTrigger::~MergeTrigger() { } void MergeTrigger::SharedDtor() { - right_data_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); if (this != internal_default_instance()) delete left_desc_; if (this != internal_default_instance()) delete right_desc_; + if (this != internal_default_instance()) delete right_mvcc_stats_; } void MergeTrigger::SetCachedSize(int size) const { @@ -1681,7 +1693,6 @@ void MergeTrigger::Clear() { // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - right_data_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); if (GetArenaNoVirtual() == NULL && left_desc_ != NULL) { delete left_desc_; } @@ -1690,6 +1701,10 @@ void MergeTrigger::Clear() { delete right_desc_; } right_desc_ = NULL; + if (GetArenaNoVirtual() == NULL && right_mvcc_stats_ != NULL) { + delete right_mvcc_stats_; + } + right_mvcc_stats_ = NULL; _internal_metadata_.Clear(); } @@ -1731,12 +1746,11 @@ bool MergeTrigger::MergePartialFromCodedStream( break; } - // bytes right_data = 3; - case 3: { + case 4: { if (static_cast< ::google::protobuf::uint8>(tag) == - static_cast< ::google::protobuf::uint8>(26u /* 26 & 0xFF */)) { - DO_(::google::protobuf::internal::WireFormatLite::ReadBytes( - input, this->mutable_right_data())); + static_cast< ::google::protobuf::uint8>(34u /* 34 & 0xFF */)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadMessage( + input, mutable_right_mvcc_stats())); } else { goto handle_unusual; } @@ -1779,10 +1793,9 @@ void MergeTrigger::SerializeWithCachedSizes( 2, this->_internal_right_desc(), output); } - // bytes right_data = 3; - if (this->right_data().size() > 0) { - ::google::protobuf::internal::WireFormatLite::WriteBytesMaybeAliased( - 3, this->right_data(), output); + if (this->has_right_mvcc_stats()) { + ::google::protobuf::internal::WireFormatLite::WriteMessage( + 4, this->_internal_right_mvcc_stats(), output); } output->WriteRaw((::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).data(), @@ -1796,13 +1809,6 @@ size_t MergeTrigger::ByteSizeLong() const { total_size += (::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).size(); - // bytes right_data = 3; - if (this->right_data().size() > 0) { - total_size += 1 + - ::google::protobuf::internal::WireFormatLite::BytesSize( - this->right_data()); - } - if (this->has_left_desc()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::MessageSize( @@ -1815,6 +1821,12 @@ size_t MergeTrigger::ByteSizeLong() const { *right_desc_); } + if (this->has_right_mvcc_stats()) { + total_size += 1 + + ::google::protobuf::internal::WireFormatLite::MessageSize( + *right_mvcc_stats_); + } + int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); SetCachedSize(cached_size); return total_size; @@ -1832,16 +1844,15 @@ void MergeTrigger::MergeFrom(const MergeTrigger& from) { ::google::protobuf::uint32 cached_has_bits = 0; (void) cached_has_bits; - if (from.right_data().size() > 0) { - - right_data_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.right_data_); - } if (from.has_left_desc()) { mutable_left_desc()->::cockroach::roachpb::RangeDescriptor::MergeFrom(from.left_desc()); } if (from.has_right_desc()) { mutable_right_desc()->::cockroach::roachpb::RangeDescriptor::MergeFrom(from.right_desc()); } + if (from.has_right_mvcc_stats()) { + mutable_right_mvcc_stats()->::cockroach::storage::engine::enginepb::MVCCStats::MergeFrom(from.right_mvcc_stats()); + } } void MergeTrigger::CopyFrom(const MergeTrigger& from) { @@ -1861,10 +1872,9 @@ void MergeTrigger::Swap(MergeTrigger* other) { } void MergeTrigger::InternalSwap(MergeTrigger* other) { using std::swap; - right_data_.Swap(&other->right_data_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), - GetArenaNoVirtual()); swap(left_desc_, other->left_desc_); swap(right_desc_, other->right_desc_); + swap(right_mvcc_stats_, other->right_mvcc_stats_); _internal_metadata_.Swap(&other->_internal_metadata_); } diff --git a/c-deps/libroach/protos/roachpb/data.pb.h b/c-deps/libroach/protos/roachpb/data.pb.h index a7e10fbdc575..0827bbd66a13 100644 --- a/c-deps/libroach/protos/roachpb/data.pb.h +++ b/c-deps/libroach/protos/roachpb/data.pb.h @@ -31,6 +31,7 @@ #include // IWYU pragma: export #include #include "roachpb/metadata.pb.h" +#include "storage/engine/enginepb/mvcc.pb.h" #include "storage/engine/enginepb/mvcc3.pb.h" #include "util/hlc/timestamp.pb.h" // @@protoc_insertion_point(includes) @@ -853,20 +854,6 @@ class MergeTrigger : public ::google::protobuf::MessageLite /* @@protoc_insertio // accessors ------------------------------------------------------- - // bytes right_data = 3; - void clear_right_data(); - static const int kRightDataFieldNumber = 3; - const ::std::string& right_data() const; - void set_right_data(const ::std::string& value); - #if LANG_CXX11 - void set_right_data(::std::string&& value); - #endif - void set_right_data(const char* value); - void set_right_data(const void* value, size_t size); - ::std::string* mutable_right_data(); - ::std::string* release_right_data(); - void set_allocated_right_data(::std::string* right_data); - bool has_left_desc() const; void clear_left_desc(); static const int kLeftDescFieldNumber = 1; @@ -889,13 +876,24 @@ class MergeTrigger : public ::google::protobuf::MessageLite /* @@protoc_insertio ::cockroach::roachpb::RangeDescriptor* mutable_right_desc(); void set_allocated_right_desc(::cockroach::roachpb::RangeDescriptor* right_desc); + bool has_right_mvcc_stats() const; + void clear_right_mvcc_stats(); + static const int kRightMvccStatsFieldNumber = 4; + private: + const ::cockroach::storage::engine::enginepb::MVCCStats& _internal_right_mvcc_stats() const; + public: + const ::cockroach::storage::engine::enginepb::MVCCStats& right_mvcc_stats() const; + ::cockroach::storage::engine::enginepb::MVCCStats* release_right_mvcc_stats(); + ::cockroach::storage::engine::enginepb::MVCCStats* mutable_right_mvcc_stats(); + void set_allocated_right_mvcc_stats(::cockroach::storage::engine::enginepb::MVCCStats* right_mvcc_stats); + // @@protoc_insertion_point(class_scope:cockroach.roachpb.MergeTrigger) private: ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; - ::google::protobuf::internal::ArenaStringPtr right_data_; ::cockroach::roachpb::RangeDescriptor* left_desc_; ::cockroach::roachpb::RangeDescriptor* right_desc_; + ::cockroach::storage::engine::enginepb::MVCCStats* right_mvcc_stats_; mutable ::google::protobuf::internal::CachedSize _cached_size_; friend struct ::protobuf_roachpb_2fdata_2eproto::TableStruct; }; @@ -2957,57 +2955,51 @@ inline void MergeTrigger::set_allocated_right_desc(::cockroach::roachpb::RangeDe // @@protoc_insertion_point(field_set_allocated:cockroach.roachpb.MergeTrigger.right_desc) } -// bytes right_data = 3; -inline void MergeTrigger::clear_right_data() { - right_data_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); -} -inline const ::std::string& MergeTrigger::right_data() const { - // @@protoc_insertion_point(field_get:cockroach.roachpb.MergeTrigger.right_data) - return right_data_.GetNoArena(); -} -inline void MergeTrigger::set_right_data(const ::std::string& value) { - - right_data_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:cockroach.roachpb.MergeTrigger.right_data) -} -#if LANG_CXX11 -inline void MergeTrigger::set_right_data(::std::string&& value) { - - right_data_.SetNoArena( - &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:cockroach.roachpb.MergeTrigger.right_data) +inline bool MergeTrigger::has_right_mvcc_stats() const { + return this != internal_default_instance() && right_mvcc_stats_ != NULL; } -#endif -inline void MergeTrigger::set_right_data(const char* value) { - GOOGLE_DCHECK(value != NULL); - - right_data_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:cockroach.roachpb.MergeTrigger.right_data) +inline const ::cockroach::storage::engine::enginepb::MVCCStats& MergeTrigger::_internal_right_mvcc_stats() const { + return *right_mvcc_stats_; } -inline void MergeTrigger::set_right_data(const void* value, size_t size) { - - right_data_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), - ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:cockroach.roachpb.MergeTrigger.right_data) +inline const ::cockroach::storage::engine::enginepb::MVCCStats& MergeTrigger::right_mvcc_stats() const { + const ::cockroach::storage::engine::enginepb::MVCCStats* p = right_mvcc_stats_; + // @@protoc_insertion_point(field_get:cockroach.roachpb.MergeTrigger.right_mvcc_stats) + return p != NULL ? *p : *reinterpret_cast( + &::cockroach::storage::engine::enginepb::_MVCCStats_default_instance_); } -inline ::std::string* MergeTrigger::mutable_right_data() { +inline ::cockroach::storage::engine::enginepb::MVCCStats* MergeTrigger::release_right_mvcc_stats() { + // @@protoc_insertion_point(field_release:cockroach.roachpb.MergeTrigger.right_mvcc_stats) - // @@protoc_insertion_point(field_mutable:cockroach.roachpb.MergeTrigger.right_data) - return right_data_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + ::cockroach::storage::engine::enginepb::MVCCStats* temp = right_mvcc_stats_; + right_mvcc_stats_ = NULL; + return temp; } -inline ::std::string* MergeTrigger::release_right_data() { - // @@protoc_insertion_point(field_release:cockroach.roachpb.MergeTrigger.right_data) +inline ::cockroach::storage::engine::enginepb::MVCCStats* MergeTrigger::mutable_right_mvcc_stats() { - return right_data_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + if (right_mvcc_stats_ == NULL) { + auto* p = CreateMaybeMessage<::cockroach::storage::engine::enginepb::MVCCStats>(GetArenaNoVirtual()); + right_mvcc_stats_ = p; + } + // @@protoc_insertion_point(field_mutable:cockroach.roachpb.MergeTrigger.right_mvcc_stats) + return right_mvcc_stats_; } -inline void MergeTrigger::set_allocated_right_data(::std::string* right_data) { - if (right_data != NULL) { +inline void MergeTrigger::set_allocated_right_mvcc_stats(::cockroach::storage::engine::enginepb::MVCCStats* right_mvcc_stats) { + ::google::protobuf::Arena* message_arena = GetArenaNoVirtual(); + if (message_arena == NULL) { + delete reinterpret_cast< ::google::protobuf::MessageLite*>(right_mvcc_stats_); + } + if (right_mvcc_stats) { + ::google::protobuf::Arena* submessage_arena = NULL; + if (message_arena != submessage_arena) { + right_mvcc_stats = ::google::protobuf::internal::GetOwnedMessage( + message_arena, right_mvcc_stats, submessage_arena); + } } else { } - right_data_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), right_data); - // @@protoc_insertion_point(field_set_allocated:cockroach.roachpb.MergeTrigger.right_data) + right_mvcc_stats_ = right_mvcc_stats; + // @@protoc_insertion_point(field_set_allocated:cockroach.roachpb.MergeTrigger.right_mvcc_stats) } // ------------------------------------------------------------------- diff --git a/pkg/kv/dist_sender.go b/pkg/kv/dist_sender.go index 9a1bfbab326b..65df0755b9bd 100644 --- a/pkg/kv/dist_sender.go +++ b/pkg/kv/dist_sender.go @@ -1277,7 +1277,9 @@ func (ds *DistSender) sendToReplicas( return nil, roachpb.NewSendError( fmt.Sprintf("sending to all %d replicas failed", len(replicas))) } - log.VEventf(ctx, 2, "r%d: sending batch %s to %s", rangeID, args.Summary(), transport.NextReplica()) + + curReplica := transport.NextReplica() + log.VEventf(ctx, 2, "r%d: sending batch %s to %s", rangeID, args.Summary(), curReplica) br, err := transport.SendNext(ctx) // This loop will retry operations that fail with errors that reflect @@ -1302,6 +1304,21 @@ func (ds *DistSender) sendToReplicas( ambiguousError = err } log.VErrEventf(ctx, 2, "RPC error: %s", err) + if storeID, ok := ds.leaseHolderCache.Lookup(ctx, rangeID); ok && curReplica.StoreID == storeID { + // If the down replica is cached as the lease holder, evict + // it. The only other eviction happens below on + // NotLeaseHolderError, but if the next replica is the + // actual lease holder, we're never going to receive one of + // those and will thus pay the price of trying the down node + // first forever. + // + // NB: we could consider instead adding a successful reply + // from the next replica into the cache, but without a + // leaseholder (and taking into account that the local + // node can't be down) it won't take long until we talk + // to a replica that tells us who the leaseholder is. + ds.leaseHolderCache.Update(ctx, rangeID, 0 /* evict */) + } } else { propagateError := false switch tErr := br.Error.GetDetail().(type) { @@ -1375,7 +1392,8 @@ func (ds *DistSender) sendToReplicas( } ds.metrics.NextReplicaErrCount.Inc(1) - log.VEventf(ctx, 2, "error: %v %v; trying next peer %s", br, err, transport.NextReplica()) + curReplica = transport.NextReplica() + log.VEventf(ctx, 2, "error: %v %v; trying next peer %s", br, err, curReplica) br, err = transport.SendNext(ctx) } } diff --git a/pkg/kv/dist_sender_test.go b/pkg/kv/dist_sender_test.go index 9a17ca9d6943..0225dc261a96 100644 --- a/pkg/kv/dist_sender_test.go +++ b/pkg/kv/dist_sender_test.go @@ -131,21 +131,24 @@ func adaptSimpleTransport(fn simpleSendFn) TransportFactory { replicas ReplicaSlice, args roachpb.BatchRequest, ) (Transport, error) { - return &simpleTransportAdapter{fn, opts, replicas, args, false}, nil + return &simpleTransportAdapter{ + fn: fn, + opts: opts, + replicas: replicas, + args: args}, nil } } type simpleTransportAdapter struct { - fn simpleSendFn - opts SendOptions - replicas ReplicaSlice - args roachpb.BatchRequest - - called bool + fn simpleSendFn + opts SendOptions + replicas ReplicaSlice + args roachpb.BatchRequest + nextReplica int } func (l *simpleTransportAdapter) IsExhausted() bool { - return l.called + return l.nextReplica >= len(l.replicas) } func (l *simpleTransportAdapter) GetPending() []roachpb.ReplicaDescriptor { @@ -153,11 +156,15 @@ func (l *simpleTransportAdapter) GetPending() []roachpb.ReplicaDescriptor { } func (l *simpleTransportAdapter) SendNext(ctx context.Context) (*roachpb.BatchResponse, error) { - l.called = true + l.args.Replica = l.replicas[l.nextReplica].ReplicaDescriptor + l.nextReplica++ return l.fn(ctx, l.opts, l.replicas, l.args) } func (l *simpleTransportAdapter) NextReplica() roachpb.ReplicaDescriptor { + if !l.IsExhausted() { + return l.replicas[l.nextReplica].ReplicaDescriptor + } return roachpb.ReplicaDescriptor{} } @@ -568,6 +575,93 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { } } +// This test verifies that when we have a cached leaseholder that is down +// it is ejected from the cache. +func TestDistSenderDownNodeEvictLeaseholder(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + g, clock := makeGossip(t, stopper) + if err := g.AddInfoProto( + gossip.MakeNodeIDKey(roachpb.NodeID(2)), + &roachpb.NodeDescriptor{ + NodeID: 2, + Address: util.MakeUnresolvedAddr("tcp", "neverused:12345"), + }, + gossip.NodeDescriptorTTL, + ); err != nil { + t.Fatal(err) + } + + var contacted1, contacted2 bool + + transport := func( + ctx context.Context, + opts SendOptions, + replicas ReplicaSlice, + ba roachpb.BatchRequest, + ) (*roachpb.BatchResponse, error) { + switch ba.Replica.StoreID { + case 1: + contacted1 = true + return nil, errors.New("mock RPC error") + case 2: + contacted2 = true + return ba.CreateReply(), nil + default: + panic("unexpected replica: " + ba.Replica.String()) + } + } + + cfg := DistSenderConfig{ + AmbientCtx: log.AmbientContext{Tracer: tracing.NewTracer()}, + Clock: clock, + TestingKnobs: ClientTestingKnobs{ + TransportFactory: adaptSimpleTransport(transport), + }, + RangeDescriptorDB: mockRangeDescriptorDBForDescs( + roachpb.RangeDescriptor{ + RangeID: 1, + StartKey: roachpb.RKeyMin, + EndKey: roachpb.RKeyMax, + Replicas: []roachpb.ReplicaDescriptor{ + { + NodeID: 1, + StoreID: 1, + }, + { + NodeID: 2, + StoreID: 2, + }, + }, + }), + } + + ds := NewDistSender(cfg, g) + ds.LeaseHolderCache().Update(ctx, roachpb.RangeID(1), roachpb.StoreID(1)) + + var ba roachpb.BatchRequest + ba.RangeID = 1 + get := &roachpb.GetRequest{} + get.Key = roachpb.Key("a") + ba.Add(get) + + if _, pErr := ds.Send(ctx, ba); pErr != nil { + t.Fatal(pErr) + } + + if !contacted1 || !contacted2 { + t.Errorf("contacted n1: %t, contacted n2: %t", contacted1, contacted2) + } + + if storeID, ok := ds.LeaseHolderCache().Lookup(ctx, roachpb.RangeID(1)); ok { + t.Fatalf("expected no lease holder for r1, but got s%d", storeID) + } +} + // TestRetryOnDescriptorLookupError verifies that the DistSender retries a descriptor // lookup on any error. func TestRetryOnDescriptorLookupError(t *testing.T) { diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index c066152c0ded..7804b9e8d412 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -1933,10 +1933,7 @@ func (*GetSnapshotForMergeRequest) Descriptor() ([]byte, []int) { return fileDes // GetSnapshotForMergeResponse is the response to a GetSnapshotForMergeRequest. type GetSnapshotForMergeResponse struct { ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - // Data is a snapshot of the right-hand range's data in the form of a - // BatchRepr, a serialized RocksDB Batch. - Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` - // MVCCStats are the MVCC statistics for data. + // MVCCStats are the MVCC statistics for the range. MVCCStats cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,3,opt,name=mvcc_stats,json=mvccStats" json:"mvcc_stats"` // LeaseAppliedIndex is the lease index of the last applied command at the // time that the GetSnapshotForMerge request executed. @@ -9930,12 +9927,6 @@ func (m *GetSnapshotForMergeResponse) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n130 - if len(m.Data) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintApi(dAtA, i, uint64(len(m.Data))) - i += copy(dAtA[i:], m.Data) - } dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) @@ -12899,10 +12890,6 @@ func (m *GetSnapshotForMergeResponse) Size() (n int) { _ = l l = m.ResponseHeader.Size() n += 1 + l + sovApi(uint64(l)) - l = len(m.Data) - if l > 0 { - n += 1 + l + sovApi(uint64(l)) - } l = m.MVCCStats.Size() n += 1 + l + sovApi(uint64(l)) if m.LeaseAppliedIndex != 0 { @@ -24726,37 +24713,6 @@ func (m *GetSnapshotForMergeResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLengthApi - } - postIndex := iNdEx + byteLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...) - if m.Data == nil { - m.Data = []byte{} - } - iNdEx = postIndex case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field MVCCStats", wireType) @@ -29065,372 +29021,372 @@ var ( func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptorApi) } var fileDescriptorApi = []byte{ - // 5870 bytes of a gzipped FileDescriptorProto + // 5866 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7c, 0x5b, 0x6c, 0x24, 0x49, 0x56, 0xb6, 0xb3, 0xaa, 0x6c, 0x57, 0x9d, 0x72, 0x95, 0xd3, 0xe1, 0xbe, 0x54, 0xbb, 0x7b, 0xda, 0xee, 0x9a, 0xbe, 0xcf, 0xae, 0xfb, 0x6f, 0xf7, 0xce, 0x3f, 0xcb, 0xcc, 0xde, 0x5c, 0x65, 0xb7, 0xab, 0xda, 0xed, 0xcb, 0x44, 0x95, 0x7b, 0xb6, 0x67, 0x99, 0xcd, 0x4d, 0x67, 0x86, 0xcb, 0x49, - 0x57, 0x65, 0x56, 0x67, 0x66, 0xb5, 0xed, 0x96, 0x10, 0x08, 0x10, 0x8b, 0x56, 0xab, 0x15, 0x12, - 0x08, 0x21, 0x01, 0x62, 0xa5, 0x7d, 0x40, 0x42, 0x62, 0x01, 0x81, 0x84, 0x40, 0x62, 0xf7, 0x85, - 0x87, 0x79, 0xd8, 0x87, 0x01, 0x89, 0x8b, 0x90, 0xb0, 0xc0, 0xf0, 0xb0, 0xe2, 0x15, 0xf1, 0x32, - 0xe2, 0x01, 0xc5, 0x25, 0x2f, 0xe5, 0xca, 0xba, 0xb4, 0x37, 0x47, 0x0c, 0xe2, 0xa9, 0x2a, 0x4f, - 0xc4, 0x39, 0x11, 0x71, 0xe2, 0xc4, 0x89, 0x2f, 0x22, 0x4e, 0x04, 0xcc, 0xd8, 0x96, 0xaa, 0xed, - 0xb7, 0x77, 0xef, 0xa9, 0x6d, 0x63, 0xb1, 0x6d, 0x5b, 0xae, 0x85, 0x66, 0x34, 0x4b, 0x7b, 0xc6, - 0xc8, 0x8b, 0x22, 0x71, 0x0e, 0x79, 0xb9, 0x74, 0xd5, 0x55, 0x79, 0xb6, 0xb9, 0x73, 0x1e, 0x8d, - 0xd8, 0xb6, 0x65, 0x3b, 0x82, 0x7a, 0xc1, 0xa3, 0xb6, 0x88, 0xab, 0x86, 0x72, 0x17, 0x1d, 0xd7, - 0xb2, 0xd5, 0x06, 0xb9, 0x47, 0xcc, 0x86, 0x61, 0x7a, 0x3f, 0x34, 0xdf, 0x0b, 0x4d, 0x13, 0x79, - 0x5e, 0x1f, 0x94, 0xe7, 0x81, 0xc8, 0x54, 0xe8, 0xb8, 0x46, 0xf3, 0xde, 0x7e, 0x53, 0xbb, 0xe7, - 0x1a, 0x2d, 0xe2, 0xb8, 0x6a, 0xab, 0x2d, 0x52, 0x16, 0x58, 0x8a, 0x6b, 0xab, 0x9a, 0x61, 0x36, - 0xee, 0xd9, 0x44, 0xb3, 0x6c, 0x9d, 0xe8, 0x8a, 0xd3, 0x56, 0x4d, 0xaf, 0xca, 0x0d, 0xab, 0x61, - 0xb1, 0xbf, 0xf7, 0xe8, 0x3f, 0x4e, 0x2d, 0xfe, 0x1c, 0x64, 0xb0, 0x6a, 0x36, 0x48, 0xd5, 0xdc, - 0xb3, 0xd0, 0x17, 0x20, 0xa5, 0x13, 0x47, 0x2b, 0x48, 0x0b, 0xd2, 0xed, 0xec, 0x52, 0x71, 0xb1, - 0x47, 0x17, 0x8b, 0x2c, 0xef, 0x0a, 0x71, 0x34, 0xdb, 0x68, 0xbb, 0x96, 0x5d, 0x4a, 0x7d, 0x78, - 0x3c, 0x3f, 0x86, 0x19, 0x17, 0xfa, 0x1c, 0x8c, 0x37, 0x89, 0xea, 0x90, 0x42, 0x82, 0xb1, 0x17, - 0x22, 0xd8, 0x1f, 0xd3, 0x74, 0xc1, 0xc4, 0x33, 0x17, 0x5f, 0x42, 0x0e, 0x93, 0xe7, 0x1d, 0xe2, - 0xb8, 0x15, 0xa2, 0xea, 0xc4, 0x46, 0x97, 0x20, 0xf9, 0x8c, 0x1c, 0x15, 0x92, 0x0b, 0xd2, 0xed, - 0xa9, 0xd2, 0xe4, 0xc7, 0xc7, 0xf3, 0xc9, 0x75, 0x72, 0x84, 0x29, 0x0d, 0x2d, 0xc0, 0x24, 0x31, - 0x75, 0x85, 0x26, 0xa7, 0xba, 0x93, 0x27, 0x88, 0xa9, 0xaf, 0x93, 0x23, 0x34, 0x07, 0x69, 0x87, - 0x4a, 0x33, 0x35, 0x52, 0x18, 0x5f, 0x90, 0x6e, 0x8f, 0x63, 0xff, 0xfb, 0xed, 0xd4, 0x8f, 0xbf, - 0x3b, 0x2f, 0x3d, 0x4a, 0xa5, 0x25, 0x39, 0xf1, 0x28, 0x95, 0x4e, 0xc8, 0xc9, 0xe2, 0xb7, 0x93, - 0x90, 0xc7, 0xc4, 0x69, 0x5b, 0xa6, 0x43, 0x44, 0xe9, 0xff, 0x0f, 0x92, 0xee, 0xa1, 0xc9, 0x4a, - 0xcf, 0x2e, 0x5d, 0x8d, 0x68, 0x42, 0xdd, 0x56, 0x4d, 0x47, 0xd5, 0x5c, 0xc3, 0x32, 0x31, 0xcd, - 0x8a, 0x3e, 0x0f, 0x59, 0x9b, 0x38, 0x9d, 0x16, 0x61, 0xca, 0x66, 0x15, 0xcb, 0x2e, 0x5d, 0x8c, - 0xe0, 0xac, 0xb5, 0x55, 0x13, 0x03, 0xcf, 0x4b, 0xff, 0xa3, 0x4b, 0x90, 0x36, 0x3b, 0x2d, 0xda, - 0x1c, 0x87, 0x55, 0x36, 0x89, 0x27, 0xcd, 0x4e, 0x6b, 0x9d, 0x1c, 0x39, 0xa8, 0x0c, 0x59, 0x9b, - 0xaa, 0x5a, 0x31, 0xcc, 0x3d, 0xcb, 0x29, 0x4c, 0x2c, 0x24, 0x6f, 0x67, 0x97, 0xae, 0xf4, 0xeb, - 0x10, 0xda, 0x79, 0x42, 0xab, 0x60, 0x7b, 0x04, 0x07, 0xd5, 0x20, 0x27, 0x6a, 0x66, 0x13, 0xd5, - 0xb1, 0xcc, 0xc2, 0xe4, 0x82, 0x74, 0x3b, 0xbf, 0xb4, 0x18, 0x25, 0xa6, 0x4b, 0x0b, 0xf4, 0xb3, - 0xd3, 0x22, 0x98, 0x71, 0xe1, 0x29, 0x3b, 0xf4, 0x55, 0x7c, 0x0a, 0x53, 0xe1, 0x54, 0x84, 0x20, - 0x8f, 0x57, 0x6b, 0x3b, 0x1b, 0xab, 0xca, 0xce, 0xe6, 0xfa, 0xe6, 0xd6, 0x7b, 0x9b, 0xf2, 0x18, - 0x3a, 0x07, 0xb2, 0xa0, 0xad, 0xaf, 0x3e, 0x55, 0x1e, 0x57, 0x37, 0xaa, 0x75, 0x59, 0x42, 0x97, - 0xe0, 0xbc, 0xa0, 0xe2, 0xe5, 0xcd, 0xb5, 0x55, 0xa5, 0xb4, 0xb5, 0xb3, 0xb9, 0xb2, 0x8c, 0x9f, - 0xca, 0x89, 0xb9, 0xd4, 0xaf, 0x7c, 0xef, 0xea, 0x58, 0xf1, 0x09, 0xc0, 0x1a, 0x71, 0x85, 0x35, - 0xa0, 0x12, 0x4c, 0xec, 0xb3, 0xda, 0x08, 0x73, 0x5c, 0x88, 0xac, 0x76, 0xc8, 0x72, 0x4a, 0x69, - 0xaa, 0x81, 0x8f, 0x8e, 0xe7, 0x25, 0x2c, 0x38, 0x79, 0x97, 0x17, 0x7f, 0x28, 0x41, 0x96, 0x09, - 0xe6, 0x6d, 0x44, 0xe5, 0x53, 0x92, 0xaf, 0x0d, 0x55, 0x48, 0xaf, 0x68, 0xb4, 0x08, 0xe3, 0x2f, - 0xd4, 0x66, 0x67, 0x90, 0xb5, 0x3f, 0xa1, 0xe9, 0x98, 0x67, 0x43, 0xef, 0xc0, 0x94, 0x61, 0xba, - 0xc4, 0x74, 0x15, 0xce, 0x96, 0x1c, 0xc2, 0x96, 0xe5, 0xb9, 0xd9, 0x47, 0xf1, 0x2f, 0x24, 0x80, - 0xed, 0x4e, 0x9c, 0xaa, 0xa1, 0xa3, 0x75, 0xa4, 0xfa, 0x7b, 0xa3, 0x95, 0xb7, 0xe2, 0x02, 0x4c, - 0x18, 0x66, 0xd3, 0x30, 0x79, 0xfd, 0xd3, 0x58, 0x7c, 0xa1, 0x73, 0x30, 0xbe, 0xdb, 0x34, 0x4c, - 0x9d, 0x99, 0x7f, 0x1a, 0xf3, 0x0f, 0xa1, 0x7e, 0x0c, 0x59, 0x56, 0xf7, 0x18, 0xb5, 0x5f, 0xfc, - 0x37, 0x09, 0xce, 0x97, 0x2d, 0x53, 0x37, 0xe8, 0x38, 0x54, 0x9b, 0x9f, 0x0a, 0xdd, 0xbc, 0x09, - 0x19, 0x72, 0xd8, 0x1e, 0xb1, 0x7b, 0xd3, 0xe4, 0xb0, 0xcd, 0xfe, 0x0d, 0x54, 0xdd, 0x07, 0x70, - 0xe1, 0x74, 0x2b, 0xe3, 0xd4, 0xe2, 0xdf, 0x48, 0x90, 0xaf, 0x9a, 0x86, 0xfb, 0xa9, 0x50, 0x9f, - 0xaf, 0x87, 0x64, 0x48, 0x0f, 0xe8, 0x2e, 0xc8, 0x7b, 0xaa, 0xd1, 0xdc, 0x32, 0xeb, 0x56, 0x6b, - 0xd7, 0x71, 0x2d, 0x93, 0x38, 0x42, 0x51, 0x3d, 0x74, 0xa1, 0xb3, 0x27, 0x30, 0xed, 0xb7, 0x29, - 0x4e, 0x65, 0xbd, 0x04, 0xb9, 0x6a, 0x6a, 0x36, 0x69, 0x11, 0x33, 0x56, 0x6d, 0x5d, 0x81, 0x8c, - 0xe1, 0xc9, 0x65, 0x1a, 0x4b, 0xe2, 0x80, 0x20, 0xda, 0xd4, 0x81, 0x99, 0x50, 0xd9, 0x71, 0xba, - 0xb1, 0xcb, 0x90, 0x31, 0xc9, 0x81, 0x12, 0xf4, 0x57, 0x12, 0xa7, 0x4d, 0x72, 0xc0, 0xdd, 0xce, - 0x53, 0xc8, 0xad, 0x90, 0x26, 0x71, 0x49, 0xfc, 0x3e, 0x79, 0x07, 0xf2, 0x9e, 0xe8, 0x38, 0x3b, - 0xe9, 0x77, 0x24, 0x40, 0x42, 0x2e, 0x9d, 0x07, 0xe3, 0xec, 0xa7, 0x79, 0x3a, 0xcf, 0xbb, 0x1d, - 0xdb, 0xe4, 0x13, 0x36, 0xb7, 0x52, 0xe0, 0x24, 0x36, 0x67, 0x07, 0xbe, 0x31, 0x15, 0xf6, 0x8d, - 0x3e, 0xee, 0xa0, 0x88, 0xe3, 0x00, 0x66, 0xbb, 0xaa, 0x17, 0x6f, 0x57, 0xa6, 0x58, 0xcd, 0x12, - 0x0b, 0xc9, 0x30, 0x34, 0x62, 0xc4, 0xe2, 0x07, 0x30, 0x53, 0x6e, 0x12, 0xd5, 0x8e, 0x5b, 0x2d, - 0xa2, 0x3b, 0x9f, 0x02, 0x0a, 0x8b, 0x8f, 0xb3, 0x4b, 0x0d, 0xc8, 0xd6, 0x34, 0xd5, 0xdc, 0x6a, - 0x53, 0x27, 0xe8, 0xa0, 0x07, 0x70, 0xc1, 0x71, 0xad, 0xb6, 0xa2, 0xba, 0x0a, 0x47, 0x48, 0xbb, - 0x56, 0xc7, 0xd4, 0x55, 0xfb, 0x88, 0x95, 0x91, 0xc6, 0xb3, 0x34, 0x75, 0xd9, 0x65, 0x15, 0x29, - 0x89, 0x24, 0xda, 0x77, 0x2d, 0xc3, 0x54, 0x28, 0x90, 0x69, 0xba, 0x8e, 0xb0, 0x73, 0x68, 0x19, - 0x26, 0xe6, 0x14, 0xd1, 0x8a, 0xef, 0x49, 0xbc, 0xac, 0x38, 0xcd, 0xe6, 0x4b, 0x90, 0x75, 0x34, - 0xd5, 0x54, 0xf6, 0x2c, 0xbb, 0xa5, 0xba, 0xcc, 0x34, 0xf2, 0x4b, 0xaf, 0x45, 0xc1, 0x43, 0x4d, - 0x35, 0x1f, 0xb2, 0x4c, 0x18, 0x1c, 0xff, 0x7f, 0xd8, 0x7a, 0x1e, 0xa5, 0xd2, 0x49, 0x39, 0x55, - 0xfc, 0x4f, 0x09, 0xa6, 0x78, 0x2d, 0xe3, 0xb4, 0x9e, 0x37, 0x21, 0x65, 0x5b, 0x07, 0xdc, 0x7a, - 0xb2, 0x4b, 0x97, 0x23, 0x44, 0xac, 0x93, 0xa3, 0xb0, 0xdb, 0x66, 0xd9, 0x51, 0x09, 0x04, 0x50, - 0x51, 0x18, 0x77, 0x72, 0x54, 0x6e, 0xe0, 0x5c, 0x98, 0xca, 0xb8, 0x01, 0xf9, 0x5d, 0xd5, 0xd5, - 0xf6, 0x69, 0xff, 0xb0, 0x4a, 0x72, 0x74, 0x8f, 0x73, 0x8c, 0xea, 0xd5, 0xbc, 0xf8, 0xfb, 0x12, - 0x20, 0x4c, 0x5e, 0x10, 0xdb, 0x21, 0x9f, 0xfe, 0x4e, 0xfa, 0x2f, 0x09, 0x66, 0xbb, 0x2a, 0xfb, - 0x7f, 0xab, 0xaf, 0x7e, 0x5e, 0x82, 0x8b, 0xe5, 0x7d, 0xa2, 0x3d, 0x2b, 0x5b, 0xa6, 0x63, 0x38, - 0x2e, 0x31, 0xb5, 0xa3, 0x38, 0x3b, 0xec, 0x32, 0x64, 0x0e, 0x0c, 0x77, 0x5f, 0xd1, 0x8d, 0xbd, - 0x3d, 0x36, 0x9c, 0xd3, 0x38, 0x4d, 0x09, 0x2b, 0xc6, 0xde, 0x9e, 0x18, 0xcc, 0x0a, 0x14, 0x7a, - 0x6b, 0x10, 0x2f, 0x20, 0x38, 0x8f, 0x89, 0x66, 0xb5, 0xda, 0x1d, 0x97, 0xd4, 0x5c, 0xd5, 0x75, - 0xe2, 0x6c, 0xe0, 0x45, 0x98, 0xd4, 0xed, 0x23, 0xc5, 0xee, 0x98, 0xa2, 0x79, 0x13, 0xba, 0x7d, - 0x84, 0x3b, 0xa6, 0x68, 0xdc, 0x9f, 0x49, 0x70, 0xe1, 0x74, 0xe1, 0x71, 0x5a, 0xd8, 0x57, 0x21, - 0xab, 0xea, 0x3a, 0xd1, 0x15, 0x9d, 0x34, 0x5d, 0x55, 0x00, 0xb9, 0xfb, 0x21, 0x49, 0x62, 0xb7, - 0x62, 0x91, 0x6f, 0x53, 0x2c, 0x7a, 0xbb, 0x15, 0x8b, 0x1b, 0x4f, 0xca, 0x65, 0x56, 0x9f, 0x15, - 0xca, 0xe8, 0x19, 0x10, 0x93, 0xc5, 0x28, 0x45, 0x0d, 0x2e, 0x96, 0x48, 0xc3, 0x30, 0xc3, 0xeb, - 0xe8, 0xd8, 0xa7, 0x23, 0x05, 0x0a, 0xbd, 0x85, 0xc4, 0xd9, 0xf7, 0x7f, 0x9d, 0x84, 0xf3, 0xab, - 0xa6, 0xfe, 0xc9, 0x34, 0x82, 0x22, 0x09, 0xcd, 0x6a, 0xb5, 0x0c, 0xd7, 0xeb, 0x7b, 0xfe, 0x85, - 0x7e, 0x0a, 0xd2, 0x3a, 0x51, 0x75, 0x7f, 0xfd, 0x95, 0xed, 0xf2, 0x51, 0x1d, 0xd7, 0x68, 0x2e, - 0xee, 0x37, 0xb5, 0xc5, 0xba, 0xb7, 0x37, 0x84, 0xfd, 0xec, 0xe8, 0x1b, 0x70, 0x91, 0x8e, 0x62, - 0xdb, 0x54, 0x9b, 0x0a, 0x97, 0xa6, 0xb8, 0xb6, 0xd1, 0x68, 0x10, 0x5b, 0xec, 0x58, 0xdc, 0x8e, - 0xa8, 0x67, 0x55, 0x70, 0x94, 0x19, 0x43, 0x9d, 0xe7, 0xc7, 0xe7, 0x8d, 0x28, 0x32, 0xfa, 0x8a, - 0xbf, 0xc0, 0x75, 0xda, 0xaa, 0xe9, 0x14, 0xc6, 0x99, 0x7b, 0xe9, 0xb7, 0x11, 0x22, 0x2c, 0x43, - 0x38, 0x24, 0x4a, 0x71, 0xd0, 0x3d, 0x8a, 0xb0, 0x9e, 0x77, 0x0c, 0x9b, 0x28, 0xf7, 0xdb, 0x5a, - 0x61, 0x82, 0xb6, 0xbd, 0x94, 0x3f, 0x39, 0x9e, 0x07, 0xcc, 0xc9, 0xf7, 0xb7, 0xcb, 0x14, 0x71, - 0xf1, 0xff, 0x6d, 0x0d, 0xdd, 0x06, 0xd9, 0xb4, 0x14, 0x9b, 0xec, 0xd9, 0xc4, 0xd9, 0x17, 0xc5, - 0xa6, 0x99, 0xc6, 0xf2, 0xa6, 0x85, 0x39, 0x99, 0x8b, 0xbe, 0x00, 0x13, 0x6d, 0xcb, 0x70, 0x2c, - 0xb3, 0x90, 0xe1, 0x1a, 0xe5, 0x5f, 0xbe, 0xe3, 0x9e, 0x94, 0xd3, 0xc5, 0x5f, 0x93, 0xe0, 0xc2, - 0xe9, 0x3e, 0x8d, 0x73, 0x4c, 0xdd, 0x06, 0xd9, 0x32, 0x89, 0xd2, 0xde, 0x57, 0x1d, 0x22, 0xfa, - 0x40, 0x20, 0xc5, 0xbc, 0x65, 0x92, 0x6d, 0x4a, 0xe6, 0x1a, 0xed, 0x9a, 0x48, 0x7e, 0x51, 0x82, - 0x99, 0x65, 0xbd, 0x65, 0x98, 0xb5, 0x76, 0xd3, 0x88, 0x75, 0xe1, 0x71, 0x1d, 0x32, 0x0e, 0x95, - 0xc9, 0xf6, 0xd3, 0x12, 0xdd, 0xfb, 0x69, 0x69, 0x96, 0xb2, 0x4e, 0x8e, 0x02, 0x7c, 0x17, 0xae, - 0x44, 0x9c, 0x43, 0xe9, 0x03, 0xd1, 0xbe, 0x0d, 0x62, 0x7f, 0x42, 0xc8, 0x34, 0x2c, 0x3e, 0xce, - 0x9a, 0x7f, 0x4b, 0x82, 0x4b, 0x4c, 0x36, 0x33, 0x99, 0x3d, 0x62, 0xb3, 0xed, 0xcd, 0x38, 0xbb, - 0xe8, 0x75, 0x98, 0x70, 0x55, 0xbb, 0x41, 0xb8, 0x23, 0x18, 0x2f, 0x65, 0x3f, 0x3e, 0x9e, 0x9f, - 0xac, 0xb9, 0x96, 0x4d, 0xaa, 0x2b, 0x58, 0x24, 0x89, 0x76, 0xaa, 0x30, 0x17, 0x55, 0x97, 0x38, - 0xdb, 0xfb, 0x1f, 0x92, 0x28, 0xa3, 0xbc, 0xcf, 0x61, 0x7e, 0xbb, 0x69, 0x68, 0x6a, 0xac, 0xd3, - 0xde, 0x2a, 0x64, 0x35, 0x26, 0x5c, 0x71, 0x8f, 0xda, 0x7c, 0x41, 0x9a, 0x5f, 0xba, 0x1e, 0x29, - 0x88, 0x15, 0xce, 0x6b, 0x52, 0x3f, 0x6a, 0x13, 0x0c, 0x9a, 0xff, 0x1f, 0xad, 0xc0, 0x24, 0x57, - 0x8e, 0x87, 0x72, 0x06, 0x88, 0xa0, 0x03, 0xbd, 0xce, 0x32, 0x0b, 0x9f, 0xe4, 0xb1, 0x0a, 0xc5, - 0xee, 0xc2, 0xe5, 0xc8, 0x46, 0xc7, 0xa9, 0xd9, 0xdf, 0x90, 0x60, 0xb6, 0x42, 0x54, 0xdb, 0xdd, - 0x25, 0xaa, 0x5b, 0x3f, 0x8c, 0x75, 0x32, 0x79, 0x13, 0x92, 0xa6, 0x75, 0x20, 0xa6, 0xf0, 0xc1, - 0xf3, 0x85, 0x50, 0x00, 0xcd, 0x2f, 0x1a, 0xff, 0x35, 0x38, 0xd7, 0x5d, 0xaf, 0x38, 0x5b, 0xfd, - 0xc7, 0x49, 0xc8, 0xac, 0x95, 0xe3, 0x6c, 0xeb, 0x17, 0xc4, 0x12, 0x98, 0x77, 0x7a, 0xd4, 0x01, - 0x86, 0x5f, 0xde, 0xe2, 0x5a, 0x79, 0x9d, 0x1c, 0x79, 0xf8, 0x98, 0x72, 0xa1, 0x65, 0xc8, 0xb8, - 0xfb, 0x74, 0xce, 0xb0, 0x9a, 0xba, 0x98, 0x15, 0x47, 0xd2, 0x57, 0xc0, 0x85, 0x9a, 0x70, 0xde, - 0x3d, 0x34, 0xd9, 0x54, 0xa4, 0x34, 0x34, 0x25, 0x10, 0x37, 0x3e, 0x8a, 0xb8, 0x39, 0x2a, 0xee, - 0xe4, 0x78, 0x1e, 0xd5, 0x0f, 0x4d, 0x3a, 0x6f, 0xad, 0x95, 0xeb, 0x9e, 0x00, 0x8c, 0x5c, 0x41, - 0xd3, 0x7c, 0xda, 0xdc, 0x33, 0x18, 0x67, 0xad, 0xf0, 0xce, 0x4c, 0xa4, 0x88, 0x33, 0x13, 0xda, - 0x28, 0xaf, 0x80, 0x57, 0x31, 0x82, 0x80, 0x8b, 0x9b, 0x82, 0x30, 0x88, 0x77, 0x01, 0xa8, 0x0a, - 0xe3, 0x34, 0x83, 0x1f, 0x24, 0x21, 0xbf, 0xdd, 0x71, 0xf6, 0x63, 0xb6, 0xfb, 0x32, 0x40, 0xbb, - 0xe3, 0xec, 0x13, 0x5b, 0x71, 0x0f, 0x4d, 0xd1, 0xf2, 0x21, 0x07, 0x3a, 0x5e, 0xd3, 0x39, 0x5f, - 0xfd, 0xd0, 0x44, 0x5b, 0x42, 0x08, 0x51, 0x82, 0x53, 0xa1, 0xbb, 0x23, 0xc0, 0xe0, 0xfa, 0xa1, - 0xb9, 0x41, 0x7c, 0xfc, 0xcb, 0x05, 0x12, 0x2a, 0xf0, 0x0b, 0x30, 0x49, 0x3f, 0x14, 0xd7, 0x7a, - 0x15, 0x0b, 0x9b, 0xa0, 0x3c, 0x75, 0xcb, 0x1b, 0xcb, 0xe3, 0xaf, 0x36, 0x96, 0xd1, 0x3b, 0x90, - 0xe1, 0x85, 0x52, 0x9f, 0x3a, 0xc1, 0x7c, 0x6a, 0x94, 0x26, 0x44, 0x27, 0x30, 0x6f, 0x9a, 0x66, - 0x25, 0x52, 0x5f, 0x7a, 0x0e, 0xc6, 0xf7, 0x2c, 0x5b, 0x23, 0xec, 0xf4, 0x28, 0x8d, 0xf9, 0x87, - 0x0f, 0x9c, 0xd2, 0x72, 0xa6, 0xf8, 0x5b, 0x12, 0x4c, 0xfb, 0x1d, 0x18, 0x27, 0x62, 0x2a, 0x77, - 0x69, 0xff, 0xd5, 0xbb, 0x90, 0x6a, 0xbc, 0xf8, 0xfd, 0x04, 0x4c, 0xbf, 0xdb, 0x21, 0xf6, 0x51, - 0xcc, 0xf6, 0x55, 0xe2, 0x27, 0x85, 0x89, 0x33, 0xda, 0x04, 0x3b, 0x3b, 0xbc, 0x09, 0xd3, 0x07, - 0xaa, 0xe1, 0x2a, 0x7b, 0x96, 0xad, 0x74, 0xda, 0xba, 0xea, 0x7a, 0xe7, 0x2a, 0x39, 0x4a, 0x7e, - 0x68, 0xd9, 0x3b, 0x8c, 0x88, 0x08, 0xa0, 0x67, 0xa6, 0x75, 0x60, 0x2a, 0x94, 0x6c, 0x98, 0x0d, - 0xaa, 0x0f, 0xa7, 0x90, 0x62, 0x1b, 0x7d, 0x6f, 0xfd, 0xe3, 0xf1, 0xfc, 0x83, 0x86, 0xe1, 0xee, - 0x77, 0x76, 0x17, 0x35, 0xab, 0x75, 0xcf, 0xaf, 0x88, 0xbe, 0x1b, 0xfc, 0xbf, 0xd7, 0x7e, 0xd6, - 0xb8, 0xc7, 0x8e, 0x88, 0x3b, 0x1d, 0x43, 0x5f, 0xdc, 0xd9, 0xa9, 0xae, 0x60, 0x99, 0x89, 0x7c, - 0x8f, 0x4b, 0xac, 0x1f, 0x9a, 0xde, 0x84, 0xf7, 0xb1, 0x04, 0x72, 0xa0, 0xb0, 0x38, 0xfb, 0x73, - 0x15, 0xb2, 0xcf, 0x3b, 0xc4, 0x36, 0x88, 0xfe, 0xca, 0x1d, 0x0a, 0x82, 0x91, 0x8e, 0xa1, 0xf7, - 0x61, 0xaa, 0x4b, 0x0f, 0xc9, 0x9f, 0x4c, 0x0f, 0xd9, 0x83, 0x40, 0x05, 0xc5, 0x1f, 0x24, 0x00, - 0xb1, 0xc6, 0x57, 0xf9, 0x9e, 0xc7, 0xa7, 0xcc, 0x60, 0x9e, 0x00, 0x18, 0x7b, 0x4a, 0xcb, 0x70, - 0x1c, 0xc3, 0x6c, 0x30, 0x5b, 0xc9, 0x2f, 0xbd, 0x15, 0x51, 0x97, 0xde, 0x26, 0x2c, 0x56, 0xf7, - 0x36, 0x38, 0x5b, 0x89, 0xec, 0xab, 0x2f, 0x0c, 0xcb, 0xc6, 0x19, 0xc3, 0x23, 0x15, 0x4b, 0x30, - 0xd3, 0x93, 0x8e, 0xf2, 0x00, 0x2b, 0x5b, 0xca, 0xe6, 0x56, 0xbd, 0x52, 0xdd, 0x5c, 0x93, 0xc7, - 0x90, 0x0c, 0x53, 0x78, 0xb5, 0xbe, 0x83, 0x37, 0x95, 0x55, 0x8c, 0xb7, 0xb0, 0x2c, 0xa1, 0x2c, - 0x4c, 0x6e, 0xe3, 0xd5, 0x27, 0xab, 0x9b, 0x75, 0x39, 0x21, 0xac, 0xe7, 0x67, 0x61, 0xb6, 0xab, - 0xf0, 0x38, 0xed, 0xe7, 0x1a, 0x4c, 0xed, 0x59, 0x1d, 0x53, 0x57, 0xf8, 0xaa, 0x51, 0xac, 0x8e, - 0xb3, 0x8c, 0xc6, 0xcb, 0x2b, 0x7e, 0x33, 0x01, 0xe7, 0x30, 0x71, 0xac, 0xe6, 0x0b, 0x12, 0x7f, - 0x0f, 0x6e, 0x81, 0xd8, 0x0a, 0x53, 0x7e, 0x92, 0x8e, 0xcc, 0x70, 0x19, 0x7c, 0x36, 0x98, 0x70, - 0x5c, 0xd5, 0xed, 0x38, 0xa2, 0x2b, 0xaf, 0x0f, 0x1e, 0x0b, 0x35, 0x96, 0x17, 0x0b, 0x9e, 0xd0, - 0xa2, 0x36, 0xd5, 0xbb, 0xa8, 0x2d, 0xfe, 0x34, 0x9c, 0x3f, 0xa5, 0x88, 0x38, 0x27, 0xed, 0xbf, - 0x4b, 0xc0, 0xa5, 0x6e, 0xf1, 0x71, 0x9f, 0xb7, 0xfc, 0xef, 0x50, 0x36, 0xaa, 0x40, 0xae, 0x65, - 0x98, 0x4a, 0x80, 0xb1, 0x5e, 0x61, 0x72, 0x9e, 0xa2, 0xeb, 0xb5, 0x6e, 0x98, 0x45, 0xd7, 0x71, - 0x51, 0x7a, 0x8d, 0xb3, 0xef, 0xbe, 0x23, 0xc1, 0x54, 0xdc, 0xab, 0xed, 0xb3, 0x1d, 0xfa, 0x8a, - 0x36, 0xd7, 0x21, 0xf7, 0x09, 0x2c, 0xcf, 0x7f, 0x4f, 0x02, 0x54, 0xb7, 0x3b, 0xa6, 0xa6, 0xba, - 0xe4, 0xb1, 0xd5, 0x88, 0xb3, 0xb1, 0xe7, 0x60, 0xdc, 0x30, 0x75, 0x72, 0xc8, 0x1a, 0x9b, 0xc2, - 0xfc, 0x03, 0xdd, 0x87, 0xb4, 0x08, 0xda, 0xe1, 0x87, 0xd8, 0xc9, 0xd2, 0x85, 0x93, 0xe3, 0xf9, - 0x49, 0x1e, 0xa2, 0xb3, 0xf2, 0x71, 0xf0, 0x17, 0x4f, 0xf2, 0x28, 0x1d, 0xef, 0x98, 0xff, 0x7d, - 0x98, 0xed, 0xaa, 0x68, 0x9c, 0x5a, 0xf8, 0x7e, 0x02, 0x66, 0x45, 0x73, 0x62, 0xdf, 0x9e, 0x38, - 0x53, 0xc4, 0x17, 0xfa, 0x22, 0x40, 0xdb, 0x26, 0x2f, 0x14, 0xce, 0x9a, 0x1c, 0x89, 0x35, 0x43, - 0x39, 0x18, 0x01, 0x7d, 0x15, 0xa6, 0xe9, 0x80, 0x6b, 0xdb, 0x56, 0xdb, 0x72, 0x28, 0x92, 0x70, - 0x46, 0x43, 0xd2, 0x33, 0x27, 0xc7, 0xf3, 0xb9, 0x0d, 0xc3, 0xdc, 0x16, 0x8c, 0xf5, 0x1a, 0xa6, - 0x23, 0xd7, 0xff, 0xf4, 0xe0, 0xcf, 0xdf, 0x4a, 0x70, 0xee, 0x13, 0xdb, 0xd0, 0xf9, 0x9f, 0xd0, - 0x98, 0x3f, 0x1f, 0xc8, 0xec, 0xb3, 0x6a, 0xee, 0x59, 0xf1, 0x6f, 0xb3, 0x7d, 0x47, 0x82, 0x99, - 0x90, 0xf8, 0x38, 0x67, 0xfd, 0xb3, 0xc5, 0x15, 0x7e, 0x8d, 0xe2, 0x80, 0xb0, 0xd9, 0xc7, 0x39, - 0xa8, 0x7e, 0x29, 0x01, 0x17, 0xca, 0xfc, 0xf0, 0x85, 0x9d, 0x31, 0x39, 0x9d, 0x56, 0x9c, 0x56, - 0x52, 0x80, 0xc9, 0x17, 0xc4, 0x76, 0x0c, 0x8b, 0xcf, 0x7b, 0x39, 0xec, 0x7d, 0xa2, 0x9f, 0x81, - 0xac, 0x26, 0x0a, 0xf4, 0xbc, 0xcc, 0x54, 0xa9, 0x4a, 0x05, 0x9c, 0x11, 0xfd, 0x9e, 0x1c, 0xcf, - 0x83, 0xd7, 0x84, 0xea, 0x0a, 0x06, 0x4f, 0x7a, 0x55, 0x67, 0xb1, 0x94, 0xa6, 0xda, 0x76, 0xf6, - 0x2d, 0x6f, 0x9f, 0xda, 0xff, 0x16, 0x9d, 0xfe, 0x75, 0xb8, 0xd8, 0xa3, 0x85, 0x38, 0xd5, 0xfc, - 0xa3, 0x49, 0xc8, 0xad, 0x1e, 0xb6, 0x2d, 0xdb, 0xad, 0xf1, 0xc9, 0x1e, 0xad, 0x40, 0xba, 0x6d, - 0x5b, 0x2f, 0x0c, 0x4f, 0x70, 0x3e, 0xf2, 0xdc, 0xa2, 0x8b, 0x67, 0x5b, 0xe4, 0xc7, 0x3e, 0x27, - 0xc2, 0x90, 0x79, 0x6c, 0x69, 0x6a, 0xf3, 0xa1, 0xd1, 0xf4, 0xac, 0x6a, 0x71, 0x98, 0x98, 0x45, - 0x9f, 0x63, 0x5b, 0x75, 0xf7, 0xbd, 0x41, 0xe6, 0x13, 0xd1, 0x1a, 0xa4, 0x2b, 0xae, 0xdb, 0xa6, - 0x89, 0x62, 0x84, 0xde, 0x18, 0x2a, 0x92, 0x32, 0x08, 0x49, 0x3e, 0x33, 0xc2, 0x30, 0xb3, 0x66, - 0x59, 0x8d, 0x26, 0x29, 0x37, 0xad, 0x8e, 0x5e, 0xb6, 0xcc, 0x3d, 0xa3, 0x21, 0x3c, 0xdc, 0xf5, - 0xa1, 0x12, 0xd7, 0xca, 0x35, 0xdc, 0xcb, 0x8e, 0xbe, 0x0c, 0xe9, 0xda, 0x03, 0x21, 0x8a, 0xe3, - 0x93, 0xd7, 0x87, 0x8a, 0xaa, 0x3d, 0xc0, 0x3e, 0x13, 0xaa, 0x40, 0x76, 0xf9, 0x65, 0xc7, 0x26, - 0x42, 0xc6, 0x04, 0x93, 0x71, 0x73, 0xa8, 0x0c, 0xc6, 0x83, 0xc3, 0xac, 0x73, 0x77, 0x20, 0xd7, - 0xa5, 0x49, 0x84, 0x20, 0xd5, 0xa6, 0x4a, 0xa3, 0xdd, 0x99, 0xc1, 0xec, 0x3f, 0x37, 0xaf, 0xb9, - 0x9b, 0x90, 0xa2, 0x5a, 0xa1, 0xc3, 0x61, 0x57, 0x75, 0xc8, 0x8e, 0x6d, 0x88, 0x4c, 0xde, 0xa7, - 0xc8, 0xf7, 0x57, 0x12, 0x24, 0x6a, 0x0f, 0x28, 0x42, 0xdb, 0xed, 0x68, 0xcf, 0x88, 0x2b, 0x72, - 0x89, 0x2f, 0x86, 0xdc, 0x6c, 0xb2, 0x67, 0xf0, 0xd9, 0x3a, 0x83, 0xc5, 0x17, 0x7a, 0x0d, 0x40, - 0xd5, 0x34, 0xe2, 0x38, 0x8a, 0x17, 0x6f, 0x9c, 0xc1, 0x19, 0x4e, 0x59, 0x27, 0x47, 0x94, 0xcd, - 0x21, 0x9a, 0x4d, 0xb8, 0xf1, 0x67, 0xb0, 0xf8, 0xa2, 0x6c, 0x2e, 0x69, 0xb5, 0x15, 0xd7, 0x7a, - 0x46, 0x4c, 0xa6, 0xcd, 0x0c, 0xce, 0x50, 0x4a, 0x9d, 0x12, 0xe8, 0xa8, 0x21, 0xa6, 0xde, 0xb6, - 0x0c, 0xd3, 0x65, 0x6a, 0xca, 0x60, 0xff, 0x9b, 0x8a, 0xb4, 0x49, 0xc3, 0x10, 0x91, 0xb8, 0x19, - 0x2c, 0xbe, 0x44, 0x33, 0xb6, 0x20, 0xb9, 0x56, 0xae, 0xbd, 0x72, 0x33, 0x10, 0xa4, 0xd4, 0x8e, - 0x30, 0xba, 0x0c, 0x66, 0xff, 0x85, 0xc0, 0x6f, 0x4a, 0x30, 0xce, 0x54, 0x8f, 0xae, 0x40, 0x46, - 0xb3, 0x4c, 0x57, 0x35, 0x4c, 0x31, 0x6e, 0x32, 0x38, 0x20, 0xf4, 0x95, 0x7c, 0x0d, 0xa6, 0x54, - 0x4d, 0xb3, 0x3a, 0xa6, 0xab, 0x98, 0x6a, 0x8b, 0x88, 0x12, 0xb2, 0x82, 0xb6, 0xa9, 0xb6, 0x08, - 0x9a, 0x07, 0xef, 0xd3, 0x8f, 0xca, 0xce, 0x60, 0x10, 0x24, 0xff, 0xf8, 0x48, 0xb8, 0x8b, 0x3f, - 0x90, 0x60, 0xe6, 0x3d, 0xdb, 0x70, 0x49, 0x89, 0xc7, 0x0a, 0xc4, 0xe7, 0x30, 0xdf, 0x86, 0x8c, - 0xae, 0xba, 0x2a, 0x8f, 0xc0, 0x4e, 0x0c, 0x8c, 0xc0, 0xf6, 0xc6, 0x1b, 0xcd, 0xcf, 0xa2, 0xb0, - 0x11, 0xa4, 0xe8, 0x7f, 0xee, 0x4b, 0x31, 0xfb, 0x1f, 0x1c, 0x1d, 0x85, 0xab, 0x1b, 0xa7, 0x67, - 0xfb, 0xa7, 0x84, 0xe7, 0xd9, 0xe2, 0x54, 0xc3, 0x57, 0x60, 0x52, 0xac, 0x8a, 0x84, 0x12, 0x16, - 0x86, 0x8d, 0x50, 0xef, 0xc8, 0x43, 0xb0, 0xa1, 0x12, 0x80, 0xe3, 0xaa, 0xb6, 0xcb, 0xd6, 0x33, - 0x23, 0x9d, 0x31, 0x7b, 0x9e, 0x90, 0xb1, 0x51, 0x2a, 0xda, 0x84, 0x6c, 0xeb, 0x85, 0xa6, 0x29, - 0x7b, 0x46, 0xd3, 0x15, 0xc7, 0xcb, 0xd1, 0xc1, 0x34, 0x1b, 0x4f, 0xca, 0xe5, 0x87, 0x2c, 0x13, - 0x3f, 0xe5, 0x0d, 0xbe, 0x31, 0x50, 0x09, 0xfc, 0x3f, 0xfa, 0x0c, 0x88, 0x28, 0x3b, 0xc5, 0x71, - 0x5c, 0x36, 0xe0, 0xd2, 0xa5, 0xdc, 0xc9, 0xf1, 0x7c, 0x06, 0x33, 0x6a, 0xad, 0x56, 0xc7, 0x19, - 0x9e, 0xa1, 0xe6, 0x78, 0x33, 0xd3, 0xb7, 0x25, 0xc8, 0x95, 0x3a, 0xcd, 0x67, 0x5b, 0xed, 0x5a, - 0xa7, 0xd5, 0x52, 0xed, 0x23, 0x74, 0xd9, 0x33, 0x11, 0xe3, 0x25, 0x61, 0x2a, 0x4e, 0x0a, 0x1b, - 0x30, 0x5e, 0x12, 0x6a, 0x03, 0x22, 0xa0, 0x86, 0xd2, 0x79, 0xb4, 0xcc, 0xeb, 0x90, 0x63, 0xb0, - 0x5e, 0x21, 0xa6, 0x6b, 0x1b, 0x84, 0xaf, 0x1a, 0x93, 0x78, 0x8a, 0x11, 0x57, 0x39, 0x0d, 0xdd, - 0x80, 0xbc, 0x73, 0xe4, 0xb8, 0xa4, 0xa5, 0xf0, 0x2b, 0x17, 0x1c, 0x8b, 0x26, 0x71, 0x8e, 0x53, - 0x31, 0x27, 0x16, 0xff, 0x28, 0x09, 0x79, 0xaf, 0xbb, 0xe3, 0x84, 0x46, 0x25, 0x18, 0xdf, 0x33, - 0x9a, 0xc4, 0x8b, 0x04, 0xea, 0xef, 0x90, 0x3d, 0x49, 0x8b, 0xd4, 0xed, 0x7a, 0x40, 0x89, 0xb1, - 0xc6, 0xd1, 0xe5, 0x73, 0x1f, 0x49, 0x90, 0x62, 0xb3, 0xe0, 0x7d, 0x48, 0xb1, 0x31, 0x28, 0x8d, - 0x32, 0x06, 0x59, 0x56, 0xdf, 0xff, 0x27, 0x02, 0xff, 0xcf, 0x7c, 0xef, 0xbe, 0xfa, 0xe6, 0xfd, - 0x25, 0xd6, 0xdd, 0x53, 0x58, 0x7c, 0xa1, 0x12, 0xa4, 0x09, 0x6b, 0x0f, 0xd1, 0xc5, 0x1c, 0x14, - 0x65, 0xe1, 0x5d, 0x1d, 0xef, 0x8d, 0x77, 0x8f, 0x0f, 0x5d, 0x82, 0x24, 0xb5, 0xa3, 0x49, 0x7e, - 0x56, 0x72, 0x72, 0x3c, 0x9f, 0xa4, 0x16, 0x44, 0x69, 0xfc, 0xc4, 0xfd, 0x51, 0x2a, 0x9d, 0x92, - 0xc7, 0x8b, 0x7f, 0x9a, 0x82, 0x5c, 0xb5, 0x15, 0xf7, 0x08, 0x5d, 0xee, 0xee, 0xb0, 0x28, 0x88, - 0xd0, 0x55, 0x68, 0x44, 0x7f, 0x75, 0xf9, 0xba, 0xe4, 0xab, 0xf9, 0xba, 0x2a, 0x9d, 0x80, 0xc4, - 0x7d, 0x13, 0x5a, 0xfe, 0x1b, 0x43, 0xcb, 0xaf, 0xab, 0xbb, 0x4d, 0x82, 0x29, 0x8f, 0x77, 0x14, - 0xc1, 0x05, 0xa0, 0x2f, 0xb1, 0x79, 0x8e, 0x1b, 0xcd, 0xc4, 0xe8, 0x46, 0x33, 0x49, 0x4c, 0x9d, - 0x99, 0xcc, 0xa1, 0xb0, 0x98, 0xcf, 0x43, 0x52, 0x37, 0x06, 0xa9, 0x34, 0xca, 0x5f, 0x51, 0x96, - 0x21, 0x86, 0x93, 0x0a, 0x1b, 0x8e, 0x7f, 0x5c, 0x91, 0x94, 0x53, 0x73, 0x5b, 0x00, 0x41, 0xab, - 0xd0, 0x02, 0x4c, 0x58, 0x4d, 0x9d, 0x82, 0x69, 0x5a, 0x85, 0x5c, 0x29, 0x73, 0x72, 0x3c, 0x3f, - 0xbe, 0xd5, 0xd4, 0xab, 0x2b, 0x78, 0xdc, 0x6a, 0xea, 0x55, 0x9d, 0x5d, 0xd3, 0x21, 0x07, 0x0a, - 0xbb, 0x19, 0xc5, 0xc2, 0x24, 0xf0, 0xa4, 0x49, 0x0e, 0x56, 0x88, 0xa3, 0x85, 0x67, 0x37, 0x61, - 0x36, 0xbf, 0x2b, 0x41, 0xde, 0xd3, 0x60, 0xbc, 0x23, 0x3d, 0x6d, 0xb4, 0x84, 0xe5, 0x27, 0x5f, - 0xcd, 0xf2, 0x3d, 0x3e, 0x11, 0x82, 0xfc, 0x2d, 0x09, 0x66, 0x79, 0x2c, 0x87, 0xa6, 0xba, 0xd4, - 0xd7, 0xc6, 0x68, 0xde, 0x77, 0x40, 0xb6, 0x55, 0x53, 0xb7, 0x5a, 0xc6, 0x4b, 0xc2, 0x57, 0xab, - 0x8e, 0xd8, 0xa4, 0x9d, 0xf6, 0xe9, 0x6c, 0x39, 0xe6, 0x2d, 0xb6, 0xff, 0x5d, 0x82, 0x73, 0xdd, - 0x95, 0x89, 0x53, 0x69, 0xeb, 0x30, 0xc1, 0x36, 0x5a, 0xbc, 0xe1, 0xf6, 0xd9, 0x08, 0x21, 0x51, - 0xa5, 0xf3, 0x5b, 0x55, 0xbe, 0xc1, 0x33, 0x11, 0x73, 0x5f, 0x81, 0x71, 0x46, 0x3e, 0x83, 0x8f, - 0x13, 0x9a, 0x7f, 0x0e, 0x33, 0xcb, 0xba, 0x5e, 0xab, 0x09, 0xeb, 0x8b, 0x4f, 0xed, 0x1e, 0x84, - 0x49, 0x44, 0x41, 0x98, 0x70, 0x91, 0x71, 0x42, 0x98, 0x36, 0xe4, 0x45, 0x88, 0x55, 0xcc, 0x3b, - 0x6b, 0x07, 0x14, 0x73, 0x09, 0xb3, 0xe1, 0x1f, 0xc1, 0xcd, 0x0e, 0xbf, 0xc4, 0x38, 0x5b, 0xd2, - 0x81, 0x59, 0x4f, 0x6e, 0xdc, 0x9b, 0xd8, 0x83, 0x9a, 0xc3, 0x76, 0x28, 0xc2, 0xc5, 0xc6, 0xd9, - 0xa6, 0x3f, 0x94, 0x60, 0x6e, 0x8d, 0xb8, 0x35, 0xb1, 0x60, 0x7f, 0x68, 0xd9, 0xb1, 0xef, 0xf8, - 0xae, 0x01, 0x34, 0xc9, 0x9e, 0x08, 0xc3, 0x17, 0x80, 0x73, 0xf4, 0x3b, 0xa3, 0x19, 0xca, 0xcb, - 0x92, 0x84, 0x3a, 0x7e, 0x39, 0x01, 0x97, 0x23, 0x6b, 0x1c, 0xa7, 0x47, 0x88, 0x18, 0x29, 0xe8, - 0xeb, 0xc0, 0xd0, 0xa6, 0xe2, 0xb8, 0xaa, 0xeb, 0x08, 0xe7, 0xfa, 0x99, 0x57, 0x09, 0x75, 0x2d, - 0xcd, 0x88, 0xb8, 0x8d, 0x8c, 0x4f, 0xc2, 0x19, 0x2a, 0x92, 0xfd, 0x45, 0x8b, 0x30, 0xcb, 0x5c, - 0xa1, 0xa2, 0xb6, 0xdb, 0x4d, 0x83, 0xe8, 0x0a, 0xdf, 0x3a, 0x4e, 0xb1, 0xad, 0xe3, 0x19, 0x96, - 0xb4, 0xcc, 0x53, 0xaa, 0x34, 0xa1, 0xf8, 0x01, 0xcc, 0x30, 0xbd, 0xc4, 0x1d, 0x53, 0x2c, 0xf4, - 0xfc, 0xe7, 0x12, 0xa0, 0xb0, 0xfc, 0x38, 0xd5, 0xdb, 0xad, 0xca, 0x44, 0xdc, 0xaa, 0x2c, 0xfe, - 0xfd, 0x45, 0x98, 0x12, 0xad, 0xdc, 0x31, 0x0d, 0xcb, 0x44, 0xf7, 0x21, 0xd9, 0x10, 0x2b, 0xe5, - 0x6c, 0xe4, 0x1a, 0x23, 0xb8, 0x50, 0x5a, 0x19, 0xc3, 0x34, 0x2f, 0x65, 0x69, 0x77, 0xdc, 0x88, - 0x50, 0x98, 0x20, 0x0c, 0x22, 0xcc, 0xd2, 0xee, 0xb8, 0xa8, 0x06, 0xd3, 0x5a, 0x70, 0x0d, 0x4f, - 0xa1, 0xec, 0xc9, 0xbe, 0x41, 0xb3, 0x91, 0xd7, 0x12, 0x2b, 0x63, 0x38, 0xaf, 0x75, 0x25, 0xa0, - 0x72, 0xf8, 0xde, 0x57, 0xaa, 0xef, 0xa6, 0xcc, 0xe9, 0x3b, 0x67, 0x95, 0xb1, 0xd0, 0xf5, 0x30, - 0xf4, 0x36, 0x4c, 0xe8, 0xec, 0x3e, 0x91, 0xd8, 0xd6, 0x89, 0x32, 0x8b, 0xae, 0x2b, 0x5c, 0x95, - 0x31, 0x2c, 0x38, 0xd0, 0x23, 0x98, 0xe2, 0xff, 0xc4, 0x08, 0x9e, 0xe8, 0xbb, 0x6b, 0xd5, 0x7b, - 0xa3, 0xaa, 0x32, 0x86, 0xb3, 0x7a, 0x40, 0x45, 0x9f, 0x83, 0x94, 0xa3, 0xa9, 0x7c, 0x5f, 0x23, - 0xfa, 0x48, 0x3f, 0x74, 0x65, 0xa3, 0x42, 0xa7, 0x45, 0x4d, 0x35, 0xd1, 0x53, 0x98, 0xd9, 0x25, - 0x0d, 0xc3, 0x54, 0xdc, 0xe0, 0xd8, 0x8d, 0x05, 0xf0, 0x76, 0x9f, 0xf4, 0xf9, 0xe8, 0x26, 0x3a, - 0x6e, 0xbc, 0x32, 0x86, 0xe5, 0xdd, 0x53, 0x49, 0xb4, 0xcb, 0x18, 0x3c, 0x0d, 0x09, 0xce, 0xf4, - 0xed, 0xb2, 0xc8, 0x48, 0x6e, 0xda, 0x65, 0xa4, 0x2b, 0x01, 0xad, 0x41, 0x56, 0xa5, 0x70, 0x41, - 0x61, 0xd1, 0xb1, 0x05, 0xe8, 0xbb, 0x29, 0xd7, 0x13, 0xb0, 0x5b, 0x61, 0x41, 0xf0, 0x1e, 0x31, - 0x10, 0xd4, 0xa2, 0x2e, 0xae, 0x90, 0x1d, 0x2c, 0x28, 0xec, 0xb9, 0x7d, 0x41, 0x8c, 0x88, 0x36, - 0x20, 0xb7, 0xef, 0xc5, 0xe7, 0xb1, 0x73, 0xd2, 0xa9, 0xbe, 0x3b, 0x73, 0x11, 0xf1, 0x85, 0x95, - 0x31, 0x3c, 0xb5, 0x1f, 0x22, 0xa3, 0x45, 0x48, 0x34, 0xb4, 0x42, 0x8e, 0xc9, 0xb8, 0x32, 0x28, - 0x7a, 0xae, 0x32, 0x86, 0x13, 0x0d, 0x8d, 0x2e, 0x02, 0x78, 0x60, 0xd1, 0xa1, 0x59, 0xc8, 0xf7, - 0x75, 0x1b, 0xdd, 0xc1, 0x5d, 0x95, 0x31, 0xcc, 0x42, 0xa0, 0x68, 0x79, 0xdb, 0x90, 0xb7, 0xf9, - 0x61, 0xa7, 0x77, 0xa4, 0x2f, 0x33, 0x29, 0xb7, 0xa2, 0x9d, 0x4f, 0xcf, 0xa9, 0x7e, 0x65, 0x0c, - 0xe7, 0xec, 0x30, 0x1d, 0x7d, 0x03, 0xce, 0x75, 0x4b, 0x14, 0xc6, 0x3d, 0xd3, 0xe3, 0x8b, 0xa2, - 0xe5, 0x76, 0xdb, 0x38, 0xb2, 0x7b, 0x12, 0xd1, 0x5b, 0x30, 0xce, 0x7b, 0x0d, 0x31, 0x91, 0xf3, - 0x51, 0x1b, 0x1b, 0xdd, 0x1d, 0xc6, 0xf3, 0xd3, 0xf1, 0xe6, 0x8a, 0x53, 0x3e, 0xa5, 0x69, 0x35, - 0x0a, 0xb3, 0x7d, 0xc7, 0x5b, 0xef, 0xa9, 0x25, 0x1d, 0x6f, 0x6e, 0x40, 0xa5, 0xfd, 0x6e, 0xf3, - 0x14, 0x71, 0x28, 0x74, 0xae, 0x6f, 0xbf, 0x47, 0x1c, 0xfe, 0xd1, 0x7e, 0xb7, 0x43, 0x64, 0x5a, - 0x35, 0x9b, 0x5f, 0x56, 0x52, 0xd8, 0x30, 0x3e, 0xdf, 0xb7, 0x6a, 0xbd, 0x17, 0xb0, 0x68, 0xd5, - 0xec, 0x80, 0x8a, 0x9e, 0x80, 0x2c, 0xee, 0xa5, 0x28, 0xde, 0x61, 0x42, 0xe1, 0x02, 0x93, 0x77, - 0x27, 0xd2, 0x5b, 0x46, 0x9d, 0xa2, 0x54, 0xc6, 0xf0, 0xb4, 0xd6, 0x9d, 0x42, 0x9d, 0x05, 0x93, - 0xa7, 0x68, 0xc1, 0x85, 0x9e, 0x42, 0xa1, 0xaf, 0xb3, 0xe8, 0x73, 0xfb, 0x88, 0x3a, 0x0b, 0xed, - 0x54, 0x12, 0x35, 0x63, 0xc3, 0x34, 0x5c, 0xe6, 0xd8, 0xe7, 0xfa, 0x9a, 0x71, 0xf7, 0x4d, 0x69, - 0x6a, 0xc6, 0x06, 0xa7, 0x50, 0x33, 0x76, 0xc5, 0x89, 0xa1, 0xe8, 0x8e, 0x2b, 0x7d, 0xcd, 0x38, - 0xea, 0x68, 0x91, 0x9a, 0xb1, 0x1b, 0xa6, 0x53, 0x33, 0xe6, 0x0e, 0xe2, 0x94, 0xdc, 0xd7, 0xfa, - 0x9a, 0x71, 0xdf, 0x40, 0x74, 0x6a, 0xc6, 0x6a, 0x4f, 0x22, 0x5a, 0xa1, 0xe8, 0x8d, 0xa2, 0x12, - 0xc3, 0xdc, 0xb3, 0x0a, 0x57, 0xfb, 0xce, 0x3f, 0xa7, 0xcf, 0x0c, 0x2b, 0x0c, 0xba, 0x09, 0x1a, - 0x75, 0x64, 0x0c, 0xd2, 0x2a, 0xec, 0xfa, 0x57, 0x61, 0xbe, 0xaf, 0x23, 0xeb, 0xd9, 0xf7, 0xa5, - 0x8e, 0xec, 0xc0, 0x27, 0xd2, 0x89, 0x8c, 0xef, 0xd0, 0x14, 0x16, 0x86, 0xec, 0x04, 0x84, 0x26, - 0x32, 0xce, 0x81, 0x96, 0x21, 0xf3, 0xbc, 0x43, 0xec, 0x23, 0xe6, 0x86, 0xae, 0xf5, 0xc5, 0xa1, - 0xa7, 0x82, 0x00, 0x2b, 0x63, 0x38, 0xfd, 0x5c, 0x90, 0x68, 0xf1, 0x7c, 0x99, 0x5c, 0x28, 0xf6, - 0x2d, 0xbe, 0x6b, 0x63, 0x84, 0x16, 0xcf, 0x39, 0x90, 0x06, 0xe7, 0x79, 0x5f, 0x89, 0xc8, 0x75, - 0x5b, 0x84, 0x88, 0x17, 0x5e, 0x67, 0xa2, 0xfa, 0x2e, 0x3a, 0x23, 0xa3, 0xe8, 0x2b, 0x63, 0x78, - 0x56, 0xed, 0x4d, 0xa5, 0x03, 0x5e, 0x4c, 0x3d, 0x7c, 0xa9, 0x5a, 0xb8, 0xde, 0x77, 0xc0, 0x47, - 0x2c, 0xee, 0xe9, 0x80, 0x57, 0x43, 0x64, 0x3e, 0x01, 0xe9, 0x8a, 0xe3, 0xb8, 0x74, 0x61, 0x58, - 0xb8, 0x31, 0x60, 0x02, 0x3a, 0xb5, 0x60, 0xe5, 0x13, 0x90, 0x5e, 0xe3, 0x9c, 0x54, 0x90, 0xd6, - 0x24, 0xaa, 0x2d, 0xdc, 0xec, 0xcd, 0xbe, 0x82, 0x7a, 0x6e, 0x1f, 0x53, 0x41, 0x9a, 0x4f, 0xa4, - 0x13, 0xb6, 0xed, 0x5d, 0x68, 0x13, 0xf8, 0xf1, 0x56, 0xdf, 0x09, 0x3b, 0xf2, 0xde, 0x1d, 0x9d, - 0xb0, 0xed, 0xae, 0x04, 0xf4, 0x45, 0x98, 0x14, 0xb7, 0x83, 0x0a, 0xb7, 0x07, 0xa0, 0xda, 0xf0, - 0x2a, 0x96, 0x8e, 0x6b, 0xc1, 0xc3, 0xbd, 0x2c, 0xbf, 0x5c, 0xc4, 0x9b, 0x77, 0x67, 0x80, 0x97, - 0xed, 0x59, 0x40, 0x72, 0x2f, 0x1b, 0x90, 0xa9, 0x97, 0xe5, 0x76, 0x2a, 0xe6, 0xba, 0xbb, 0x7d, - 0xbd, 0x6c, 0x6f, 0xf8, 0x1e, 0xf5, 0xb2, 0xcf, 0x03, 0x2a, 0xd2, 0xe1, 0x42, 0x83, 0xb8, 0x8a, - 0x77, 0x20, 0xcb, 0x22, 0x48, 0xf9, 0xb4, 0xf4, 0x46, 0x5f, 0xab, 0xeb, 0xbf, 0x1e, 0xa4, 0x56, - 0xd7, 0xe8, 0x4d, 0xa5, 0xbd, 0xcb, 0x23, 0x5a, 0x78, 0x87, 0x7c, 0xa6, 0x6f, 0xef, 0xf6, 0x2c, - 0x58, 0x2a, 0xde, 0x53, 0x34, 0x1c, 0xd6, 0x4f, 0x8a, 0xe8, 0xa0, 0x47, 0xa9, 0xf4, 0xb4, 0x2c, - 0x3f, 0x4a, 0xa5, 0x2f, 0xca, 0x85, 0x47, 0xa9, 0xf4, 0x25, 0x79, 0xee, 0x51, 0x2a, 0x7d, 0x59, - 0xbe, 0x52, 0xfc, 0x93, 0x8b, 0x90, 0xf3, 0x96, 0x17, 0x1c, 0xda, 0x2f, 0x85, 0xa1, 0xfd, 0xd5, - 0x7e, 0xd0, 0x5e, 0x2c, 0x48, 0x04, 0xb6, 0x5f, 0x0a, 0x63, 0xfb, 0xab, 0xfd, 0xb0, 0x7d, 0xc0, - 0x43, 0xc1, 0x7d, 0xbd, 0x1f, 0xb8, 0xbf, 0x33, 0x02, 0xb8, 0xf7, 0x45, 0x9d, 0x46, 0xf7, 0x2b, - 0xbd, 0xe8, 0xfe, 0xfa, 0x60, 0x74, 0xef, 0x8b, 0x0a, 0xc1, 0xfb, 0x77, 0x4e, 0xc1, 0xfb, 0x6b, - 0x03, 0xe0, 0xbd, 0xcf, 0xef, 0xe1, 0xfb, 0xf5, 0x48, 0x7c, 0x7f, 0x73, 0x18, 0xbe, 0xf7, 0xe5, - 0x74, 0x01, 0xfc, 0x37, 0xbb, 0x00, 0xfe, 0x7c, 0x5f, 0x80, 0xef, 0x73, 0x73, 0x84, 0xff, 0x7e, - 0x7f, 0x84, 0xff, 0xc6, 0x48, 0x08, 0xdf, 0x97, 0xd7, 0x0b, 0xf1, 0xeb, 0xfd, 0x20, 0xfe, 0x9d, - 0x11, 0x20, 0x7e, 0xd0, 0x71, 0xa7, 0x30, 0x7e, 0x25, 0x0a, 0xe3, 0xdf, 0x18, 0x82, 0xf1, 0x7d, - 0x69, 0x61, 0x90, 0x5f, 0x89, 0x02, 0xf9, 0x37, 0x86, 0x80, 0xfc, 0x53, 0x92, 0xf8, 0x30, 0xdc, - 0x8c, 0x46, 0xf9, 0xb7, 0x86, 0xa2, 0x7c, 0x5f, 0x5a, 0x37, 0xcc, 0xbf, 0x17, 0x82, 0xf9, 0xaf, - 0xf5, 0x81, 0xf9, 0x3e, 0x2b, 0xc5, 0xf9, 0x5f, 0xee, 0xc1, 0xf9, 0xc5, 0x41, 0x38, 0xdf, 0xe7, - 0xf5, 0x81, 0xfe, 0xbb, 0x7d, 0x80, 0xfe, 0xed, 0xe1, 0x40, 0xdf, 0x17, 0x76, 0x0a, 0xe9, 0xab, - 0x03, 0x91, 0xfe, 0x67, 0x47, 0x44, 0xfa, 0xbe, 0xf4, 0x28, 0xa8, 0xff, 0xf9, 0x6e, 0xa8, 0xbf, - 0xd0, 0x1f, 0xea, 0xfb, 0x62, 0x04, 0xd6, 0x5f, 0x8f, 0xc4, 0xfa, 0x37, 0x87, 0x61, 0xfd, 0x60, - 0xec, 0x85, 0xc1, 0xfe, 0x66, 0x34, 0xd8, 0xbf, 0x35, 0x14, 0xec, 0x07, 0xdd, 0xdf, 0x85, 0xf6, - 0xd7, 0x23, 0xd1, 0xfe, 0xcd, 0x61, 0x68, 0x3f, 0xa8, 0x5c, 0x18, 0xee, 0xbf, 0xd7, 0x17, 0xee, - 0xdf, 0x1d, 0x05, 0xee, 0xfb, 0x42, 0x7b, 0xf0, 0xfe, 0xfb, 0xfd, 0xf1, 0xfe, 0x1b, 0x23, 0xe1, - 0xfd, 0xc0, 0x75, 0xf4, 0x00, 0xfe, 0x2f, 0xf7, 0x00, 0xfe, 0xe2, 0x20, 0xc0, 0x1f, 0xd8, 0xb3, - 0x87, 0xf8, 0xd5, 0x81, 0xf8, 0xfc, 0xb3, 0x23, 0xe2, 0xf3, 0xc0, 0xf8, 0x22, 0x00, 0xfa, 0x6a, - 0x04, 0x40, 0xbf, 0x3e, 0x18, 0xa0, 0x07, 0x53, 0x48, 0x80, 0xd0, 0x2b, 0x51, 0x08, 0xfd, 0xc6, - 0x10, 0x84, 0x1e, 0x78, 0xa1, 0x10, 0x44, 0x7f, 0xe7, 0x14, 0x44, 0xbf, 0x36, 0xf4, 0xb4, 0x39, - 0x84, 0xd1, 0x4b, 0xbd, 0x18, 0xfd, 0xf5, 0x81, 0x18, 0xdd, 0x97, 0x10, 0x80, 0xf4, 0x77, 0x4e, - 0x81, 0xf4, 0x6b, 0x03, 0x40, 0x7a, 0x50, 0x01, 0x81, 0xd2, 0xf5, 0xc1, 0x28, 0x7d, 0x71, 0x54, - 0x94, 0xee, 0x0b, 0x8e, 0x84, 0xe9, 0x9b, 0xd1, 0x30, 0xfd, 0xd6, 0x88, 0x07, 0x4f, 0x3d, 0x38, - 0xbd, 0x12, 0x85, 0xd3, 0x6f, 0x0c, 0xc1, 0xe9, 0xe1, 0x39, 0xc4, 0x07, 0xea, 0x95, 0x28, 0xa0, - 0x7e, 0x63, 0x08, 0x50, 0x0f, 0x24, 0x85, 0x90, 0x7a, 0xbd, 0x1f, 0x52, 0xbf, 0x33, 0x02, 0x52, - 0x0f, 0xe6, 0xdd, 0x53, 0x50, 0xfd, 0x4b, 0xa7, 0xa1, 0x7a, 0x71, 0x10, 0x54, 0x0f, 0x46, 0xa4, - 0x87, 0xd5, 0x37, 0xa3, 0xb1, 0xfa, 0xad, 0xa1, 0x58, 0x3d, 0xec, 0x24, 0x43, 0x60, 0x7d, 0x3d, - 0x12, 0xac, 0xdf, 0x1c, 0x06, 0xd6, 0x03, 0x27, 0x19, 0x46, 0xeb, 0x64, 0x08, 0x5a, 0x5f, 0x1c, - 0x15, 0xad, 0x07, 0xd6, 0x17, 0x05, 0xd7, 0x2b, 0x51, 0x70, 0xfd, 0xc6, 0x10, 0xb8, 0x1e, 0xf4, - 0xf1, 0x2b, 0xe0, 0xf5, 0x47, 0xa9, 0xf4, 0x15, 0xf9, 0xb5, 0xe2, 0x5f, 0x8e, 0xc3, 0x44, 0xc5, - 0x8b, 0x6c, 0x08, 0xdd, 0x33, 0x95, 0xce, 0x72, 0xcf, 0x14, 0xad, 0x50, 0x13, 0x60, 0x03, 0x49, - 0x20, 0xf8, 0x01, 0xb7, 0xb6, 0x7b, 0xce, 0x93, 0x3c, 0xd6, 0x33, 0x44, 0xe1, 0xa3, 0x37, 0x21, - 0xd7, 0x71, 0x88, 0xad, 0xb4, 0x6d, 0xc3, 0xb2, 0x0d, 0x97, 0xc7, 0xb1, 0x49, 0x25, 0xf9, 0xe3, - 0xe3, 0xf9, 0xa9, 0x1d, 0x87, 0xd8, 0xdb, 0x82, 0x8e, 0xa7, 0x3a, 0xa1, 0x2f, 0xef, 0xad, 0xd0, - 0xf1, 0xd1, 0xdf, 0x0a, 0x7d, 0x17, 0x64, 0x9b, 0xa8, 0x7a, 0xd7, 0x94, 0xc6, 0xef, 0x63, 0x46, - 0xcf, 0xbe, 0xaa, 0x1e, 0x9a, 0xb5, 0xd8, 0xbd, 0xcc, 0x69, 0xbb, 0x9b, 0x88, 0xee, 0xc3, 0xf9, - 0x96, 0x7a, 0xc8, 0x6f, 0x1c, 0x7b, 0x28, 0x81, 0x45, 0x78, 0xa4, 0x59, 0x20, 0x12, 0x6a, 0xa9, - 0x87, 0xec, 0xe1, 0x51, 0x9e, 0xc4, 0x1e, 0x2a, 0xbb, 0x01, 0x79, 0xdd, 0x70, 0x5c, 0xc3, 0xd4, - 0xbc, 0xb7, 0x3a, 0xf8, 0xa3, 0x18, 0x39, 0x8f, 0xca, 0xdf, 0xcc, 0xb8, 0x0b, 0x33, 0x22, 0xee, - 0x2a, 0x78, 0x8a, 0x94, 0xe1, 0xe1, 0x34, 0xad, 0x05, 0x4d, 0x08, 0x5e, 0x8e, 0x2d, 0xc3, 0x74, - 0x43, 0x75, 0xc9, 0x81, 0x7a, 0xa4, 0x98, 0x96, 0xce, 0x74, 0x9f, 0x65, 0x2f, 0x16, 0x5c, 0x3e, - 0x39, 0x9e, 0xcf, 0xad, 0xf1, 0xa4, 0x4d, 0x4b, 0xe7, 0x3d, 0x30, 0xc1, 0xff, 0xe1, 0x5c, 0x23, - 0x94, 0xa0, 0xa3, 0x65, 0x98, 0x62, 0xaf, 0x30, 0x59, 0xfc, 0xa9, 0x2f, 0x81, 0x72, 0xfb, 0x1d, - 0x26, 0x88, 0x07, 0xc1, 0x30, 0x7b, 0xb9, 0xc9, 0x7b, 0x1d, 0xec, 0x16, 0x4c, 0xab, 0xce, 0x91, - 0xa9, 0x31, 0x0d, 0x13, 0xd3, 0xe9, 0x38, 0x0c, 0xe6, 0xa6, 0x71, 0x9e, 0x91, 0xcb, 0x1e, 0x55, - 0x3c, 0xf9, 0xf1, 0xeb, 0x12, 0x4c, 0x75, 0x05, 0x23, 0xbe, 0x73, 0xea, 0x14, 0xec, 0x52, 0x34, - 0xc4, 0xee, 0x17, 0xdc, 0x93, 0x16, 0x3d, 0xe0, 0x05, 0x1c, 0xcc, 0xf7, 0x87, 0x68, 0x6c, 0x91, - 0xeb, 0x85, 0x68, 0x78, 0x6c, 0x6f, 0xa7, 0x7e, 0xf3, 0xbb, 0xf3, 0x63, 0xc5, 0x1f, 0x27, 0x21, - 0xd7, 0x1d, 0x74, 0x58, 0x3d, 0x55, 0xaf, 0x28, 0xb7, 0xd6, 0xc5, 0xd1, 0xbf, 0x96, 0x2b, 0x90, - 0xf1, 0xde, 0x6e, 0xf2, 0xaa, 0xb9, 0x30, 0xe0, 0xac, 0x2f, 0x5c, 0xcf, 0x80, 0x71, 0xee, 0x87, - 0x09, 0x7f, 0xe4, 0x2f, 0xc2, 0x38, 0x7b, 0x05, 0x59, 0x54, 0x2d, 0x2a, 0x3e, 0x7f, 0x95, 0xa6, - 0x63, 0x9e, 0x8d, 0x7a, 0x8a, 0xfa, 0x99, 0x6e, 0xa4, 0xfb, 0x84, 0x33, 0xbc, 0xd2, 0x7b, 0xc6, - 0x9b, 0xd3, 0xec, 0xe4, 0xaf, 0xd9, 0x24, 0x9a, 0x2b, 0x1e, 0x53, 0xf6, 0xde, 0xe2, 0xbd, 0x7e, - 0x5a, 0x84, 0x78, 0x7a, 0x79, 0x11, 0x8b, 0xa7, 0x97, 0x43, 0x31, 0x20, 0x79, 0x5f, 0x04, 0x1b, - 0x58, 0x3c, 0x52, 0x48, 0x74, 0xf5, 0x2f, 0x48, 0x20, 0xb3, 0x61, 0xf4, 0x90, 0x10, 0x3d, 0x16, - 0x2b, 0xf4, 0xc2, 0x53, 0x12, 0x23, 0x87, 0xa7, 0x14, 0x55, 0xc8, 0xfb, 0x75, 0xe0, 0x8f, 0x92, - 0x0e, 0x78, 0x50, 0xe0, 0x4c, 0x17, 0xbd, 0x8a, 0xbf, 0x2d, 0xc1, 0xac, 0x5f, 0x06, 0xc3, 0xea, - 0x3c, 0x24, 0xfa, 0x0c, 0x01, 0x83, 0x98, 0x3d, 0xb8, 0x4c, 0xd7, 0x6a, 0xec, 0xf2, 0xcf, 0x48, - 0x16, 0x84, 0xc4, 0xb1, 0x32, 0x88, 0x35, 0xa0, 0x5e, 0xaf, 0xb1, 0xa7, 0x98, 0xf9, 0x7f, 0xa7, - 0xf8, 0x30, 0xa4, 0x01, 0x66, 0xac, 0xb4, 0x99, 0x23, 0x59, 0xb5, 0xd7, 0x4c, 0x96, 0xb9, 0xf8, - 0x23, 0x29, 0x2c, 0xe8, 0x05, 0x9d, 0xfb, 0x1f, 0x40, 0xf2, 0x85, 0xda, 0x1c, 0x74, 0xaa, 0xde, - 0xa5, 0x7a, 0x4c, 0x73, 0xa3, 0x87, 0xc0, 0x6f, 0x62, 0xf0, 0x38, 0xf2, 0x44, 0xff, 0x05, 0x5a, - 0xaf, 0x4a, 0x71, 0x88, 0x13, 0xbd, 0xe5, 0xb5, 0x22, 0x39, 0xbc, 0xf8, 0xf0, 0x20, 0x7d, 0x3b, - 0xf5, 0xe1, 0x77, 0xe7, 0xa5, 0xbb, 0x35, 0x98, 0x8d, 0x98, 0x84, 0x50, 0x1e, 0xa0, 0xbc, 0xb5, - 0x59, 0xab, 0xd6, 0xea, 0xab, 0x9b, 0x75, 0xef, 0xbd, 0xe7, 0xe5, 0x15, 0x65, 0x67, 0xb3, 0xbc, - 0xb5, 0xb1, 0x51, 0xad, 0xd7, 0x57, 0x57, 0x64, 0x09, 0xc9, 0x30, 0x55, 0xdd, 0x0c, 0xe5, 0x13, - 0xcf, 0x3c, 0xdf, 0xfd, 0xff, 0x00, 0xc1, 0x33, 0x7a, 0x54, 0xd6, 0xfa, 0xea, 0x53, 0xe5, 0xc9, - 0xf2, 0xe3, 0x9d, 0xd5, 0x9a, 0x3c, 0x86, 0x10, 0xe4, 0x4b, 0xcb, 0xf5, 0x72, 0x45, 0xc1, 0xab, - 0xb5, 0xed, 0xad, 0xcd, 0xda, 0xaa, 0x2c, 0x09, 0xbe, 0xf7, 0x20, 0x1b, 0x7a, 0xa1, 0x80, 0x66, - 0xdc, 0xde, 0xa9, 0x55, 0x94, 0x7a, 0x75, 0x63, 0xb5, 0x56, 0x5f, 0xde, 0xd8, 0x96, 0xc7, 0xa8, - 0x30, 0x46, 0x5b, 0x2e, 0x6d, 0xe1, 0xba, 0x2c, 0xf9, 0xdf, 0xf5, 0xad, 0x9d, 0x72, 0x45, 0x4e, - 0xf8, 0xdf, 0xef, 0xee, 0xac, 0xe2, 0xa7, 0x72, 0x52, 0x08, 0x56, 0xe1, 0x7c, 0xe4, 0x8d, 0x11, - 0x94, 0x85, 0xc9, 0x1d, 0x93, 0xdd, 0x92, 0x97, 0xc7, 0x50, 0x2e, 0x74, 0x69, 0x44, 0x96, 0x50, - 0x9a, 0x5f, 0x4e, 0x90, 0x13, 0x68, 0x02, 0x12, 0xb5, 0x07, 0x72, 0x12, 0x4d, 0x43, 0x36, 0x74, - 0xf3, 0x42, 0x4e, 0xa1, 0x8c, 0x08, 0xbf, 0x97, 0xc7, 0xef, 0x5e, 0x83, 0x50, 0x74, 0x33, 0x02, - 0x98, 0x78, 0xac, 0xba, 0xc4, 0x71, 0xe5, 0x31, 0x34, 0x09, 0xc9, 0xe5, 0x66, 0x53, 0x96, 0x96, - 0xbe, 0x0a, 0x69, 0xef, 0xbd, 0x2d, 0xf4, 0x18, 0xc6, 0xf9, 0x9a, 0x6b, 0xbe, 0xbf, 0x9f, 0x67, - 0xae, 0x62, 0x6e, 0x61, 0xd8, 0x44, 0x50, 0x1c, 0x2b, 0x5d, 0xfb, 0xf0, 0x5f, 0xae, 0x8e, 0x7d, - 0x78, 0x72, 0x55, 0xfa, 0xe8, 0xe4, 0xaa, 0xf4, 0x0f, 0x27, 0x57, 0xa5, 0x7f, 0x3e, 0xb9, 0x2a, - 0xfd, 0xea, 0xbf, 0x5e, 0x1d, 0x7b, 0x7f, 0x52, 0xb0, 0xec, 0x4e, 0xb0, 0xd7, 0xe0, 0x1f, 0xfc, - 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0xa9, 0x95, 0x7f, 0x27, 0x12, 0x5f, 0x00, 0x00, + 0x57, 0x65, 0x56, 0x67, 0x66, 0xb5, 0xed, 0x96, 0x10, 0x08, 0x90, 0x16, 0xad, 0x56, 0x2b, 0x24, + 0x10, 0x42, 0x02, 0xc4, 0x4a, 0xfb, 0x80, 0x84, 0xc4, 0x02, 0x02, 0x09, 0x81, 0xc4, 0xee, 0x0b, + 0x0f, 0xf3, 0xb0, 0x0f, 0x03, 0x12, 0x17, 0x21, 0x61, 0x81, 0xe1, 0x61, 0xc5, 0x2b, 0x42, 0x42, + 0x23, 0x1e, 0x50, 0x5c, 0xf2, 0x52, 0xae, 0xac, 0x4b, 0x7b, 0x73, 0xc4, 0x20, 0x9e, 0xaa, 0xf2, + 0x44, 0x9c, 0x13, 0x11, 0x27, 0x4e, 0x9c, 0xf8, 0x22, 0xe2, 0x44, 0xc0, 0x8c, 0x6d, 0xa9, 0xda, + 0x7e, 0x7b, 0xf7, 0x9e, 0xda, 0x36, 0x16, 0xdb, 0xb6, 0xe5, 0x5a, 0x68, 0x46, 0xb3, 0xb4, 0x67, + 0x8c, 0xbc, 0x28, 0x12, 0xe7, 0x90, 0x97, 0x4b, 0x57, 0x5d, 0x95, 0x67, 0x9b, 0x3b, 0xe7, 0xd1, + 0x88, 0x6d, 0x5b, 0xb6, 0x23, 0xa8, 0x17, 0x3c, 0x6a, 0x8b, 0xb8, 0x6a, 0x28, 0x77, 0xd1, 0x71, + 0x2d, 0x5b, 0x6d, 0x90, 0x7b, 0xc4, 0x6c, 0x18, 0xa6, 0xf7, 0x43, 0xf3, 0xbd, 0xd0, 0x34, 0x91, + 0xe7, 0xf5, 0x41, 0x79, 0x1e, 0x88, 0x4c, 0x85, 0x8e, 0x6b, 0x34, 0xef, 0xed, 0x37, 0xb5, 0x7b, + 0xae, 0xd1, 0x22, 0x8e, 0xab, 0xb6, 0xda, 0x22, 0x65, 0x81, 0xa5, 0xb8, 0xb6, 0xaa, 0x19, 0x66, + 0xe3, 0x9e, 0x4d, 0x34, 0xcb, 0xd6, 0x89, 0xae, 0x38, 0x6d, 0xd5, 0xf4, 0xaa, 0xdc, 0xb0, 0x1a, + 0x16, 0xfb, 0x7b, 0x8f, 0xfe, 0xe3, 0xd4, 0xe2, 0xcf, 0x41, 0x06, 0xab, 0x66, 0x83, 0x54, 0xcd, + 0x3d, 0x0b, 0x7d, 0x01, 0x52, 0x3a, 0x71, 0xb4, 0x82, 0xb4, 0x20, 0xdd, 0xce, 0x2e, 0x15, 0x17, + 0x7b, 0x74, 0xb1, 0xc8, 0xf2, 0xae, 0x10, 0x47, 0xb3, 0x8d, 0xb6, 0x6b, 0xd9, 0xa5, 0xd4, 0x87, + 0xc7, 0xf3, 0x63, 0x98, 0x71, 0xa1, 0xcf, 0xc1, 0x78, 0x93, 0xa8, 0x0e, 0x29, 0x24, 0x18, 0x7b, + 0x21, 0x82, 0xfd, 0x31, 0x4d, 0x17, 0x4c, 0x3c, 0x73, 0xf1, 0x25, 0xe4, 0x30, 0x79, 0xde, 0x21, + 0x8e, 0x5b, 0x21, 0xaa, 0x4e, 0x6c, 0x74, 0x09, 0x92, 0xcf, 0xc8, 0x51, 0x21, 0xb9, 0x20, 0xdd, + 0x9e, 0x2a, 0x4d, 0x7e, 0x7c, 0x3c, 0x9f, 0x5c, 0x27, 0x47, 0x98, 0xd2, 0xd0, 0x02, 0x4c, 0x12, + 0x53, 0x57, 0x68, 0x72, 0xaa, 0x3b, 0x79, 0x82, 0x98, 0xfa, 0x3a, 0x39, 0x42, 0x73, 0x90, 0x76, + 0xa8, 0x34, 0x53, 0x23, 0x85, 0xf1, 0x05, 0xe9, 0xf6, 0x38, 0xf6, 0xbf, 0xdf, 0x4e, 0xfd, 0xf8, + 0xbb, 0xf3, 0xd2, 0xa3, 0x54, 0x5a, 0x92, 0x13, 0x8f, 0x52, 0xe9, 0x84, 0x9c, 0x2c, 0x7e, 0x3b, + 0x09, 0x79, 0x4c, 0x9c, 0xb6, 0x65, 0x3a, 0x44, 0x94, 0xfe, 0xff, 0x20, 0xe9, 0x1e, 0x9a, 0xac, + 0xf4, 0xec, 0xd2, 0xd5, 0x88, 0x26, 0xd4, 0x6d, 0xd5, 0x74, 0x54, 0xcd, 0x35, 0x2c, 0x13, 0xd3, + 0xac, 0xe8, 0xf3, 0x90, 0xb5, 0x89, 0xd3, 0x69, 0x11, 0xa6, 0x6c, 0x56, 0xb1, 0xec, 0xd2, 0xc5, + 0x08, 0xce, 0x5a, 0x5b, 0x35, 0x31, 0xf0, 0xbc, 0xf4, 0x3f, 0xba, 0x04, 0x69, 0xb3, 0xd3, 0xa2, + 0xcd, 0x71, 0x58, 0x65, 0x93, 0x78, 0xd2, 0xec, 0xb4, 0xd6, 0xc9, 0x91, 0x83, 0xca, 0x90, 0xb5, + 0xa9, 0xaa, 0x15, 0xc3, 0xdc, 0xb3, 0x9c, 0xc2, 0xc4, 0x42, 0xf2, 0x76, 0x76, 0xe9, 0x4a, 0xbf, + 0x0e, 0xa1, 0x9d, 0x27, 0xb4, 0x0a, 0xb6, 0x47, 0x70, 0x50, 0x0d, 0x72, 0xa2, 0x66, 0x36, 0x51, + 0x1d, 0xcb, 0x2c, 0x4c, 0x2e, 0x48, 0xb7, 0xf3, 0x4b, 0x8b, 0x51, 0x62, 0xba, 0xb4, 0x40, 0x3f, + 0x3b, 0x2d, 0x82, 0x19, 0x17, 0x9e, 0xb2, 0x43, 0x5f, 0xc5, 0xa7, 0x30, 0x15, 0x4e, 0x45, 0x08, + 0xf2, 0x78, 0xb5, 0xb6, 0xb3, 0xb1, 0xaa, 0xec, 0x6c, 0xae, 0x6f, 0x6e, 0xbd, 0xb7, 0x29, 0x8f, + 0xa1, 0x73, 0x20, 0x0b, 0xda, 0xfa, 0xea, 0x53, 0xe5, 0x71, 0x75, 0xa3, 0x5a, 0x97, 0x25, 0x74, + 0x09, 0xce, 0x0b, 0x2a, 0x5e, 0xde, 0x5c, 0x5b, 0x55, 0x4a, 0x5b, 0x3b, 0x9b, 0x2b, 0xcb, 0xf8, + 0xa9, 0x9c, 0x98, 0x4b, 0xfd, 0xf2, 0xf7, 0xae, 0x8e, 0x15, 0x9f, 0x00, 0xac, 0x11, 0x57, 0x58, + 0x03, 0x2a, 0xc1, 0xc4, 0x3e, 0xab, 0x8d, 0x30, 0xc7, 0x85, 0xc8, 0x6a, 0x87, 0x2c, 0xa7, 0x94, + 0xa6, 0x1a, 0xf8, 0xe8, 0x78, 0x5e, 0xc2, 0x82, 0x93, 0x77, 0x79, 0xf1, 0x87, 0x12, 0x64, 0x99, + 0x60, 0xde, 0x46, 0x54, 0x3e, 0x25, 0xf9, 0xda, 0x50, 0x85, 0xf4, 0x8a, 0x46, 0x8b, 0x30, 0xfe, + 0x42, 0x6d, 0x76, 0x06, 0x59, 0xfb, 0x13, 0x9a, 0x8e, 0x79, 0x36, 0xf4, 0x0e, 0x4c, 0x19, 0xa6, + 0x4b, 0x4c, 0x57, 0xe1, 0x6c, 0xc9, 0x21, 0x6c, 0x59, 0x9e, 0x9b, 0x7d, 0x14, 0xff, 0x5c, 0x02, + 0xd8, 0xee, 0xc4, 0xa9, 0x1a, 0x3a, 0x5a, 0x47, 0xaa, 0xbf, 0x37, 0x5a, 0x79, 0x2b, 0x2e, 0xc0, + 0x84, 0x61, 0x36, 0x0d, 0x93, 0xd7, 0x3f, 0x8d, 0xc5, 0x17, 0x3a, 0x07, 0xe3, 0xbb, 0x4d, 0xc3, + 0xd4, 0x99, 0xf9, 0xa7, 0x31, 0xff, 0x10, 0xea, 0xc7, 0x90, 0x65, 0x75, 0x8f, 0x51, 0xfb, 0xc5, + 0x7f, 0x95, 0xe0, 0x7c, 0xd9, 0x32, 0x75, 0x83, 0x8e, 0x43, 0xb5, 0xf9, 0xa9, 0xd0, 0xcd, 0x9b, + 0x90, 0x21, 0x87, 0xed, 0x11, 0xbb, 0x37, 0x4d, 0x0e, 0xdb, 0xec, 0xdf, 0x40, 0xd5, 0x7d, 0x00, + 0x17, 0x4e, 0xb7, 0x32, 0x4e, 0x2d, 0xfe, 0xb5, 0x04, 0xf9, 0xaa, 0x69, 0xb8, 0x9f, 0x0a, 0xf5, + 0xf9, 0x7a, 0x48, 0x86, 0xf4, 0x80, 0xee, 0x82, 0xbc, 0xa7, 0x1a, 0xcd, 0x2d, 0xb3, 0x6e, 0xb5, + 0x76, 0x1d, 0xd7, 0x32, 0x89, 0x23, 0x14, 0xd5, 0x43, 0x17, 0x3a, 0x7b, 0x02, 0xd3, 0x7e, 0x9b, + 0xe2, 0x54, 0xd6, 0x4b, 0x90, 0xab, 0xa6, 0x66, 0x93, 0x16, 0x31, 0x63, 0xd5, 0xd6, 0x15, 0xc8, + 0x18, 0x9e, 0x5c, 0xa6, 0xb1, 0x24, 0x0e, 0x08, 0xa2, 0x4d, 0x1d, 0x98, 0x09, 0x95, 0x1d, 0xa7, + 0x1b, 0xbb, 0x0c, 0x19, 0x93, 0x1c, 0x28, 0x41, 0x7f, 0x25, 0x71, 0xda, 0x24, 0x07, 0xdc, 0xed, + 0x3c, 0x85, 0xdc, 0x0a, 0x69, 0x12, 0x97, 0xc4, 0xef, 0x93, 0x77, 0x20, 0xef, 0x89, 0x8e, 0xb3, + 0x93, 0x7e, 0x5b, 0x02, 0x24, 0xe4, 0xd2, 0x79, 0x30, 0xce, 0x7e, 0x9a, 0xa7, 0xf3, 0xbc, 0xdb, + 0xb1, 0x4d, 0x3e, 0x61, 0x73, 0x2b, 0x05, 0x4e, 0x62, 0x73, 0x76, 0xe0, 0x1b, 0x53, 0x61, 0xdf, + 0xe8, 0xe3, 0x0e, 0x8a, 0x38, 0x0e, 0x60, 0xb6, 0xab, 0x7a, 0xf1, 0x76, 0x65, 0x8a, 0xd5, 0x2c, + 0xb1, 0x90, 0x0c, 0x43, 0x23, 0x46, 0x2c, 0x7e, 0x00, 0x33, 0xe5, 0x26, 0x51, 0xed, 0xb8, 0xd5, + 0x22, 0xba, 0xf3, 0x29, 0xa0, 0xb0, 0xf8, 0x38, 0xbb, 0xd4, 0x80, 0x6c, 0x4d, 0x53, 0xcd, 0xad, + 0x36, 0x75, 0x82, 0x0e, 0x7a, 0x00, 0x17, 0x1c, 0xd7, 0x6a, 0x2b, 0xaa, 0xab, 0x70, 0x84, 0xb4, + 0x6b, 0x75, 0x4c, 0x5d, 0xb5, 0x8f, 0x58, 0x19, 0x69, 0x3c, 0x4b, 0x53, 0x97, 0x5d, 0x56, 0x91, + 0x92, 0x48, 0xa2, 0x7d, 0xd7, 0x32, 0x4c, 0x85, 0x02, 0x99, 0xa6, 0xeb, 0x08, 0x3b, 0x87, 0x96, + 0x61, 0x62, 0x4e, 0x11, 0xad, 0xf8, 0x9e, 0xc4, 0xcb, 0x8a, 0xd3, 0x6c, 0xbe, 0x04, 0x59, 0x47, + 0x53, 0x4d, 0x65, 0xcf, 0xb2, 0x5b, 0xaa, 0xcb, 0x4c, 0x23, 0xbf, 0xf4, 0x5a, 0x14, 0x3c, 0xd4, + 0x54, 0xf3, 0x21, 0xcb, 0x84, 0xc1, 0xf1, 0xff, 0x87, 0xad, 0xe7, 0x51, 0x2a, 0x9d, 0x94, 0x53, + 0xc5, 0xff, 0x90, 0x60, 0x8a, 0xd7, 0x32, 0x4e, 0xeb, 0x79, 0x13, 0x52, 0xb6, 0x75, 0xc0, 0xad, + 0x27, 0xbb, 0x74, 0x39, 0x42, 0xc4, 0x3a, 0x39, 0x0a, 0xbb, 0x6d, 0x96, 0x1d, 0x95, 0x40, 0x00, + 0x15, 0x85, 0x71, 0x27, 0x47, 0xe5, 0x06, 0xce, 0x85, 0xa9, 0x8c, 0x1b, 0x90, 0xdf, 0x55, 0x5d, + 0x6d, 0x9f, 0xf6, 0x0f, 0xab, 0x24, 0x47, 0xf7, 0x38, 0xc7, 0xa8, 0x5e, 0xcd, 0x8b, 0xbf, 0x27, + 0x01, 0xc2, 0xe4, 0x05, 0xb1, 0x1d, 0xf2, 0xe9, 0xef, 0xa4, 0xff, 0x92, 0x60, 0xb6, 0xab, 0xb2, + 0xff, 0xb7, 0xfa, 0xea, 0xe7, 0x25, 0xb8, 0x58, 0xde, 0x27, 0xda, 0xb3, 0xb2, 0x65, 0x3a, 0x86, + 0xe3, 0x12, 0x53, 0x3b, 0x8a, 0xb3, 0xc3, 0x2e, 0x43, 0xe6, 0xc0, 0x70, 0xf7, 0x15, 0xdd, 0xd8, + 0xdb, 0x63, 0xc3, 0x39, 0x8d, 0xd3, 0x94, 0xb0, 0x62, 0xec, 0xed, 0x89, 0xc1, 0xac, 0x40, 0xa1, + 0xb7, 0x06, 0xf1, 0x02, 0x82, 0xf3, 0x98, 0x68, 0x56, 0xab, 0xdd, 0x71, 0x49, 0xcd, 0x55, 0x5d, + 0x27, 0xce, 0x06, 0x5e, 0x84, 0x49, 0xdd, 0x3e, 0x52, 0xec, 0x8e, 0x29, 0x9a, 0x37, 0xa1, 0xdb, + 0x47, 0xb8, 0x63, 0x8a, 0xc6, 0xfd, 0xa9, 0x04, 0x17, 0x4e, 0x17, 0x1e, 0xa7, 0x85, 0x7d, 0x15, + 0xb2, 0xaa, 0xae, 0x13, 0x5d, 0xd1, 0x49, 0xd3, 0x55, 0x05, 0x90, 0xbb, 0x1f, 0x92, 0x24, 0x76, + 0x2b, 0x16, 0xf9, 0x36, 0xc5, 0xa2, 0xb7, 0x5b, 0xb1, 0xb8, 0xf1, 0xa4, 0x5c, 0x66, 0xf5, 0x59, + 0xa1, 0x8c, 0x9e, 0x01, 0x31, 0x59, 0x8c, 0x52, 0xd4, 0xe0, 0x62, 0x89, 0x34, 0x0c, 0x33, 0xbc, + 0x8e, 0x8e, 0x7d, 0x3a, 0x52, 0xa0, 0xd0, 0x5b, 0x48, 0x9c, 0x7d, 0xff, 0x57, 0x49, 0x38, 0xbf, + 0x6a, 0xea, 0x9f, 0x4c, 0x23, 0x28, 0x92, 0xd0, 0xac, 0x56, 0xcb, 0x70, 0xbd, 0xbe, 0xe7, 0x5f, + 0xe8, 0xa7, 0x20, 0xad, 0x13, 0x55, 0xf7, 0xd7, 0x5f, 0xd9, 0x2e, 0x1f, 0xd5, 0x71, 0x8d, 0xe6, + 0xe2, 0x7e, 0x53, 0x5b, 0xac, 0x7b, 0x7b, 0x43, 0xd8, 0xcf, 0x8e, 0xbe, 0x01, 0x17, 0xe9, 0x28, + 0xb6, 0x4d, 0xb5, 0xa9, 0x70, 0x69, 0x8a, 0x6b, 0x1b, 0x8d, 0x06, 0xb1, 0xc5, 0x8e, 0xc5, 0xed, + 0x88, 0x7a, 0x56, 0x05, 0x47, 0x99, 0x31, 0xd4, 0x79, 0x7e, 0x7c, 0xde, 0x88, 0x22, 0xa3, 0xaf, + 0xf8, 0x0b, 0x5c, 0xa7, 0xad, 0x9a, 0x4e, 0x61, 0x9c, 0xb9, 0x97, 0x7e, 0x1b, 0x21, 0xc2, 0x32, + 0x84, 0x43, 0xa2, 0x14, 0x07, 0xdd, 0xa3, 0x08, 0xeb, 0x79, 0xc7, 0xb0, 0x89, 0x72, 0xbf, 0xad, + 0x15, 0x26, 0x68, 0xdb, 0x4b, 0xf9, 0x93, 0xe3, 0x79, 0xc0, 0x9c, 0x7c, 0x7f, 0xbb, 0x4c, 0x11, + 0x17, 0xff, 0xdf, 0xd6, 0xd0, 0x6d, 0x90, 0x4d, 0x4b, 0xb1, 0xc9, 0x9e, 0x4d, 0x9c, 0x7d, 0x51, + 0x6c, 0x9a, 0x69, 0x2c, 0x6f, 0x5a, 0x98, 0x93, 0xb9, 0xe8, 0x0b, 0x30, 0xd1, 0xb6, 0x0c, 0xc7, + 0x32, 0x0b, 0x19, 0xae, 0x51, 0xfe, 0xe5, 0x3b, 0xee, 0x49, 0x39, 0x5d, 0xfc, 0x55, 0x09, 0x2e, + 0x9c, 0xee, 0xd3, 0x38, 0xc7, 0xd4, 0x6d, 0x90, 0x2d, 0x93, 0x28, 0xed, 0x7d, 0xd5, 0x21, 0xa2, + 0x0f, 0x04, 0x52, 0xcc, 0x5b, 0x26, 0xd9, 0xa6, 0x64, 0xae, 0xd1, 0xae, 0x89, 0xe4, 0x17, 0x25, + 0x98, 0x59, 0xd6, 0x5b, 0x86, 0x59, 0x6b, 0x37, 0x8d, 0x58, 0x17, 0x1e, 0xd7, 0x21, 0xe3, 0x50, + 0x99, 0x6c, 0x3f, 0x2d, 0xd1, 0xbd, 0x9f, 0x96, 0x66, 0x29, 0xeb, 0xe4, 0x28, 0xc0, 0x77, 0xe1, + 0x4a, 0xc4, 0x39, 0x94, 0x3e, 0x10, 0xed, 0xdb, 0x20, 0xf6, 0x27, 0x84, 0x4c, 0xc3, 0xe2, 0xe3, + 0xac, 0xf9, 0xb7, 0x24, 0xb8, 0xc4, 0x64, 0x33, 0x93, 0xd9, 0x23, 0x36, 0xdb, 0xde, 0x8c, 0xb3, + 0x8b, 0x5e, 0x87, 0x09, 0x57, 0xb5, 0x1b, 0x84, 0x3b, 0x82, 0xf1, 0x52, 0xf6, 0xe3, 0xe3, 0xf9, + 0xc9, 0x9a, 0x6b, 0xd9, 0xa4, 0xba, 0x82, 0x45, 0x92, 0x68, 0xa7, 0x0a, 0x73, 0x51, 0x75, 0x89, + 0xb3, 0xbd, 0xff, 0x2e, 0x89, 0x32, 0xca, 0xfb, 0x1c, 0xe6, 0xb7, 0x9b, 0x86, 0xa6, 0xc6, 0x3a, + 0xed, 0xad, 0x42, 0x56, 0x63, 0xc2, 0x15, 0xf7, 0xa8, 0xcd, 0x17, 0xa4, 0xf9, 0xa5, 0xeb, 0x91, + 0x82, 0x58, 0xe1, 0xbc, 0x26, 0xf5, 0xa3, 0x36, 0xc1, 0xa0, 0xf9, 0xff, 0xd1, 0x0a, 0x4c, 0x72, + 0xe5, 0x78, 0x28, 0x67, 0x80, 0x08, 0x3a, 0xd0, 0xeb, 0x2c, 0xb3, 0xf0, 0x49, 0x1e, 0xab, 0x50, + 0xec, 0x2e, 0x5c, 0x8e, 0x6c, 0x74, 0x9c, 0x9a, 0xfd, 0x75, 0x09, 0x66, 0x2b, 0x44, 0xb5, 0xdd, + 0x5d, 0xa2, 0xba, 0xf5, 0xc3, 0x58, 0x27, 0x93, 0x37, 0x21, 0x69, 0x5a, 0x07, 0x62, 0x0a, 0x1f, + 0x3c, 0x5f, 0x08, 0x05, 0xd0, 0xfc, 0xa2, 0xf1, 0x5f, 0x83, 0x73, 0xdd, 0xf5, 0x8a, 0xb3, 0xd5, + 0x7f, 0x94, 0x84, 0xcc, 0x5a, 0x39, 0xce, 0xb6, 0x7e, 0x41, 0x2c, 0x81, 0x79, 0xa7, 0x47, 0x1d, + 0x60, 0xf8, 0xe5, 0x2d, 0xae, 0x95, 0xd7, 0xc9, 0x91, 0x87, 0x8f, 0x29, 0x17, 0x5a, 0x86, 0x8c, + 0xbb, 0x4f, 0xe7, 0x0c, 0xab, 0xa9, 0x8b, 0x59, 0x71, 0x24, 0x7d, 0x05, 0x5c, 0xa8, 0x09, 0xe7, + 0xdd, 0x43, 0x93, 0x4d, 0x45, 0x4a, 0x43, 0x53, 0x02, 0x71, 0xe3, 0xa3, 0x88, 0x9b, 0xa3, 0xe2, + 0x4e, 0x8e, 0xe7, 0x51, 0xfd, 0xd0, 0xa4, 0xf3, 0xd6, 0x5a, 0xb9, 0xee, 0x09, 0xc0, 0xc8, 0x15, + 0x34, 0xcd, 0xa7, 0xcd, 0x3d, 0x83, 0x71, 0xd6, 0x0a, 0xef, 0xcc, 0x44, 0x8a, 0x38, 0x33, 0xa1, + 0x8d, 0xf2, 0x0a, 0x78, 0x15, 0x23, 0x08, 0xb8, 0xb8, 0x29, 0x08, 0x83, 0x78, 0x17, 0x80, 0xaa, + 0x30, 0x4e, 0x33, 0xf8, 0x41, 0x12, 0xf2, 0xdb, 0x1d, 0x67, 0x3f, 0x66, 0xbb, 0x2f, 0x03, 0xb4, + 0x3b, 0xce, 0x3e, 0xb1, 0x15, 0xf7, 0xd0, 0x14, 0x2d, 0x1f, 0x72, 0xa0, 0xe3, 0x35, 0x9d, 0xf3, + 0xd5, 0x0f, 0x4d, 0xb4, 0x25, 0x84, 0x10, 0x25, 0x38, 0x15, 0xba, 0x3b, 0x02, 0x0c, 0xae, 0x1f, + 0x9a, 0x1b, 0xc4, 0xc7, 0xbf, 0x5c, 0x20, 0xa1, 0x02, 0xbf, 0x00, 0x93, 0xf4, 0x43, 0x71, 0xad, + 0x57, 0xb1, 0xb0, 0x09, 0xca, 0x53, 0xb7, 0xbc, 0xb1, 0x3c, 0xfe, 0x6a, 0x63, 0x19, 0xbd, 0x03, + 0x19, 0x5e, 0x28, 0xf5, 0xa9, 0x13, 0xcc, 0xa7, 0x46, 0x69, 0x42, 0x74, 0x02, 0xf3, 0xa6, 0x69, + 0x56, 0x22, 0xf5, 0xa5, 0xe7, 0x60, 0x7c, 0xcf, 0xb2, 0x35, 0xc2, 0x4e, 0x8f, 0xd2, 0x98, 0x7f, + 0xf8, 0xc0, 0x29, 0x2d, 0x67, 0x8a, 0xbf, 0x29, 0xc1, 0xb4, 0xdf, 0x81, 0x71, 0x22, 0xa6, 0x72, + 0x97, 0xf6, 0x5f, 0xbd, 0x0b, 0xa9, 0xc6, 0x8b, 0xdf, 0x4f, 0xc0, 0xf4, 0xbb, 0x1d, 0x62, 0x1f, + 0xc5, 0x6c, 0x5f, 0x25, 0x7e, 0x52, 0x98, 0x38, 0xa3, 0x4d, 0xb0, 0xb3, 0xc3, 0x9b, 0x30, 0x7d, + 0xa0, 0x1a, 0xae, 0xb2, 0x67, 0xd9, 0x4a, 0xa7, 0xad, 0xab, 0xae, 0x77, 0xae, 0x92, 0xa3, 0xe4, + 0x87, 0x96, 0xbd, 0xc3, 0x88, 0x88, 0x00, 0x7a, 0x66, 0x5a, 0x07, 0xa6, 0x42, 0xc9, 0x86, 0xd9, + 0xa0, 0xfa, 0x70, 0x0a, 0x29, 0xb6, 0xd1, 0xf7, 0xd6, 0x3f, 0x1c, 0xcf, 0x3f, 0x68, 0x18, 0xee, + 0x7e, 0x67, 0x77, 0x51, 0xb3, 0x5a, 0xf7, 0xfc, 0x8a, 0xe8, 0xbb, 0xc1, 0xff, 0x7b, 0xed, 0x67, + 0x8d, 0x7b, 0xec, 0x88, 0xb8, 0xd3, 0x31, 0xf4, 0xc5, 0x9d, 0x9d, 0xea, 0x0a, 0x96, 0x99, 0xc8, + 0xf7, 0xb8, 0xc4, 0xfa, 0xa1, 0xe9, 0x4d, 0x78, 0x1f, 0x4b, 0x20, 0x07, 0x0a, 0x8b, 0xb3, 0x3f, + 0x57, 0x21, 0xfb, 0xbc, 0x43, 0x6c, 0x83, 0xe8, 0xaf, 0xdc, 0xa1, 0x20, 0x18, 0xe9, 0x18, 0x7a, + 0x1f, 0xa6, 0xba, 0xf4, 0x90, 0xfc, 0xc9, 0xf4, 0x90, 0x3d, 0x08, 0x54, 0x50, 0xfc, 0x41, 0x02, + 0x10, 0x6b, 0x7c, 0x95, 0xef, 0x79, 0x7c, 0xca, 0x0c, 0xe6, 0x09, 0x80, 0xb1, 0xa7, 0xb4, 0x0c, + 0xc7, 0x31, 0xcc, 0x06, 0xb3, 0x95, 0xfc, 0xd2, 0x5b, 0x11, 0x75, 0xe9, 0x6d, 0xc2, 0x62, 0x75, + 0x6f, 0x83, 0xb3, 0x95, 0xc8, 0xbe, 0xfa, 0xc2, 0xb0, 0x6c, 0x9c, 0x31, 0x3c, 0x52, 0xb1, 0x04, + 0x33, 0x3d, 0xe9, 0x28, 0x0f, 0xb0, 0xb2, 0xa5, 0x6c, 0x6e, 0xd5, 0x2b, 0xd5, 0xcd, 0x35, 0x79, + 0x0c, 0xc9, 0x30, 0x85, 0x57, 0xeb, 0x3b, 0x78, 0x53, 0x59, 0xc5, 0x78, 0x0b, 0xcb, 0x12, 0xca, + 0xc2, 0xe4, 0x36, 0x5e, 0x7d, 0xb2, 0xba, 0x59, 0x97, 0x13, 0xc2, 0x7a, 0x7e, 0x16, 0x66, 0xbb, + 0x0a, 0x8f, 0xd3, 0x7e, 0xae, 0xc1, 0xd4, 0x9e, 0xd5, 0x31, 0x75, 0x85, 0xaf, 0x1a, 0xc5, 0xea, + 0x38, 0xcb, 0x68, 0xbc, 0xbc, 0xe2, 0x37, 0x13, 0x70, 0x0e, 0x13, 0xc7, 0x6a, 0xbe, 0x20, 0xf1, + 0xf7, 0xe0, 0x16, 0x88, 0xad, 0x30, 0xe5, 0x27, 0xe9, 0xc8, 0x0c, 0x97, 0xc1, 0x67, 0x83, 0x09, + 0xc7, 0x55, 0xdd, 0x8e, 0x23, 0xba, 0xf2, 0xfa, 0xe0, 0xb1, 0x50, 0x63, 0x79, 0xb1, 0xe0, 0x09, + 0x2d, 0x6a, 0x53, 0xbd, 0x8b, 0xda, 0xe2, 0x4f, 0xc3, 0xf9, 0x53, 0x8a, 0x88, 0x73, 0xd2, 0xfe, + 0xdb, 0x04, 0x5c, 0xea, 0x16, 0x1f, 0xf7, 0x79, 0xcb, 0xff, 0x0e, 0x65, 0xa3, 0x0a, 0xe4, 0x5a, + 0x86, 0xa9, 0x04, 0x18, 0xeb, 0x15, 0x26, 0xe7, 0x29, 0xba, 0x5e, 0xeb, 0x86, 0x59, 0x74, 0x1d, + 0x17, 0xa5, 0xd7, 0x38, 0xfb, 0xee, 0x3b, 0x12, 0x4c, 0xc5, 0xbd, 0xda, 0x3e, 0xdb, 0xa1, 0xaf, + 0x68, 0x73, 0x1d, 0x72, 0x9f, 0xc0, 0xf2, 0xfc, 0x77, 0x25, 0x40, 0x75, 0xbb, 0x63, 0x6a, 0xaa, + 0x4b, 0x1e, 0x5b, 0x8d, 0x38, 0x1b, 0x7b, 0x0e, 0xc6, 0x0d, 0x53, 0x27, 0x87, 0xac, 0xb1, 0x29, + 0xcc, 0x3f, 0xd0, 0x7d, 0x48, 0x8b, 0xa0, 0x1d, 0x7e, 0x88, 0x9d, 0x2c, 0x5d, 0x38, 0x39, 0x9e, + 0x9f, 0xe4, 0x21, 0x3a, 0x2b, 0x1f, 0x07, 0x7f, 0xf1, 0x24, 0x8f, 0xd2, 0xf1, 0x8e, 0xf9, 0xdf, + 0x87, 0xd9, 0xae, 0x8a, 0xc6, 0xa9, 0x85, 0xef, 0x27, 0x60, 0x56, 0x34, 0x27, 0xf6, 0xed, 0x89, + 0x33, 0x45, 0x7c, 0xa1, 0x2f, 0x02, 0xb4, 0x6d, 0xf2, 0x42, 0xe1, 0xac, 0xc9, 0x91, 0x58, 0x33, + 0x94, 0x83, 0x11, 0xd0, 0x57, 0x61, 0x9a, 0x0e, 0xb8, 0xb6, 0x6d, 0xb5, 0x2d, 0x87, 0x22, 0x09, + 0x67, 0x34, 0x24, 0x3d, 0x73, 0x72, 0x3c, 0x9f, 0xdb, 0x30, 0xcc, 0x6d, 0xc1, 0x58, 0xaf, 0x61, + 0x3a, 0x72, 0xfd, 0x4f, 0x0f, 0xfe, 0xfc, 0x8d, 0x04, 0xe7, 0x3e, 0xb1, 0x0d, 0x9d, 0xff, 0x09, + 0x8d, 0xf9, 0xf3, 0x81, 0xcc, 0x3e, 0xab, 0xe6, 0x9e, 0x15, 0xff, 0x36, 0xdb, 0x77, 0x24, 0x98, + 0x09, 0x89, 0x8f, 0x73, 0xd6, 0x3f, 0x5b, 0x5c, 0xe1, 0xd7, 0x28, 0x0e, 0x08, 0x9b, 0x7d, 0x9c, + 0x83, 0xea, 0x97, 0x12, 0x70, 0xa1, 0xcc, 0x0f, 0x5f, 0xd8, 0x19, 0x93, 0xd3, 0x69, 0xc5, 0x69, + 0x25, 0x05, 0x98, 0x7c, 0x41, 0x6c, 0xc7, 0xb0, 0xf8, 0xbc, 0x97, 0xc3, 0xde, 0x27, 0xfa, 0x19, + 0xc8, 0x6a, 0xa2, 0x40, 0xcf, 0xcb, 0x4c, 0x95, 0xaa, 0x54, 0xc0, 0x19, 0xd1, 0xef, 0xc9, 0xf1, + 0x3c, 0x78, 0x4d, 0xa8, 0xae, 0x60, 0xf0, 0xa4, 0x57, 0x75, 0x16, 0x4b, 0x69, 0xaa, 0x6d, 0x67, + 0xdf, 0xf2, 0xf6, 0xa9, 0xfd, 0x6f, 0xd1, 0xe9, 0x5f, 0x87, 0x8b, 0x3d, 0x5a, 0x88, 0x53, 0xcd, + 0x3f, 0x9a, 0x84, 0xdc, 0xea, 0x61, 0xdb, 0xb2, 0xdd, 0x1a, 0x9f, 0xec, 0xd1, 0x0a, 0xa4, 0xdb, + 0xb6, 0xf5, 0xc2, 0xf0, 0x04, 0xe7, 0x23, 0xcf, 0x2d, 0xba, 0x78, 0xb6, 0x45, 0x7e, 0xec, 0x73, + 0x22, 0x0c, 0x99, 0xc7, 0x96, 0xa6, 0x36, 0x1f, 0x1a, 0x4d, 0xcf, 0xaa, 0x16, 0x87, 0x89, 0x59, + 0xf4, 0x39, 0xb6, 0x55, 0x77, 0xdf, 0x1b, 0x64, 0x3e, 0x11, 0xad, 0x41, 0xba, 0xe2, 0xba, 0x6d, + 0x9a, 0x28, 0x46, 0xe8, 0x8d, 0xa1, 0x22, 0x29, 0x83, 0x90, 0xe4, 0x33, 0x23, 0x0c, 0x33, 0x6b, + 0x96, 0xd5, 0x68, 0x92, 0x72, 0xd3, 0xea, 0xe8, 0x65, 0xcb, 0xdc, 0x33, 0x1a, 0xc2, 0xc3, 0x5d, + 0x1f, 0x2a, 0x71, 0xad, 0x5c, 0xc3, 0xbd, 0xec, 0xe8, 0xcb, 0x90, 0xae, 0x3d, 0x10, 0xa2, 0x38, + 0x3e, 0x79, 0x7d, 0xa8, 0xa8, 0xda, 0x03, 0xec, 0x33, 0xa1, 0x0a, 0x64, 0x97, 0x5f, 0x76, 0x6c, + 0x22, 0x64, 0x4c, 0x30, 0x19, 0x37, 0x87, 0xca, 0x60, 0x3c, 0x38, 0xcc, 0x3a, 0x77, 0x07, 0x72, + 0x5d, 0x9a, 0x44, 0x08, 0x52, 0x6d, 0xaa, 0x34, 0xda, 0x9d, 0x19, 0xcc, 0xfe, 0x73, 0xf3, 0x9a, + 0xbb, 0x09, 0x29, 0xaa, 0x15, 0x3a, 0x1c, 0x76, 0x55, 0x87, 0xec, 0xd8, 0x86, 0xc8, 0xe4, 0x7d, + 0x8a, 0x7c, 0x7f, 0x29, 0x41, 0xa2, 0xf6, 0x80, 0x22, 0xb4, 0xdd, 0x8e, 0xf6, 0x8c, 0xb8, 0x22, + 0x97, 0xf8, 0x62, 0xc8, 0xcd, 0x26, 0x7b, 0x06, 0x9f, 0xad, 0x33, 0x58, 0x7c, 0xa1, 0xd7, 0x00, + 0x54, 0x4d, 0x23, 0x8e, 0xa3, 0x78, 0xf1, 0xc6, 0x19, 0x9c, 0xe1, 0x94, 0x75, 0x72, 0x44, 0xd9, + 0x1c, 0xa2, 0xd9, 0x84, 0x1b, 0x7f, 0x06, 0x8b, 0x2f, 0xca, 0xe6, 0x92, 0x56, 0x5b, 0x71, 0xad, + 0x67, 0xc4, 0x64, 0xda, 0xcc, 0xe0, 0x0c, 0xa5, 0xd4, 0x29, 0x81, 0x8e, 0x1a, 0x62, 0xea, 0x6d, + 0xcb, 0x30, 0x5d, 0xa6, 0xa6, 0x0c, 0xf6, 0xbf, 0xa9, 0x48, 0x9b, 0x34, 0x0c, 0x11, 0x89, 0x9b, + 0xc1, 0xe2, 0x4b, 0x34, 0x63, 0x0b, 0x92, 0x6b, 0xe5, 0xda, 0x2b, 0x37, 0x03, 0x41, 0x4a, 0xed, + 0x08, 0xa3, 0xcb, 0x60, 0xf6, 0x5f, 0x08, 0xfc, 0xa6, 0x04, 0xe3, 0x4c, 0xf5, 0xe8, 0x0a, 0x64, + 0x34, 0xcb, 0x74, 0x55, 0xc3, 0x14, 0xe3, 0x26, 0x83, 0x03, 0x42, 0x5f, 0xc9, 0xd7, 0x60, 0x4a, + 0xd5, 0x34, 0xab, 0x63, 0xba, 0x8a, 0xa9, 0xb6, 0x88, 0x28, 0x21, 0x2b, 0x68, 0x9b, 0x6a, 0x8b, + 0xa0, 0x79, 0xf0, 0x3e, 0xfd, 0xa8, 0xec, 0x0c, 0x06, 0x41, 0xf2, 0x8f, 0x8f, 0x84, 0xbb, 0xf8, + 0x7d, 0x09, 0x66, 0xde, 0xb3, 0x0d, 0x97, 0x94, 0x78, 0xac, 0x40, 0x7c, 0x0e, 0xf3, 0x6d, 0xc8, + 0xe8, 0xaa, 0xab, 0xf2, 0x08, 0xec, 0xc4, 0xc0, 0x08, 0x6c, 0x6f, 0xbc, 0xd1, 0xfc, 0x2c, 0x0a, + 0x1b, 0x41, 0x8a, 0xfe, 0xe7, 0xbe, 0x14, 0xb3, 0xff, 0xc1, 0xd1, 0x51, 0xb8, 0xba, 0x71, 0x7a, + 0xb6, 0x7f, 0x4c, 0x78, 0x9e, 0x2d, 0x4e, 0x35, 0x7c, 0x05, 0x26, 0xc5, 0xaa, 0x48, 0x28, 0x61, + 0x61, 0xd8, 0x08, 0xf5, 0x8e, 0x3c, 0x04, 0x1b, 0x2a, 0x01, 0x38, 0xae, 0x6a, 0xbb, 0x6c, 0x3d, + 0x33, 0xd2, 0x19, 0xb3, 0xe7, 0x09, 0x19, 0x1b, 0xa5, 0xa2, 0x4d, 0xc8, 0xb6, 0x5e, 0x68, 0x9a, + 0xb2, 0x67, 0x34, 0x5d, 0x71, 0xbc, 0x1c, 0x1d, 0x4c, 0xb3, 0xf1, 0xa4, 0x5c, 0x7e, 0xc8, 0x32, + 0xf1, 0x53, 0xde, 0xe0, 0x1b, 0x03, 0x95, 0xc0, 0xff, 0xa3, 0xcf, 0x80, 0x88, 0xb2, 0x53, 0x1c, + 0xc7, 0x65, 0x03, 0x2e, 0x5d, 0xca, 0x9d, 0x1c, 0xcf, 0x67, 0x30, 0xa3, 0xd6, 0x6a, 0x75, 0x9c, + 0xe1, 0x19, 0x6a, 0x8e, 0x37, 0x33, 0x7d, 0x5b, 0x82, 0x5c, 0xa9, 0xd3, 0x7c, 0xb6, 0xd5, 0xae, + 0x75, 0x5a, 0x2d, 0xd5, 0x3e, 0x42, 0x97, 0x3d, 0x13, 0x31, 0x5e, 0x12, 0xa6, 0xe2, 0xa4, 0xb0, + 0x01, 0xe3, 0x25, 0xa1, 0x36, 0x20, 0x02, 0x6a, 0x28, 0x9d, 0x47, 0xcb, 0xbc, 0x0e, 0x39, 0x06, + 0xeb, 0x15, 0x62, 0xba, 0xb6, 0x41, 0xf8, 0xaa, 0x31, 0x89, 0xa7, 0x18, 0x71, 0x95, 0xd3, 0xd0, + 0x0d, 0xc8, 0x3b, 0x47, 0x8e, 0x4b, 0x5a, 0x0a, 0xbf, 0x72, 0xc1, 0xb1, 0x68, 0x12, 0xe7, 0x38, + 0x15, 0x73, 0x62, 0xf1, 0x0f, 0x93, 0x90, 0xf7, 0xba, 0x3b, 0x4e, 0x68, 0x54, 0x82, 0xf1, 0x3d, + 0xa3, 0x49, 0xbc, 0x48, 0xa0, 0xfe, 0x0e, 0xd9, 0x93, 0xb4, 0x48, 0xdd, 0xae, 0x07, 0x94, 0x18, + 0x6b, 0x1c, 0x5d, 0x3e, 0xf7, 0x91, 0x04, 0x29, 0x36, 0x0b, 0xde, 0x87, 0x14, 0x1b, 0x83, 0xd2, + 0x28, 0x63, 0x90, 0x65, 0xf5, 0xfd, 0x7f, 0x22, 0xf0, 0xff, 0xcc, 0xf7, 0xee, 0xab, 0x6f, 0xde, + 0x5f, 0x62, 0xdd, 0x3d, 0x85, 0xc5, 0x17, 0x2a, 0x41, 0x9a, 0xb0, 0xf6, 0x10, 0x5d, 0xcc, 0x41, + 0x51, 0x16, 0xde, 0xd5, 0xf1, 0xde, 0x78, 0xf7, 0xf8, 0xd0, 0x25, 0x48, 0x52, 0x3b, 0x9a, 0xe4, + 0x67, 0x25, 0x27, 0xc7, 0xf3, 0x49, 0x6a, 0x41, 0x94, 0xc6, 0x4f, 0xdc, 0x1f, 0xa5, 0xd2, 0x29, + 0x79, 0xbc, 0xf8, 0x27, 0x29, 0xc8, 0x55, 0x5b, 0x71, 0x8f, 0xd0, 0xe5, 0xee, 0x0e, 0x8b, 0x82, + 0x08, 0x5d, 0x85, 0x46, 0xf4, 0x57, 0x97, 0xaf, 0x4b, 0xbe, 0x9a, 0xaf, 0xab, 0xd2, 0x09, 0x48, + 0xdc, 0x37, 0xa1, 0xe5, 0xbf, 0x31, 0xb4, 0xfc, 0xba, 0xba, 0xdb, 0x24, 0x98, 0xf2, 0x78, 0x47, + 0x11, 0x5c, 0x00, 0xfa, 0x12, 0x9b, 0xe7, 0xb8, 0xd1, 0x4c, 0x8c, 0x6e, 0x34, 0x93, 0xc4, 0xd4, + 0x99, 0xc9, 0x1c, 0x0a, 0x8b, 0xf9, 0x3c, 0x24, 0x75, 0x63, 0x90, 0x4a, 0xa3, 0xfc, 0x15, 0x65, + 0x19, 0x62, 0x38, 0xa9, 0xb0, 0xe1, 0xf8, 0xc7, 0x15, 0x49, 0x39, 0x35, 0xb7, 0x05, 0x10, 0xb4, + 0x0a, 0x2d, 0xc0, 0x84, 0xd5, 0xd4, 0x29, 0x98, 0xa6, 0x55, 0xc8, 0x95, 0x32, 0x27, 0xc7, 0xf3, + 0xe3, 0x5b, 0x4d, 0xbd, 0xba, 0x82, 0xc7, 0xad, 0xa6, 0x5e, 0xd5, 0xd9, 0x35, 0x1d, 0x72, 0xa0, + 0xb0, 0x9b, 0x51, 0x2c, 0x4c, 0x02, 0x4f, 0x9a, 0xe4, 0x60, 0x85, 0x38, 0x5a, 0x78, 0x76, 0x13, + 0x66, 0xf3, 0x3b, 0x12, 0xe4, 0x3d, 0x0d, 0xc6, 0x3b, 0xd2, 0xd3, 0x46, 0x4b, 0x58, 0x7e, 0xf2, + 0xd5, 0x2c, 0xdf, 0xe3, 0x13, 0x21, 0xc8, 0xdf, 0x92, 0x60, 0x96, 0xc7, 0x72, 0x68, 0xaa, 0x4b, + 0x7d, 0x6d, 0x8c, 0xe6, 0x7d, 0x07, 0x64, 0x5b, 0x35, 0x75, 0xab, 0x65, 0xbc, 0x24, 0x7c, 0xb5, + 0xea, 0x88, 0x4d, 0xda, 0x69, 0x9f, 0xce, 0x96, 0x63, 0xde, 0x62, 0xfb, 0xdf, 0x24, 0x38, 0xd7, + 0x5d, 0x99, 0x38, 0x95, 0xb6, 0x0e, 0x13, 0x6c, 0xa3, 0xc5, 0x1b, 0x6e, 0x9f, 0x8d, 0x10, 0x12, + 0x55, 0x3a, 0xbf, 0x55, 0xe5, 0x1b, 0x3c, 0x13, 0x31, 0xf7, 0x15, 0x18, 0x67, 0xe4, 0x33, 0xf8, + 0x38, 0xa1, 0xf9, 0xe7, 0x30, 0xb3, 0xac, 0xeb, 0xb5, 0x9a, 0xb0, 0xbe, 0xf8, 0xd4, 0xee, 0x41, + 0x98, 0x44, 0x14, 0x84, 0x09, 0x17, 0x19, 0x27, 0x84, 0x69, 0x43, 0x5e, 0x84, 0x58, 0xc5, 0xbc, + 0xb3, 0x76, 0x40, 0x31, 0x97, 0x30, 0x1b, 0xfe, 0x11, 0xdc, 0xec, 0xf0, 0x4b, 0x8c, 0xb3, 0x25, + 0x1d, 0x98, 0xf5, 0xe4, 0xc6, 0xbd, 0x89, 0x3d, 0xa8, 0x39, 0x6c, 0x87, 0x22, 0x5c, 0x6c, 0x9c, + 0x6d, 0xfa, 0x03, 0x09, 0xe6, 0xd6, 0x88, 0x5b, 0x13, 0x0b, 0xf6, 0x87, 0x96, 0x1d, 0xfb, 0x8e, + 0xef, 0x1a, 0x40, 0x93, 0xec, 0x89, 0x30, 0x7c, 0x01, 0x38, 0x47, 0xbf, 0x33, 0x9a, 0xa1, 0xbc, + 0x2c, 0x49, 0xa8, 0xe3, 0x3f, 0x25, 0xb8, 0x1c, 0x59, 0xe3, 0x38, 0x3d, 0xc2, 0xd7, 0x81, 0x21, + 0x4b, 0xc5, 0x71, 0x55, 0xd7, 0x11, 0x8e, 0xf4, 0x33, 0xaf, 0x12, 0xd6, 0x5a, 0x9a, 0x11, 0x31, + 0x1a, 0x19, 0x9f, 0x84, 0x33, 0x54, 0x24, 0xfb, 0x8b, 0x16, 0x61, 0x96, 0xb9, 0x3d, 0x45, 0x6d, + 0xb7, 0x9b, 0x06, 0xd1, 0x15, 0xbe, 0x4d, 0x9c, 0x62, 0xdb, 0xc4, 0x33, 0x2c, 0x69, 0x99, 0xa7, + 0x54, 0x69, 0x82, 0x70, 0x0c, 0x1f, 0xc0, 0x0c, 0xd3, 0x44, 0xdc, 0x51, 0xc4, 0x42, 0xb3, 0x7f, + 0x26, 0x01, 0x0a, 0xcb, 0xff, 0xe4, 0x14, 0x9a, 0x88, 0x5b, 0xa1, 0xc5, 0xbf, 0xbb, 0x08, 0x53, + 0xa2, 0x95, 0x3b, 0xa6, 0x61, 0x99, 0xe8, 0x3e, 0x24, 0x1b, 0x62, 0x6d, 0x9c, 0x8d, 0x5c, 0x55, + 0x04, 0x57, 0x48, 0x2b, 0x63, 0x98, 0xe6, 0xa5, 0x2c, 0xed, 0x8e, 0x1b, 0x11, 0xfc, 0x12, 0x04, + 0x3e, 0x84, 0x59, 0xda, 0x1d, 0x17, 0xd5, 0x60, 0x5a, 0x0b, 0x2e, 0xde, 0x29, 0x94, 0x3d, 0xd9, + 0x37, 0x4c, 0x36, 0xf2, 0x22, 0x62, 0x65, 0x0c, 0xe7, 0xb5, 0xae, 0x04, 0x54, 0x0e, 0xdf, 0xf4, + 0x4a, 0xf5, 0xdd, 0x86, 0x39, 0x7d, 0xcb, 0xac, 0x32, 0x16, 0xba, 0x10, 0x86, 0xde, 0x86, 0x09, + 0x9d, 0xdd, 0x20, 0x12, 0x1b, 0x39, 0x51, 0x66, 0xd1, 0x75, 0x69, 0xab, 0x32, 0x86, 0x05, 0x07, + 0x7a, 0x04, 0x53, 0xfc, 0x9f, 0x18, 0xb3, 0x13, 0x7d, 0xf7, 0xa9, 0x7a, 0xef, 0x50, 0x55, 0xc6, + 0x70, 0x56, 0x0f, 0xa8, 0xe8, 0x73, 0x90, 0x72, 0x34, 0x95, 0xef, 0x64, 0x44, 0x1f, 0xe2, 0x87, + 0x2e, 0x69, 0x54, 0xe8, 0x44, 0xa8, 0xa9, 0x26, 0x7a, 0x0a, 0x33, 0xbb, 0xa4, 0x61, 0x98, 0x8a, + 0x1b, 0x1c, 0xb4, 0xb1, 0x90, 0xdd, 0xee, 0xb3, 0x3d, 0x1f, 0xcf, 0x44, 0x47, 0x8a, 0x57, 0xc6, + 0xb0, 0xbc, 0x7b, 0x2a, 0x89, 0x76, 0x19, 0x03, 0xa4, 0x21, 0xc1, 0x99, 0xbe, 0x5d, 0x16, 0x19, + 0xbb, 0x4d, 0xbb, 0x8c, 0x74, 0x25, 0xa0, 0x35, 0xc8, 0xaa, 0x14, 0x20, 0x28, 0x2c, 0x1e, 0xb6, + 0x00, 0x7d, 0xb7, 0xe1, 0x7a, 0x42, 0x74, 0x2b, 0x2c, 0xec, 0xdd, 0x23, 0x06, 0x82, 0x5a, 0xd4, + 0xa9, 0x15, 0xb2, 0x83, 0x05, 0x85, 0x7d, 0xb5, 0x2f, 0x88, 0x11, 0xd1, 0x06, 0xe4, 0xf6, 0xbd, + 0x88, 0x3c, 0x76, 0x32, 0x3a, 0xd5, 0x77, 0x2f, 0x2e, 0x22, 0xa2, 0xb0, 0x32, 0x86, 0xa7, 0xf6, + 0x43, 0x64, 0xb4, 0x08, 0x89, 0x86, 0x56, 0xc8, 0x31, 0x19, 0x57, 0x06, 0xc5, 0xcb, 0x55, 0xc6, + 0x70, 0xa2, 0xa1, 0x51, 0xd8, 0xcf, 0x43, 0x89, 0x0e, 0xcd, 0x42, 0xbe, 0xaf, 0xdb, 0xe8, 0x0e, + 0xe7, 0xaa, 0x8c, 0x61, 0x16, 0xf4, 0x44, 0xcb, 0xdb, 0x86, 0xbc, 0xcd, 0x8f, 0x37, 0xbd, 0x43, + 0x7c, 0x99, 0x49, 0xb9, 0x15, 0xed, 0x7c, 0x7a, 0xce, 0xf1, 0x2b, 0x63, 0x38, 0x67, 0x87, 0xe9, + 0xe8, 0x1b, 0x70, 0xae, 0x5b, 0xa2, 0x30, 0xee, 0x99, 0x1e, 0x5f, 0x14, 0x2d, 0xb7, 0xdb, 0xc6, + 0x91, 0xdd, 0x93, 0x88, 0xde, 0x82, 0x71, 0xde, 0x6b, 0x88, 0x89, 0x9c, 0x8f, 0xda, 0xca, 0xe8, + 0xee, 0x30, 0x9e, 0x9f, 0x8e, 0x37, 0x57, 0x9c, 0xeb, 0x29, 0x4d, 0xab, 0x51, 0x98, 0xed, 0x3b, + 0xde, 0x7a, 0xcf, 0x29, 0xe9, 0x78, 0x73, 0x03, 0x2a, 0xed, 0x77, 0x9b, 0xa7, 0x88, 0x63, 0xa0, + 0x73, 0x7d, 0xfb, 0x3d, 0xe2, 0xb8, 0x8f, 0xf6, 0xbb, 0x1d, 0x22, 0xd3, 0xaa, 0xd9, 0xfc, 0x7a, + 0x92, 0xc2, 0x86, 0xf1, 0xf9, 0xbe, 0x55, 0xeb, 0xbd, 0x72, 0x45, 0xab, 0x66, 0x07, 0x54, 0xf4, + 0x04, 0x64, 0x71, 0x13, 0x45, 0xf1, 0x8e, 0x0f, 0x0a, 0x17, 0x98, 0xbc, 0x3b, 0x91, 0xde, 0x32, + 0xea, 0xdc, 0xa4, 0x32, 0x86, 0xa7, 0xb5, 0xee, 0x14, 0xea, 0x2c, 0x98, 0x3c, 0x45, 0x0b, 0xae, + 0xf0, 0x14, 0x0a, 0x7d, 0x9d, 0x45, 0x9f, 0xfb, 0x46, 0xd4, 0x59, 0x68, 0xa7, 0x92, 0xa8, 0x19, + 0x1b, 0xa6, 0xe1, 0x32, 0xc7, 0x3e, 0xd7, 0xd7, 0x8c, 0xbb, 0xef, 0x46, 0x53, 0x33, 0x36, 0x38, + 0x85, 0x9a, 0xb1, 0x2b, 0xce, 0x08, 0x45, 0x77, 0x5c, 0xe9, 0x6b, 0xc6, 0x51, 0x87, 0x89, 0xd4, + 0x8c, 0xdd, 0x30, 0x9d, 0x9a, 0x31, 0x77, 0x10, 0xa7, 0xe4, 0xbe, 0xd6, 0xd7, 0x8c, 0xfb, 0x86, + 0x9e, 0x53, 0x33, 0x56, 0x7b, 0x12, 0xd1, 0x0a, 0xc5, 0x6b, 0x14, 0x9b, 0x18, 0xe6, 0x9e, 0x55, + 0xb8, 0xda, 0x77, 0xfe, 0x39, 0x7d, 0x4a, 0x58, 0x61, 0x60, 0x4d, 0xd0, 0xa8, 0x23, 0x63, 0x20, + 0x56, 0x61, 0x17, 0xbe, 0x0a, 0xf3, 0x7d, 0x1d, 0x59, 0xcf, 0x4e, 0x2f, 0x75, 0x64, 0x07, 0x3e, + 0x91, 0x4e, 0x64, 0x7c, 0x4f, 0xa6, 0xb0, 0x30, 0x64, 0xed, 0x1f, 0x9a, 0xc8, 0x38, 0x07, 0x5a, + 0x86, 0xcc, 0xf3, 0x0e, 0xb1, 0x8f, 0x98, 0x1b, 0xba, 0xd6, 0x17, 0x79, 0x9e, 0x0a, 0xfb, 0xab, + 0x8c, 0xe1, 0xf4, 0x73, 0x41, 0xa2, 0xc5, 0xf3, 0x85, 0x71, 0xa1, 0xd8, 0xb7, 0xf8, 0xae, 0xad, + 0x10, 0x5a, 0x3c, 0xe7, 0x40, 0x1a, 0x9c, 0xe7, 0x7d, 0x25, 0x62, 0xd5, 0x6d, 0x11, 0x14, 0x5e, + 0x78, 0x9d, 0x89, 0xea, 0xbb, 0xcc, 0x8c, 0x8c, 0x9b, 0xaf, 0x8c, 0xe1, 0x59, 0xb5, 0x37, 0x95, + 0x0e, 0x78, 0x31, 0xf5, 0xf0, 0xc5, 0x69, 0xe1, 0x7a, 0xdf, 0x01, 0x1f, 0xb1, 0x9c, 0xa7, 0x03, + 0x5e, 0x0d, 0x91, 0xf9, 0x04, 0xa4, 0x2b, 0x8e, 0xe3, 0xd2, 0xa5, 0x60, 0xe1, 0xc6, 0x80, 0x09, + 0xe8, 0xd4, 0x12, 0x95, 0x4f, 0x40, 0x7a, 0x8d, 0x73, 0x52, 0x41, 0x5a, 0x93, 0xa8, 0xb6, 0x70, + 0xb3, 0x37, 0xfb, 0x0a, 0xea, 0xb9, 0x6f, 0x4c, 0x05, 0x69, 0x3e, 0x91, 0x4e, 0xd8, 0xb6, 0x77, + 0x85, 0x4d, 0xe0, 0xc7, 0x5b, 0x7d, 0x27, 0xec, 0xc8, 0x9b, 0x76, 0x74, 0xc2, 0xb6, 0xbb, 0x12, + 0xd0, 0x17, 0x61, 0x52, 0xdc, 0x07, 0x2a, 0xdc, 0x1e, 0x80, 0x6a, 0xc3, 0xeb, 0x56, 0x3a, 0xae, + 0x05, 0x0f, 0xf7, 0xb2, 0xfc, 0x3a, 0x11, 0x6f, 0xde, 0x9d, 0x01, 0x5e, 0xb6, 0x67, 0xc9, 0xc8, + 0xbd, 0x6c, 0x40, 0xa6, 0x5e, 0x96, 0xdb, 0xa9, 0x98, 0xeb, 0xee, 0xf6, 0xf5, 0xb2, 0xbd, 0x01, + 0x7b, 0xd4, 0xcb, 0x3e, 0x0f, 0xa8, 0x48, 0x87, 0x0b, 0x0d, 0xe2, 0x2a, 0xde, 0x11, 0x2c, 0x8b, + 0x19, 0xe5, 0xd3, 0xd2, 0x1b, 0x7d, 0xad, 0xae, 0xff, 0x0a, 0x90, 0x5a, 0x5d, 0xa3, 0x37, 0x95, + 0xf6, 0x2e, 0x8f, 0x61, 0xe1, 0x1d, 0xf2, 0x99, 0xbe, 0xbd, 0xdb, 0xb3, 0x60, 0xa9, 0x78, 0x8f, + 0xcf, 0x70, 0x58, 0x3f, 0x29, 0xe2, 0x81, 0x1e, 0xa5, 0xd2, 0xd3, 0xb2, 0xfc, 0x28, 0x95, 0xbe, + 0x28, 0x17, 0x1e, 0xa5, 0xd2, 0x97, 0xe4, 0xb9, 0x47, 0xa9, 0xf4, 0x65, 0xf9, 0x4a, 0xf1, 0x8f, + 0x2f, 0x42, 0xce, 0x5b, 0x5e, 0x70, 0x68, 0xbf, 0x14, 0x86, 0xf6, 0x57, 0xfb, 0x41, 0x7b, 0xb1, + 0x20, 0x11, 0xd8, 0x7e, 0x29, 0x8c, 0xed, 0xaf, 0xf6, 0xc3, 0xf6, 0x01, 0x0f, 0x05, 0xf7, 0xf5, + 0x7e, 0xe0, 0xfe, 0xce, 0x08, 0xe0, 0xde, 0x17, 0x75, 0x1a, 0xdd, 0xaf, 0xf4, 0xa2, 0xfb, 0xeb, + 0x83, 0xd1, 0xbd, 0x2f, 0x2a, 0x04, 0xef, 0xdf, 0x39, 0x05, 0xef, 0xaf, 0x0d, 0x80, 0xf7, 0x3e, + 0xbf, 0x87, 0xef, 0xd7, 0x23, 0xf1, 0xfd, 0xcd, 0x61, 0xf8, 0xde, 0x97, 0xd3, 0x05, 0xf0, 0xdf, + 0xec, 0x02, 0xf8, 0xf3, 0x7d, 0x01, 0xbe, 0xcf, 0xcd, 0x11, 0xfe, 0xfb, 0xfd, 0x11, 0xfe, 0x1b, + 0x23, 0x21, 0x7c, 0x5f, 0x5e, 0x2f, 0xc4, 0xaf, 0xf7, 0x83, 0xf8, 0x77, 0x46, 0x80, 0xf8, 0x41, + 0xc7, 0x9d, 0xc2, 0xf8, 0x95, 0x28, 0x8c, 0x7f, 0x63, 0x08, 0xc6, 0xf7, 0xa5, 0x85, 0x41, 0x7e, + 0x25, 0x0a, 0xe4, 0xdf, 0x18, 0x02, 0xf2, 0x4f, 0x49, 0xe2, 0xc3, 0x70, 0x33, 0x1a, 0xe5, 0xdf, + 0x1a, 0x8a, 0xf2, 0x7d, 0x69, 0xdd, 0x30, 0xff, 0x5e, 0x08, 0xe6, 0xbf, 0xd6, 0x07, 0xe6, 0xfb, + 0xac, 0x14, 0xe7, 0x7f, 0xb9, 0x07, 0xe7, 0x17, 0x07, 0xe1, 0x7c, 0x9f, 0xd7, 0x07, 0xfa, 0xef, + 0xf6, 0x01, 0xfa, 0xb7, 0x87, 0x03, 0x7d, 0x5f, 0xd8, 0x29, 0xa4, 0xaf, 0x0e, 0x44, 0xfa, 0x9f, + 0x1d, 0x11, 0xe9, 0xfb, 0xd2, 0xa3, 0xa0, 0xfe, 0xe7, 0xbb, 0xa1, 0xfe, 0x42, 0x7f, 0xa8, 0xef, + 0x8b, 0x11, 0x58, 0x7f, 0x3d, 0x12, 0xeb, 0xdf, 0x1c, 0x86, 0xf5, 0x83, 0xb1, 0x17, 0x06, 0xfb, + 0x9b, 0xd1, 0x60, 0xff, 0xd6, 0x50, 0xb0, 0x1f, 0x74, 0x7f, 0x17, 0xda, 0x5f, 0x8f, 0x44, 0xfb, + 0x37, 0x87, 0xa1, 0xfd, 0xa0, 0x72, 0x61, 0xb8, 0xff, 0x5e, 0x5f, 0xb8, 0x7f, 0x77, 0x14, 0xb8, + 0xef, 0x0b, 0xed, 0xc1, 0xfb, 0xef, 0xf7, 0xc7, 0xfb, 0x6f, 0x8c, 0x84, 0xf7, 0x03, 0xd7, 0xd1, + 0x03, 0xf8, 0xbf, 0xdc, 0x03, 0xf8, 0x8b, 0x83, 0x00, 0x7f, 0x60, 0xcf, 0x1e, 0xe2, 0x57, 0x07, + 0xe2, 0xf3, 0xcf, 0x8e, 0x88, 0xcf, 0x03, 0xe3, 0x8b, 0x00, 0xe8, 0xab, 0x11, 0x00, 0xfd, 0xfa, + 0x60, 0x80, 0x1e, 0x4c, 0x21, 0x01, 0x42, 0xaf, 0x44, 0x21, 0xf4, 0x1b, 0x43, 0x10, 0x7a, 0xe0, + 0x85, 0x42, 0x10, 0xfd, 0x9d, 0x53, 0x10, 0xfd, 0xda, 0xd0, 0xf3, 0xe5, 0x10, 0x46, 0x2f, 0xf5, + 0x62, 0xf4, 0xd7, 0x07, 0x62, 0x74, 0x5f, 0x42, 0x00, 0xd2, 0xdf, 0x39, 0x05, 0xd2, 0xaf, 0x0d, + 0x00, 0xe9, 0x41, 0x05, 0x04, 0x4a, 0xd7, 0x07, 0xa3, 0xf4, 0xc5, 0x51, 0x51, 0xba, 0x2f, 0x38, + 0x12, 0xa6, 0x6f, 0x46, 0xc3, 0xf4, 0x5b, 0x23, 0x1e, 0x35, 0xf5, 0xe0, 0xf4, 0x4a, 0x14, 0x4e, + 0xbf, 0x31, 0x04, 0xa7, 0x87, 0xe7, 0x10, 0x1f, 0xa8, 0x57, 0xa2, 0x80, 0xfa, 0x8d, 0x21, 0x40, + 0x3d, 0x90, 0x14, 0x42, 0xea, 0xf5, 0x7e, 0x48, 0xfd, 0xce, 0x08, 0x48, 0x3d, 0x98, 0x77, 0x4f, + 0x41, 0xf5, 0x2f, 0x9d, 0x86, 0xea, 0xc5, 0x41, 0x50, 0x3d, 0x18, 0x91, 0x1e, 0x56, 0xdf, 0x8c, + 0xc6, 0xea, 0xb7, 0x86, 0x62, 0xf5, 0xb0, 0x93, 0x0c, 0x81, 0xf5, 0xf5, 0x48, 0xb0, 0x7e, 0x73, + 0x18, 0x58, 0x0f, 0x9c, 0x64, 0x18, 0xad, 0x93, 0x21, 0x68, 0x7d, 0x71, 0x54, 0xb4, 0x1e, 0x58, + 0x5f, 0x14, 0x5c, 0xaf, 0x44, 0xc1, 0xf5, 0x1b, 0x43, 0xe0, 0x7a, 0xd0, 0xc7, 0xaf, 0x80, 0xd7, + 0x1f, 0xa5, 0xd2, 0x57, 0xe4, 0xd7, 0x8a, 0x7f, 0x31, 0x0e, 0x13, 0x15, 0x2f, 0x96, 0x21, 0x74, + 0xb3, 0x54, 0x3a, 0xcb, 0xcd, 0x52, 0xb4, 0x42, 0x4d, 0x80, 0x0d, 0x24, 0x81, 0xe0, 0x07, 0xdc, + 0xd3, 0xee, 0x39, 0x41, 0xf2, 0x58, 0xcf, 0x10, 0x77, 0x8f, 0xde, 0x84, 0x5c, 0xc7, 0x21, 0xb6, + 0xd2, 0xb6, 0x0d, 0xcb, 0x36, 0x5c, 0x1e, 0xb9, 0x26, 0x95, 0xe4, 0x8f, 0x8f, 0xe7, 0xa7, 0x76, + 0x1c, 0x62, 0x6f, 0x0b, 0x3a, 0x9e, 0xea, 0x84, 0xbe, 0xbc, 0xd7, 0x41, 0xc7, 0x47, 0x7f, 0x1d, + 0xf4, 0x5d, 0x90, 0x6d, 0xa2, 0xea, 0x5d, 0x53, 0x1a, 0xbf, 0x81, 0x19, 0x3d, 0xfb, 0xaa, 0x7a, + 0x68, 0xd6, 0x62, 0x37, 0x31, 0xa7, 0xed, 0x6e, 0x22, 0xba, 0x0f, 0xe7, 0x5b, 0xea, 0x21, 0xbf, + 0x63, 0xec, 0xa1, 0x04, 0x16, 0xd3, 0x91, 0x66, 0xa1, 0x47, 0xa8, 0xa5, 0x1e, 0xb2, 0xa7, 0x46, + 0x79, 0x12, 0x7b, 0x9a, 0xec, 0x06, 0xe4, 0x75, 0xc3, 0x71, 0x0d, 0x53, 0xf3, 0x5e, 0xe7, 0xe0, + 0xcf, 0x60, 0xe4, 0x3c, 0x2a, 0x7f, 0x25, 0xe3, 0x2e, 0xcc, 0x88, 0x48, 0xab, 0xe0, 0xf1, 0x51, + 0x86, 0x87, 0xd3, 0xb4, 0x16, 0x34, 0x21, 0x78, 0x2b, 0xb6, 0x0c, 0xd3, 0x0d, 0xd5, 0x25, 0x07, + 0xea, 0x91, 0x62, 0x5a, 0x3a, 0xd3, 0x7d, 0x96, 0xbd, 0x51, 0x70, 0xf9, 0xe4, 0x78, 0x3e, 0xb7, + 0xc6, 0x93, 0x36, 0x2d, 0x9d, 0xf7, 0xc0, 0x04, 0xff, 0x87, 0x73, 0x8d, 0x50, 0x82, 0x8e, 0x96, + 0x61, 0x8a, 0xbd, 0xbb, 0x64, 0xf1, 0xc7, 0xbd, 0x04, 0xca, 0xed, 0x77, 0x98, 0x20, 0x9e, 0x00, + 0xc3, 0xec, 0xad, 0x26, 0xef, 0x3d, 0xb0, 0x5b, 0x30, 0xad, 0x3a, 0x47, 0xa6, 0xc6, 0x34, 0x4c, + 0x4c, 0xa7, 0xe3, 0x30, 0x98, 0x9b, 0xc6, 0x79, 0x46, 0x2e, 0x7b, 0x54, 0xf1, 0xc8, 0xc7, 0xaf, + 0x49, 0x30, 0xd5, 0x15, 0x7e, 0xf8, 0xce, 0xa9, 0x53, 0xb0, 0x4b, 0xd1, 0x10, 0xbb, 0x5f, 0x38, + 0x4f, 0x5a, 0xf4, 0x80, 0x17, 0x62, 0x30, 0xdf, 0x1f, 0xa2, 0xb1, 0x45, 0xae, 0x17, 0x94, 0xe1, + 0xb1, 0xbd, 0x9d, 0xfa, 0x8d, 0xef, 0xce, 0x8f, 0x15, 0x7f, 0x9c, 0x84, 0x5c, 0x77, 0x98, 0x61, + 0xf5, 0x54, 0xbd, 0xa2, 0xdc, 0x5a, 0x17, 0x47, 0xff, 0x5a, 0xae, 0x40, 0xc6, 0x7b, 0xad, 0xc9, + 0xab, 0xe6, 0xc2, 0x80, 0xb3, 0xbe, 0x70, 0x3d, 0x03, 0xc6, 0xb9, 0x1f, 0x26, 0xfc, 0x91, 0xbf, + 0x08, 0xe3, 0xec, 0xdd, 0x63, 0x51, 0xb5, 0xa8, 0x88, 0xfc, 0x55, 0x9a, 0x8e, 0x79, 0x36, 0xea, + 0x29, 0xea, 0x67, 0xba, 0x83, 0xee, 0x13, 0xce, 0xf0, 0x2e, 0xef, 0x19, 0xef, 0x4a, 0xb3, 0x93, + 0xbf, 0x66, 0x93, 0x68, 0xae, 0x78, 0x3e, 0xd9, 0x7b, 0x7d, 0xf7, 0xfa, 0x69, 0x11, 0xe2, 0xb1, + 0xe5, 0x45, 0x2c, 0x1e, 0x5b, 0x0e, 0x45, 0x7d, 0xe4, 0x7d, 0x11, 0x6c, 0x60, 0xf1, 0xd8, 0x20, + 0xd1, 0xd5, 0xbf, 0x20, 0x81, 0xcc, 0x86, 0xd1, 0x43, 0x42, 0xf4, 0x58, 0xac, 0xd0, 0x0b, 0x48, + 0x49, 0x8c, 0x1c, 0x90, 0x52, 0x54, 0x21, 0xef, 0xd7, 0x81, 0x3f, 0x43, 0x3a, 0xe0, 0x09, 0x81, + 0x33, 0x5d, 0xed, 0x2a, 0xfe, 0x96, 0x04, 0xb3, 0x7e, 0x19, 0x0c, 0xab, 0xf3, 0x20, 0xe8, 0x33, + 0x84, 0x08, 0x62, 0xf6, 0xc4, 0x32, 0x5d, 0xab, 0xb1, 0xeb, 0x3e, 0x23, 0x59, 0x10, 0x12, 0xc7, + 0xca, 0x20, 0xd6, 0x80, 0x7a, 0xbd, 0xc6, 0x1e, 0x5f, 0xe6, 0xff, 0x9d, 0xe2, 0xc3, 0x90, 0x06, + 0x98, 0xb1, 0xd2, 0x66, 0x8e, 0x64, 0xd5, 0x5e, 0x33, 0x59, 0xe6, 0xe2, 0x8f, 0xa4, 0xb0, 0xa0, + 0x17, 0x74, 0xee, 0x7f, 0x00, 0xc9, 0x17, 0x6a, 0x73, 0xd0, 0xa9, 0x7a, 0x97, 0xea, 0x31, 0xcd, + 0x8d, 0x1e, 0x02, 0xbf, 0x7b, 0xc1, 0x23, 0xc7, 0x13, 0xfd, 0x17, 0x68, 0xbd, 0x2a, 0xc5, 0x21, + 0x4e, 0xf4, 0x96, 0xd7, 0x8a, 0xe4, 0xf0, 0xe2, 0xc3, 0x83, 0xf4, 0xed, 0xd4, 0x87, 0xdf, 0x9d, + 0x97, 0xee, 0xd6, 0x60, 0x36, 0x62, 0x12, 0x42, 0x79, 0x80, 0xf2, 0xd6, 0x66, 0xad, 0x5a, 0xab, + 0xaf, 0x6e, 0xd6, 0xbd, 0x17, 0x9e, 0x97, 0x57, 0x94, 0x9d, 0xcd, 0xf2, 0xd6, 0xc6, 0x46, 0xb5, + 0x5e, 0x5f, 0x5d, 0x91, 0x25, 0x24, 0xc3, 0x54, 0x75, 0x33, 0x94, 0x4f, 0x3c, 0xec, 0x7c, 0xf7, + 0xff, 0x03, 0x04, 0x0f, 0xe7, 0x51, 0x59, 0xeb, 0xab, 0x4f, 0x95, 0x27, 0xcb, 0x8f, 0x77, 0x56, + 0x6b, 0xf2, 0x18, 0x42, 0x90, 0x2f, 0x2d, 0xd7, 0xcb, 0x15, 0x05, 0xaf, 0xd6, 0xb6, 0xb7, 0x36, + 0x6b, 0xab, 0xb2, 0x24, 0xf8, 0xde, 0x83, 0x6c, 0xe8, 0x4d, 0x02, 0x9a, 0x71, 0x7b, 0xa7, 0x56, + 0x51, 0xea, 0xd5, 0x8d, 0xd5, 0x5a, 0x7d, 0x79, 0x63, 0x5b, 0x1e, 0xa3, 0xc2, 0x18, 0x6d, 0xb9, + 0xb4, 0x85, 0xeb, 0xb2, 0xe4, 0x7f, 0xd7, 0xb7, 0x76, 0xca, 0x15, 0x39, 0xe1, 0x7f, 0xbf, 0xbb, + 0xb3, 0x8a, 0x9f, 0xca, 0x49, 0x21, 0x58, 0x85, 0xf3, 0x91, 0x77, 0x44, 0x50, 0x16, 0x26, 0x77, + 0x4c, 0x76, 0x2f, 0x5e, 0x1e, 0x43, 0xb9, 0xd0, 0x35, 0x11, 0x59, 0x42, 0x69, 0x7e, 0x1d, 0x41, + 0x4e, 0xa0, 0x09, 0x48, 0xd4, 0x1e, 0xc8, 0x49, 0x34, 0x0d, 0xd9, 0xd0, 0x5d, 0x0b, 0x39, 0x85, + 0x32, 0x22, 0xe0, 0x5e, 0x1e, 0xbf, 0x7b, 0x0d, 0x42, 0xf1, 0xcc, 0x08, 0x60, 0xe2, 0xb1, 0xea, + 0x12, 0xc7, 0x95, 0xc7, 0xd0, 0x24, 0x24, 0x97, 0x9b, 0x4d, 0x59, 0x5a, 0xfa, 0x2a, 0xa4, 0xbd, + 0x17, 0xb6, 0xd0, 0x63, 0x18, 0xe7, 0x6b, 0xae, 0xf9, 0xfe, 0x7e, 0x9e, 0xb9, 0x8a, 0xb9, 0x85, + 0x61, 0x13, 0x41, 0x71, 0xac, 0x74, 0xed, 0xc3, 0x7f, 0xbe, 0x3a, 0xf6, 0xe1, 0xc9, 0x55, 0xe9, + 0xa3, 0x93, 0xab, 0xd2, 0xdf, 0x9f, 0x5c, 0x95, 0xfe, 0xe9, 0xe4, 0xaa, 0xf4, 0x2b, 0xff, 0x72, + 0x75, 0xec, 0xfd, 0x49, 0xc1, 0xb2, 0x3b, 0xc1, 0xde, 0x7f, 0x7f, 0xf0, 0xdf, 0x01, 0x00, 0x00, + 0xff, 0xff, 0xfc, 0xbd, 0x73, 0xa8, 0x04, 0x5f, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index ef53c2accce0..3db2d56b3531 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1279,11 +1279,9 @@ message GetSnapshotForMergeRequest { message GetSnapshotForMergeResponse { ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - // Data is a snapshot of the right-hand range's data in the form of a - // BatchRepr, a serialized RocksDB Batch. - bytes data = 2; + reserved 2; - // MVCCStats are the MVCC statistics for data. + // MVCCStats are the MVCC statistics for the range. storage.engine.enginepb.MVCCStats mvcc_stats = 3 [ (gogoproto.nullable) = false, (gogoproto.customname) = "MVCCStats" diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 653cf4dcdb63..88875f6bd03f 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -6,6 +6,7 @@ package roachpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" +import cockroach_storage_engine_enginepb1 "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" import cockroach_storage_engine_enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -229,11 +230,9 @@ func (*SplitTrigger) Descriptor() ([]byte, []int) { return fileDescriptorData, [ // (right_desc). This information allows the final bookkeeping for the // merge to be completed and put into operation. type MergeTrigger struct { - LeftDesc RangeDescriptor `protobuf:"bytes,1,opt,name=left_desc,json=leftDesc" json:"left_desc"` - RightDesc RangeDescriptor `protobuf:"bytes,2,opt,name=right_desc,json=rightDesc" json:"right_desc"` - // A snapshot of the replicated data from the right-hand side of the merge - // as a serialized RocksDB write batch. - RightData []byte `protobuf:"bytes,3,opt,name=right_data,json=rightData,proto3" json:"right_data,omitempty"` + LeftDesc RangeDescriptor `protobuf:"bytes,1,opt,name=left_desc,json=leftDesc" json:"left_desc"` + RightDesc RangeDescriptor `protobuf:"bytes,2,opt,name=right_desc,json=rightDesc" json:"right_desc"` + RightMVCCStats cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,4,opt,name=right_mvcc_stats,json=rightMvccStats" json:"right_mvcc_stats"` } func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } @@ -743,7 +742,7 @@ func (this *MergeTrigger) Equal(that interface{}) bool { if !this.RightDesc.Equal(&that1.RightDesc) { return false } - if !bytes.Equal(this.RightData, that1.RightData) { + if !this.RightMVCCStats.Equal(&that1.RightMVCCStats) { return false } return true @@ -1201,12 +1200,14 @@ func (m *MergeTrigger) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n7 - if len(m.RightData) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintData(dAtA, i, uint64(len(m.RightData))) - i += copy(dAtA[i:], m.RightData) + dAtA[i] = 0x22 + i++ + i = encodeVarintData(dAtA, i, uint64(m.RightMVCCStats.Size())) + n8, err := m.RightMVCCStats.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } + i += n8 return i, nil } @@ -1233,11 +1234,11 @@ func (m *ChangeReplicasTrigger) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Replica.Size())) - n8, err := m.Replica.MarshalTo(dAtA[i:]) + n9, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n8 + i += n9 if len(m.UpdatedReplicas) > 0 { for _, msg := range m.UpdatedReplicas { dAtA[i] = 0x1a @@ -1287,11 +1288,11 @@ func (m *ModifiedSpanTrigger) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.NodeLivenessSpan.Size())) - n9, err := m.NodeLivenessSpan.MarshalTo(dAtA[i:]) + n10, err := m.NodeLivenessSpan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n9 + i += n10 } return i, nil } @@ -1315,41 +1316,41 @@ func (m *InternalCommitTrigger) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.SplitTrigger.Size())) - n10, err := m.SplitTrigger.MarshalTo(dAtA[i:]) + n11, err := m.SplitTrigger.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n10 + i += n11 } if m.MergeTrigger != nil { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.MergeTrigger.Size())) - n11, err := m.MergeTrigger.MarshalTo(dAtA[i:]) + n12, err := m.MergeTrigger.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n12 } if m.ChangeReplicasTrigger != nil { dAtA[i] = 0x1a i++ i = encodeVarintData(dAtA, i, uint64(m.ChangeReplicasTrigger.Size())) - n12, err := m.ChangeReplicasTrigger.MarshalTo(dAtA[i:]) + n13, err := m.ChangeReplicasTrigger.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n12 + i += n13 } if m.ModifiedSpanTrigger != nil { dAtA[i] = 0x22 i++ i = encodeVarintData(dAtA, i, uint64(m.ModifiedSpanTrigger.Size())) - n13, err := m.ModifiedSpanTrigger.MarshalTo(dAtA[i:]) + n14, err := m.ModifiedSpanTrigger.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n13 + i += n14 } return i, nil } @@ -1377,11 +1378,11 @@ func (m *ObservedTimestamp) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Timestamp.Size())) - n14, err := m.Timestamp.MarshalTo(dAtA[i:]) + n15, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n14 + i += n15 return i, nil } @@ -1403,11 +1404,11 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.TxnMeta.Size())) - n15, err := m.TxnMeta.MarshalTo(dAtA[i:]) + n16, err := m.TxnMeta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n15 + i += n16 if len(m.Name) > 0 { dAtA[i] = 0x12 i++ @@ -1422,27 +1423,27 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintData(dAtA, i, uint64(m.LastHeartbeat.Size())) - n16, err := m.LastHeartbeat.MarshalTo(dAtA[i:]) + n17, err := m.LastHeartbeat.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n16 + i += n17 dAtA[i] = 0x32 i++ i = encodeVarintData(dAtA, i, uint64(m.OrigTimestamp.Size())) - n17, err := m.OrigTimestamp.MarshalTo(dAtA[i:]) + n18, err := m.OrigTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n17 + i += n18 dAtA[i] = 0x3a i++ i = encodeVarintData(dAtA, i, uint64(m.MaxTimestamp.Size())) - n18, err := m.MaxTimestamp.MarshalTo(dAtA[i:]) + n19, err := m.MaxTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n18 + i += n19 if len(m.ObservedTimestamps) > 0 { for _, msg := range m.ObservedTimestamps { dAtA[i] = 0x42 @@ -1500,19 +1501,19 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintData(dAtA, i, uint64(m.EpochZeroTimestamp.Size())) - n19, err := m.EpochZeroTimestamp.MarshalTo(dAtA[i:]) + n20, err := m.EpochZeroTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n19 + i += n20 dAtA[i] = 0x7a i++ i = encodeVarintData(dAtA, i, uint64(m.RefreshedTimestamp.Size())) - n20, err := m.RefreshedTimestamp.MarshalTo(dAtA[i:]) + n21, err := m.RefreshedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n20 + i += n21 if m.OrigTimestampWasObserved { dAtA[i] = 0x80 i++ @@ -1546,19 +1547,19 @@ func (m *Intent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Span.Size())) - n21, err := m.Span.MarshalTo(dAtA[i:]) + n22, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n22 dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Txn.Size())) - n22, err := m.Txn.MarshalTo(dAtA[i:]) + n23, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n23 if m.Status != 0 { dAtA[i] = 0x18 i++ @@ -1614,48 +1615,48 @@ func (m *Lease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Start.Size())) - n23, err := m.Start.MarshalTo(dAtA[i:]) + n24, err := m.Start.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n23 + i += n24 if m.Expiration != nil { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Expiration.Size())) - n24, err := m.Expiration.MarshalTo(dAtA[i:]) + n25, err := m.Expiration.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n25 } dAtA[i] = 0x1a i++ i = encodeVarintData(dAtA, i, uint64(m.Replica.Size())) - n25, err := m.Replica.MarshalTo(dAtA[i:]) + n26, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n25 + i += n26 if m.DeprecatedStartStasis != nil { dAtA[i] = 0x22 i++ i = encodeVarintData(dAtA, i, uint64(m.DeprecatedStartStasis.Size())) - n26, err := m.DeprecatedStartStasis.MarshalTo(dAtA[i:]) + n27, err := m.DeprecatedStartStasis.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n27 } if m.ProposedTS != nil { dAtA[i] = 0x2a i++ i = encodeVarintData(dAtA, i, uint64(m.ProposedTS.Size())) - n27, err := m.ProposedTS.MarshalTo(dAtA[i:]) + n28, err := m.ProposedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n28 } if m.Epoch != 0 { dAtA[i] = 0x30 @@ -1694,11 +1695,11 @@ func (m *AbortSpanEntry) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Timestamp.Size())) - n28, err := m.Timestamp.MarshalTo(dAtA[i:]) + n29, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n29 if m.Priority != 0 { dAtA[i] = 0x18 i++ @@ -1725,11 +1726,11 @@ func (m *TxnCoordMeta) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Txn.Size())) - n29, err := m.Txn.MarshalTo(dAtA[i:]) + n30, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n30 if len(m.Intents) > 0 { for _, msg := range m.Intents { dAtA[i] = 0x12 @@ -2073,10 +2074,8 @@ func (m *MergeTrigger) Size() (n int) { n += 1 + l + sovData(uint64(l)) l = m.RightDesc.Size() n += 1 + l + sovData(uint64(l)) - l = len(m.RightData) - if l > 0 { - n += 1 + l + sovData(uint64(l)) - } + l = m.RightMVCCStats.Size() + n += 1 + l + sovData(uint64(l)) return n } @@ -2969,11 +2968,11 @@ func (m *MergeTrigger) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 3: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RightData", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RightMVCCStats", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowData @@ -2983,21 +2982,20 @@ func (m *MergeTrigger) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthData } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex > l { return io.ErrUnexpectedEOF } - m.RightData = append(m.RightData[:0], dAtA[iNdEx:postIndex]...) - if m.RightData == nil { - m.RightData = []byte{} + if err := m.RightMVCCStats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } iNdEx = postIndex default: @@ -4951,120 +4949,123 @@ var ( func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptorData) } var fileDescriptorData = []byte{ - // 1836 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcb, 0x6f, 0xdb, 0xc8, - 0x19, 0x37, 0xf5, 0xb0, 0xa8, 0xcf, 0x92, 0x4d, 0x8f, 0xe3, 0x44, 0xf5, 0x62, 0xa5, 0xac, 0xb6, - 0x68, 0x83, 0xa0, 0x2b, 0xa3, 0x49, 0x9f, 0x46, 0x5b, 0xc0, 0xb2, 0xb4, 0x09, 0x13, 0x5b, 0x36, - 0x28, 0x3a, 0x41, 0xb3, 0x58, 0xb0, 0x63, 0x72, 0x22, 0x11, 0x91, 0x48, 0x76, 0x66, 0xe4, 0x58, - 0xfd, 0x0b, 0xf6, 0xd6, 0x3d, 0xf4, 0xd0, 0x63, 0x80, 0x9e, 0xda, 0xbf, 0xa0, 0xf7, 0x1e, 0x9a, - 0x4b, 0x81, 0xbd, 0xb5, 0xe8, 0x41, 0x68, 0xdd, 0x4b, 0x8b, 0xde, 0x7a, 0x0c, 0xb0, 0x40, 0x31, - 0xc3, 0xa1, 0x44, 0xd7, 0x8a, 0x21, 0x37, 0x3d, 0xf4, 0x92, 0x0c, 0xbf, 0xc7, 0x6f, 0xbe, 0xf7, - 0x37, 0x32, 0x20, 0x1a, 0x62, 0xb7, 0x1f, 0x9d, 0x6c, 0x7b, 0x98, 0xe3, 0x46, 0x44, 0x43, 0x1e, - 0xa2, 0x75, 0x37, 0x74, 0x5f, 0x48, 0x7a, 0x43, 0x71, 0xb7, 0x6e, 0x26, 0x62, 0x43, 0xc2, 0xf1, - 0x4c, 0x74, 0xeb, 0x43, 0xc6, 0x43, 0x8a, 0x7b, 0x64, 0x9b, 0x04, 0x3d, 0x3f, 0x48, 0xfe, 0x13, - 0x72, 0xa7, 0xae, 0x7b, 0x5f, 0x09, 0x55, 0x46, 0xdc, 0x1f, 0x6c, 0xf7, 0x07, 0xee, 0x36, 0xf7, - 0x87, 0x84, 0x71, 0x3c, 0x8c, 0x14, 0xe7, 0x46, 0x2f, 0xec, 0x85, 0xf2, 0xb8, 0x2d, 0x4e, 0x31, - 0xb5, 0xfe, 0x29, 0xe4, 0xba, 0x11, 0x0e, 0xd0, 0x57, 0x20, 0xfb, 0x82, 0x8c, 0x2b, 0xd9, 0xdb, - 0xda, 0x9d, 0x52, 0xb3, 0xf0, 0x66, 0x52, 0xcb, 0x3e, 0x26, 0x63, 0x4b, 0xd0, 0xd0, 0x6d, 0x28, - 0x90, 0xc0, 0x73, 0x04, 0x3b, 0x77, 0x91, 0xbd, 0x4c, 0x02, 0xef, 0x31, 0x19, 0xef, 0x94, 0x7e, - 0xf9, 0xaa, 0xb6, 0xf4, 0xdb, 0x57, 0x35, 0xed, 0xef, 0xaf, 0x6a, 0xda, 0xa3, 0x9c, 0xae, 0x19, - 0x99, 0x47, 0x39, 0x3d, 0x63, 0x64, 0xeb, 0x43, 0xc8, 0x3f, 0xc1, 0x83, 0x11, 0x41, 0xef, 0x41, - 0x91, 0xe2, 0x97, 0xce, 0xc9, 0x98, 0x13, 0x56, 0xd1, 0x04, 0x8c, 0xa5, 0x53, 0xfc, 0xb2, 0x29, - 0xbe, 0xd1, 0x2e, 0x14, 0xa7, 0xd6, 0x56, 0x32, 0xb7, 0xb5, 0x3b, 0x2b, 0xf7, 0xde, 0x6f, 0xcc, - 0x02, 0x23, 0x5c, 0x6a, 0xf4, 0x07, 0x6e, 0xc3, 0x4e, 0x84, 0x9a, 0xb9, 0xd7, 0x93, 0xda, 0x92, - 0x35, 0xd3, 0xda, 0xc9, 0x89, 0xab, 0xeb, 0x9f, 0x80, 0xfe, 0x98, 0x8c, 0xe3, 0x1b, 0x95, 0x47, - 0xda, 0x1c, 0x8f, 0xbe, 0x05, 0xf9, 0x53, 0x21, 0xa3, 0xee, 0xaa, 0x34, 0x2e, 0x25, 0xa1, 0x21, - 0x31, 0xd4, 0x35, 0xb1, 0x70, 0xfd, 0x8f, 0x1a, 0x40, 0x97, 0x87, 0x94, 0x98, 0x1e, 0x09, 0x38, - 0xea, 0x01, 0xb8, 0x83, 0x11, 0xe3, 0x84, 0x3a, 0xbe, 0xa7, 0xae, 0x79, 0x28, 0xe4, 0xff, 0x3c, - 0xa9, 0xdd, 0xef, 0xf9, 0xbc, 0x3f, 0x3a, 0x69, 0xb8, 0xe1, 0x70, 0x7b, 0x8a, 0xed, 0x9d, 0xcc, - 0xce, 0xdb, 0xd1, 0x8b, 0xde, 0xb6, 0x4c, 0xd5, 0x68, 0xe4, 0x7b, 0x8d, 0xe3, 0x63, 0xb3, 0x75, - 0x3e, 0xa9, 0x15, 0xf7, 0x62, 0x40, 0xb3, 0x65, 0x15, 0x15, 0xb6, 0xe9, 0xa1, 0x8f, 0xa0, 0x10, - 0x84, 0x1e, 0x11, 0xb7, 0x08, 0x7b, 0xf3, 0xcd, 0x1b, 0xe7, 0x93, 0xda, 0x72, 0x27, 0xf4, 0x88, - 0xd9, 0x7a, 0x33, 0x3d, 0x59, 0xcb, 0x42, 0xc8, 0xf4, 0xd0, 0x37, 0x41, 0x17, 0x85, 0x22, 0xe5, - 0xb3, 0x52, 0xfe, 0xe6, 0xf9, 0xa4, 0x56, 0x88, 0x2d, 0x17, 0x0a, 0xc9, 0xd1, 0x2a, 0xb0, 0xd8, - 0x9b, 0xfa, 0x6f, 0x34, 0x28, 0x75, 0xa3, 0x81, 0xcf, 0x6d, 0xea, 0xf7, 0x7a, 0x84, 0xa2, 0x36, - 0x14, 0x07, 0xe4, 0x39, 0x77, 0x3c, 0xc2, 0x5c, 0xe9, 0xda, 0xca, 0xbd, 0xfa, 0x9c, 0x20, 0x59, - 0x38, 0xe8, 0x91, 0x16, 0x61, 0x2e, 0xf5, 0x23, 0x1e, 0x52, 0x15, 0x2e, 0x5d, 0xa8, 0x0a, 0x2a, - 0x7a, 0x00, 0x40, 0xfd, 0x5e, 0x5f, 0xe1, 0x64, 0xae, 0x89, 0x53, 0x94, 0xba, 0x82, 0x1c, 0x67, - 0xf7, 0x51, 0x4e, 0xcf, 0x1a, 0xb9, 0xfa, 0xef, 0x34, 0x28, 0x1d, 0x10, 0xda, 0x23, 0xff, 0xa7, - 0xc6, 0xa2, 0xf7, 0xa7, 0x40, 0x98, 0xe3, 0xb8, 0xa3, 0x12, 0x36, 0xe6, 0x58, 0x55, 0xea, 0x1f, - 0x32, 0xb0, 0xb9, 0xd7, 0x17, 0x50, 0x16, 0x89, 0x06, 0xbe, 0x8b, 0xd9, 0xcc, 0x9d, 0x15, 0x57, - 0x32, 0x1c, 0x3e, 0x8e, 0x88, 0x74, 0x68, 0xf5, 0xde, 0x57, 0xe7, 0x19, 0x12, 0x2b, 0xc6, 0x28, - 0xf6, 0x38, 0x22, 0x16, 0xb8, 0xd3, 0x33, 0x6a, 0x41, 0x81, 0xc6, 0x02, 0xca, 0x97, 0x2b, 0x20, - 0x2e, 0x79, 0x93, 0xa8, 0xa2, 0x63, 0x30, 0x46, 0x91, 0x87, 0x39, 0xf1, 0x1c, 0x45, 0x62, 0x95, - 0xec, 0xed, 0xec, 0x35, 0xe1, 0xd6, 0x14, 0x46, 0xe2, 0x2a, 0xfa, 0x18, 0xd6, 0x02, 0x72, 0xc6, - 0x13, 0x4c, 0x51, 0xaa, 0x39, 0x59, 0xaa, 0xd5, 0xf3, 0x49, 0xad, 0xdc, 0x21, 0x67, 0x5c, 0x89, - 0xca, 0x82, 0x2d, 0x4e, 0x3f, 0xac, 0x72, 0x90, 0xe2, 0x79, 0x3b, 0xba, 0x18, 0x3c, 0x32, 0x9e, - 0x9f, 0x6b, 0xb0, 0x71, 0x10, 0x7a, 0xfe, 0x73, 0x9f, 0x78, 0x62, 0xa0, 0x25, 0xd1, 0xfc, 0x06, - 0x20, 0x36, 0x66, 0x9c, 0x0c, 0x1d, 0x37, 0x0c, 0x9e, 0xfb, 0x3d, 0x87, 0x45, 0x38, 0x90, 0x41, - 0xd5, 0x2d, 0x23, 0xe6, 0xec, 0x49, 0x86, 0x9c, 0x82, 0x6d, 0x40, 0xb2, 0xd5, 0x06, 0xfe, 0x29, - 0x09, 0x08, 0x63, 0xb1, 0x74, 0x1c, 0xbf, 0x5b, 0x73, 0x1c, 0x16, 0x4a, 0x96, 0x21, 0x54, 0xf6, - 0x95, 0x86, 0xa0, 0xa8, 0x14, 0xff, 0x33, 0x03, 0x9b, 0x66, 0xc0, 0x09, 0x0d, 0xf0, 0x60, 0x2f, - 0x1c, 0x0e, 0x67, 0xed, 0xd5, 0x82, 0x32, 0x13, 0xed, 0xe6, 0xf0, 0x98, 0xa0, 0xaa, 0xb6, 0x36, - 0xf7, 0x86, 0x59, 0x5b, 0x5a, 0x25, 0x96, 0x6e, 0xd2, 0x16, 0x94, 0x87, 0xa2, 0x0f, 0xa6, 0x28, - 0x99, 0xb7, 0xa2, 0xa4, 0xfb, 0xc5, 0x2a, 0x0d, 0xd3, 0xdd, 0xf3, 0x13, 0xb8, 0xa5, 0xca, 0x2d, - 0x49, 0xf0, 0x14, 0x2f, 0x2b, 0xf1, 0xee, 0xcc, 0xc1, 0x9b, 0x5b, 0xb9, 0xd6, 0xa6, 0x3b, 0xb7, - 0xa0, 0x9f, 0xc1, 0xe6, 0x50, 0x65, 0x46, 0xc6, 0x73, 0x8a, 0x9f, 0x93, 0xf8, 0x5f, 0x9b, 0x67, - 0xef, 0xe5, 0x4c, 0x5a, 0x1b, 0xc3, 0xcb, 0xc4, 0x1d, 0xfd, 0x33, 0xb5, 0x75, 0xea, 0x3f, 0xd7, - 0x60, 0xfd, 0xf0, 0x84, 0x11, 0x7a, 0x4a, 0xbc, 0xe9, 0x9e, 0x48, 0xcf, 0x4e, 0x6d, 0x81, 0xd9, - 0xf9, 0x3f, 0x58, 0x44, 0x7a, 0xb2, 0x07, 0xeb, 0x5f, 0x2e, 0xc3, 0x8a, 0x4d, 0x71, 0xc0, 0xb0, - 0xcb, 0xfd, 0x30, 0x40, 0x0f, 0x21, 0x27, 0x36, 0xba, 0x4a, 0xf6, 0xdd, 0x14, 0xae, 0x5a, 0xec, - 0x8d, 0x78, 0xa3, 0x37, 0x92, 0xc5, 0xde, 0xb0, 0xcf, 0x82, 0x03, 0xc2, 0x71, 0x53, 0x17, 0x97, - 0x7c, 0x31, 0xa9, 0x69, 0x96, 0x44, 0x40, 0x08, 0x72, 0x01, 0x1e, 0xc6, 0xeb, 0xab, 0x68, 0xc9, - 0x33, 0xfa, 0x01, 0x2c, 0x33, 0x8e, 0xf9, 0x88, 0xc9, 0xb0, 0xce, 0x9f, 0x18, 0x29, 0x6b, 0xba, - 0x52, 0xd6, 0x52, 0x3a, 0xe8, 0x11, 0xac, 0x0e, 0x30, 0xe3, 0x4e, 0x9f, 0x60, 0xca, 0x4f, 0x08, - 0xe6, 0x95, 0xfc, 0xe2, 0xde, 0x97, 0x85, 0xea, 0xc3, 0x44, 0x53, 0x60, 0x85, 0xd4, 0xef, 0x39, - 0xb3, 0x48, 0x2e, 0x5f, 0x03, 0x4b, 0xa8, 0xce, 0xf2, 0xf7, 0x10, 0xca, 0x43, 0x7c, 0x96, 0x82, - 0x2a, 0x2c, 0x0e, 0x55, 0x1a, 0xe2, 0xb3, 0x19, 0xd2, 0x27, 0xb0, 0x11, 0xaa, 0xf2, 0x98, 0xc1, - 0xb1, 0x8a, 0xfe, 0xd6, 0x61, 0x76, 0xa9, 0x98, 0x14, 0x2c, 0x0a, 0xff, 0x93, 0xc1, 0x50, 0x05, - 0x0a, 0x2f, 0xa9, 0xcf, 0xfd, 0xa0, 0x57, 0x29, 0xca, 0xd1, 0x92, 0x7c, 0xa2, 0xef, 0x42, 0xc1, - 0x0f, 0x38, 0x09, 0x38, 0xab, 0xac, 0xc8, 0xab, 0xde, 0x36, 0x46, 0x92, 0xc9, 0xab, 0xa4, 0x51, - 0x1d, 0xca, 0x02, 0x83, 0x38, 0x3c, 0x0c, 0x9d, 0x70, 0xe0, 0x55, 0x4a, 0x12, 0x78, 0x45, 0x12, - 0xed, 0x30, 0x3c, 0x1c, 0x78, 0x42, 0x86, 0x12, 0x4e, 0xc7, 0x4e, 0x18, 0x38, 0xd1, 0x88, 0xf5, - 0x2b, 0xe5, 0x58, 0x46, 0x12, 0x0f, 0x83, 0xa3, 0x11, 0xeb, 0xa3, 0x63, 0xb8, 0x41, 0xa2, 0xd0, - 0xed, 0x3b, 0x3f, 0x23, 0x34, 0x4c, 0x05, 0x72, 0x75, 0xf1, 0x40, 0x22, 0x09, 0xf0, 0x8c, 0xd0, - 0x70, 0x16, 0x4e, 0x1b, 0x36, 0x28, 0x79, 0x4e, 0x09, 0xeb, 0xa7, 0xe3, 0x59, 0x59, 0xbb, 0x06, - 0xea, 0x54, 0x7f, 0x86, 0xfa, 0x43, 0x78, 0xef, 0x62, 0xe9, 0x38, 0x2f, 0x31, 0x73, 0x92, 0x80, - 0x57, 0x0c, 0xe9, 0x5e, 0xe5, 0x42, 0x89, 0x3c, 0xc5, 0x2c, 0xc9, 0xd4, 0xc5, 0x77, 0x68, 0xfd, - 0xf7, 0x1a, 0x2c, 0x9b, 0x32, 0x9a, 0xe8, 0xdb, 0x90, 0x9b, 0xce, 0xfd, 0x2b, 0x52, 0x90, 0xea, - 0x33, 0x21, 0x8e, 0x9a, 0x90, 0xe5, 0x67, 0xc9, 0xfc, 0xbf, 0x4e, 0xc3, 0xc6, 0x1e, 0x0a, 0xe5, - 0x54, 0x5f, 0x66, 0xaf, 0xdf, 0x97, 0x6a, 0x93, 0x3c, 0x80, 0xd5, 0x2e, 0xf9, 0xe9, 0x88, 0x04, - 0x2e, 0xf1, 0x9e, 0x8a, 0xfc, 0x5f, 0xf5, 0xb8, 0xdd, 0x02, 0x9d, 0x29, 0xe1, 0xf8, 0xbd, 0x68, - 0x4d, 0xbf, 0xeb, 0x5f, 0x66, 0x21, 0xbf, 0x4f, 0x30, 0x23, 0xe8, 0xfb, 0x90, 0x67, 0x1c, 0x53, - 0xae, 0x42, 0xb2, 0x50, 0xc6, 0x62, 0x0d, 0xf4, 0x29, 0x00, 0x39, 0x8b, 0x7c, 0x8a, 0x85, 0xbd, - 0x8b, 0x4d, 0xc9, 0xea, 0xbf, 0x26, 0xb5, 0xad, 0xd4, 0x43, 0x78, 0xa7, 0x4e, 0x71, 0xe0, 0x05, - 0xa3, 0xc1, 0x00, 0x9f, 0x0c, 0x48, 0xdd, 0x4a, 0x01, 0xa6, 0x1f, 0x2e, 0xd9, 0xff, 0xfe, 0xe1, - 0x32, 0x82, 0x5b, 0x1e, 0x89, 0x28, 0x71, 0xe5, 0xdb, 0x45, 0x1a, 0x2e, 0xfe, 0x65, 0x3e, 0x53, - 0x6b, 0xe7, 0x1d, 0x2d, 0xde, 0x9c, 0xa1, 0x77, 0x05, 0x78, 0x57, 0x62, 0xa3, 0x0e, 0xac, 0x44, - 0x34, 0x8c, 0x42, 0x26, 0xba, 0x82, 0x2d, 0x36, 0x44, 0x57, 0xcf, 0x27, 0x35, 0x38, 0x52, 0x5a, - 0x76, 0xd7, 0x82, 0x04, 0xc1, 0x66, 0xe8, 0x06, 0xe4, 0x65, 0xf3, 0xc9, 0x11, 0x9a, 0xb5, 0xe2, - 0x0f, 0xf4, 0x51, 0x2a, 0xc5, 0x62, 0x20, 0x66, 0x9b, 0xeb, 0x6f, 0x26, 0xb5, 0xb2, 0xcc, 0x6c, - 0x52, 0x28, 0xb3, 0xac, 0xc7, 0xaf, 0x24, 0xd1, 0x16, 0xf5, 0x5f, 0x68, 0xb0, 0xba, 0x7b, 0x12, - 0x52, 0x2e, 0xca, 0xbd, 0x1d, 0x70, 0x3a, 0xbe, 0xaa, 0x92, 0xde, 0x7d, 0x1b, 0x8a, 0x62, 0x8c, - 0xa8, 0x1f, 0x52, 0x9f, 0xc7, 0xbf, 0x2d, 0xf3, 0xd6, 0xf4, 0x3b, 0xb5, 0x29, 0xff, 0x91, 0x85, - 0x92, 0x7d, 0x16, 0xec, 0x85, 0x21, 0xf5, 0x44, 0xff, 0xa0, 0xef, 0xc4, 0x8d, 0x17, 0xd7, 0x66, - 0xf5, 0xea, 0x8e, 0x49, 0x37, 0x5b, 0x6a, 0xda, 0x66, 0xae, 0x35, 0x6d, 0x3f, 0x84, 0xb2, 0x1b, - 0x0e, 0x87, 0x38, 0xf0, 0x1c, 0x37, 0x1c, 0x05, 0x5c, 0x19, 0x5b, 0x52, 0xc4, 0x3d, 0x41, 0x43, - 0x4d, 0x31, 0x6e, 0xe5, 0xcc, 0x72, 0x28, 0xc1, 0x9e, 0xa8, 0xa4, 0x05, 0xee, 0x28, 0x29, 0x1d, - 0x4b, 0xa8, 0xa0, 0x16, 0xac, 0x26, 0x18, 0x72, 0x92, 0x8b, 0x1a, 0x59, 0x00, 0x24, 0xb9, 0x58, - 0x76, 0x3f, 0x43, 0xdf, 0x83, 0x4a, 0xaa, 0xba, 0x13, 0xc0, 0x53, 0x3c, 0xf0, 0x3d, 0x59, 0x29, - 0xba, 0x75, 0x73, 0xc6, 0xb7, 0x62, 0xf6, 0x13, 0xc1, 0x45, 0x5f, 0x87, 0xb5, 0x44, 0xdc, 0x0f, - 0x62, 0x85, 0x82, 0x54, 0x48, 0xcc, 0x32, 0x63, 0x2a, 0x7a, 0x02, 0x28, 0x1c, 0x71, 0xc6, 0x71, - 0xe0, 0xf9, 0x41, 0x2f, 0x31, 0x36, 0x5e, 0x97, 0x1f, 0xcc, 0x33, 0xf6, 0xc2, 0x80, 0x52, 0x66, - 0xaf, 0xa7, 0x20, 0x62, 0xd3, 0xef, 0xfe, 0x5a, 0x83, 0xa2, 0xfc, 0x71, 0x2d, 0x7f, 0xa5, 0xac, - 0x40, 0xe1, 0xb8, 0xf3, 0xb8, 0x73, 0xf8, 0xb4, 0x63, 0x2c, 0xa1, 0x02, 0x64, 0xcd, 0x8e, 0x6d, - 0x68, 0xa8, 0x08, 0xf9, 0x8f, 0xf7, 0x0f, 0x77, 0x6d, 0x23, 0x23, 0x8e, 0xcd, 0x1f, 0xdb, 0xed, - 0xae, 0x91, 0x45, 0x1b, 0xb0, 0xd6, 0x6a, 0xef, 0x9b, 0x07, 0xa6, 0xdd, 0x6e, 0x39, 0x31, 0x51, - 0x47, 0x3a, 0xe4, 0x6c, 0xf3, 0xa0, 0x6d, 0xe4, 0x04, 0x54, 0xab, 0xbd, 0x67, 0x1e, 0xec, 0xee, - 0x1b, 0x79, 0xb4, 0x09, 0xeb, 0x33, 0xd9, 0x84, 0x5c, 0x44, 0x25, 0xd0, 0x5b, 0xc7, 0xd6, 0xae, - 0x6d, 0x1e, 0x76, 0x8c, 0x65, 0x81, 0x6d, 0x1f, 0x1f, 0xed, 0xb7, 0x0d, 0x40, 0xab, 0x00, 0x02, - 0xa6, 0xdb, 0xb6, 0xcc, 0x76, 0xd7, 0xf0, 0xea, 0x39, 0xbd, 0x60, 0x14, 0xee, 0xfe, 0x08, 0xd6, - 0x2f, 0xfd, 0xc8, 0x42, 0x6b, 0xb0, 0xb2, 0xdb, 0x6a, 0x39, 0x56, 0xfb, 0x68, 0xdf, 0xdc, 0xdb, - 0x35, 0x96, 0x10, 0x82, 0x55, 0xab, 0x7d, 0x70, 0xf8, 0xa4, 0x3d, 0xa5, 0x69, 0x5b, 0xb9, 0xcf, - 0x7e, 0x55, 0x5d, 0xba, 0xdb, 0x84, 0xf5, 0x4b, 0xa3, 0x5d, 0xd8, 0x79, 0xd4, 0xee, 0xb4, 0xcc, - 0xce, 0x03, 0x63, 0x09, 0x95, 0xa1, 0xb8, 0x77, 0x78, 0x70, 0x60, 0xda, 0x76, 0xbb, 0x65, 0x68, - 0x82, 0xb7, 0xdb, 0x3c, 0xb4, 0xc4, 0x47, 0x26, 0xc6, 0x68, 0x7e, 0xf0, 0xfa, 0xaf, 0xd5, 0xa5, - 0xd7, 0xe7, 0x55, 0xed, 0x8b, 0xf3, 0xaa, 0xf6, 0xa7, 0xf3, 0xaa, 0xf6, 0x97, 0xf3, 0xaa, 0xf6, - 0xf9, 0xdf, 0xaa, 0x4b, 0xcf, 0x0a, 0x2a, 0xf4, 0x27, 0xcb, 0xf2, 0x4f, 0x39, 0xf7, 0xff, 0x1d, - 0x00, 0x00, 0xff, 0xff, 0xce, 0x58, 0xcb, 0x38, 0x60, 0x12, 0x00, 0x00, + // 1877 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcd, 0x6f, 0xdb, 0xc8, + 0x15, 0x37, 0xf5, 0x61, 0x51, 0x4f, 0x1f, 0xa6, 0x27, 0x71, 0x56, 0xcd, 0xa2, 0x52, 0x56, 0x5b, + 0xb4, 0x41, 0xb0, 0x2b, 0xa3, 0x49, 0x3f, 0x8d, 0xb6, 0x80, 0xf5, 0xb1, 0x09, 0x13, 0x4b, 0x36, + 0x28, 0xda, 0x41, 0xb3, 0x58, 0xb0, 0x34, 0x39, 0x91, 0x88, 0x48, 0x1c, 0x76, 0x66, 0xe4, 0x58, + 0xfd, 0x0b, 0xf6, 0xd6, 0x3d, 0xf4, 0x50, 0xf4, 0x14, 0xa0, 0xa7, 0xf6, 0x2f, 0xe8, 0x7f, 0xd0, + 0x5c, 0x0a, 0xec, 0xad, 0x45, 0x0f, 0x42, 0xab, 0x5e, 0x5a, 0xf4, 0xd6, 0x63, 0x80, 0x05, 0x8a, + 0x19, 0x92, 0x12, 0x5d, 0x2b, 0x86, 0xdc, 0xed, 0xa1, 0x17, 0x7b, 0xe6, 0xcd, 0x7b, 0xbf, 0x79, + 0xf3, 0xde, 0xfb, 0xbd, 0x19, 0x0a, 0x10, 0x25, 0xb6, 0x33, 0x0c, 0x4e, 0x77, 0x5d, 0x9b, 0xdb, + 0x8d, 0x80, 0x12, 0x4e, 0xd0, 0xb6, 0x43, 0x9c, 0x17, 0x52, 0xde, 0x88, 0x56, 0x6f, 0xdf, 0x8a, + 0xd5, 0xc6, 0x98, 0xdb, 0x4b, 0xd5, 0xdb, 0x75, 0xc6, 0x09, 0xb5, 0x07, 0x78, 0x17, 0xfb, 0x03, + 0xcf, 0x8f, 0xff, 0x09, 0xbd, 0x33, 0xc7, 0x89, 0x74, 0xde, 0xbf, 0x4a, 0xe7, 0x41, 0xa4, 0x54, + 0x99, 0x70, 0x6f, 0xb4, 0x3b, 0x1c, 0x39, 0xbb, 0xdc, 0x1b, 0x63, 0xc6, 0xed, 0x71, 0x10, 0xad, + 0xdc, 0x1c, 0x90, 0x01, 0x91, 0xc3, 0x5d, 0x31, 0x0a, 0xa5, 0xf5, 0x4f, 0x20, 0xd3, 0x0f, 0x6c, + 0x1f, 0x7d, 0x05, 0xd2, 0x2f, 0xf0, 0xb4, 0x92, 0xbe, 0xa3, 0xdc, 0x2d, 0x36, 0x73, 0x6f, 0x66, + 0xb5, 0xf4, 0x13, 0x3c, 0x35, 0x84, 0x0c, 0xdd, 0x81, 0x1c, 0xf6, 0x5d, 0x4b, 0x2c, 0x67, 0x2e, + 0x2e, 0x6f, 0x62, 0xdf, 0x7d, 0x82, 0xa7, 0x7b, 0xc5, 0x5f, 0xbe, 0xaa, 0x6d, 0xfc, 0xee, 0x55, + 0x4d, 0xf9, 0xfb, 0xab, 0x9a, 0xf2, 0x38, 0xa3, 0x2a, 0x5a, 0xea, 0x71, 0x46, 0x4d, 0x69, 0xe9, + 0xfa, 0x18, 0xb2, 0x27, 0xf6, 0x68, 0x82, 0xd1, 0xbb, 0x90, 0xa7, 0xf6, 0x4b, 0xeb, 0x74, 0xca, + 0x31, 0xab, 0x28, 0x02, 0xc6, 0x50, 0xa9, 0xfd, 0xb2, 0x29, 0xe6, 0x68, 0x1f, 0xf2, 0x0b, 0x6f, + 0x2b, 0xa9, 0x3b, 0xca, 0xdd, 0xc2, 0xfd, 0xaf, 0x36, 0x96, 0xc1, 0x13, 0x47, 0x6a, 0x0c, 0x47, + 0x4e, 0xc3, 0x8c, 0x95, 0x9a, 0x99, 0xd7, 0xb3, 0xda, 0x86, 0xb1, 0xb4, 0xda, 0xcb, 0x88, 0xad, + 0xeb, 0x1f, 0x83, 0xfa, 0x04, 0x4f, 0xc3, 0x1d, 0xa3, 0x13, 0x29, 0x2b, 0x4e, 0xf4, 0x2d, 0xc8, + 0x9e, 0x09, 0x9d, 0x68, 0xaf, 0x4a, 0xe3, 0x52, 0xa2, 0x1a, 0x12, 0x23, 0xda, 0x26, 0x54, 0xae, + 0xff, 0x51, 0x01, 0xe8, 0x73, 0x42, 0xb1, 0xee, 0x62, 0x9f, 0xa3, 0x01, 0x80, 0x33, 0x9a, 0x30, + 0x8e, 0xa9, 0xe5, 0xb9, 0xd1, 0x36, 0x8f, 0x84, 0xfe, 0x9f, 0x67, 0xb5, 0x07, 0x03, 0x8f, 0x0f, + 0x27, 0xa7, 0x0d, 0x87, 0x8c, 0x77, 0x17, 0xd8, 0xee, 0xe9, 0x72, 0xbc, 0x1b, 0xbc, 0x18, 0xec, + 0xca, 0x54, 0x4d, 0x26, 0x9e, 0xdb, 0x38, 0x3e, 0xd6, 0xdb, 0xf3, 0x59, 0x2d, 0xdf, 0x0a, 0x01, + 0xf5, 0xb6, 0x91, 0x8f, 0xb0, 0x75, 0x17, 0x7d, 0x08, 0x39, 0x9f, 0xb8, 0x58, 0xec, 0x22, 0xfc, + 0xcd, 0x36, 0x6f, 0xce, 0x67, 0xb5, 0xcd, 0x1e, 0x71, 0xb1, 0xde, 0x7e, 0xb3, 0x18, 0x19, 0x9b, + 0x42, 0x49, 0x77, 0xd1, 0x37, 0x41, 0x15, 0x85, 0x22, 0xf5, 0xd3, 0x52, 0xff, 0xd6, 0x7c, 0x56, + 0xcb, 0x85, 0x9e, 0x0b, 0x83, 0x78, 0x68, 0xe4, 0x58, 0x78, 0x9a, 0xfa, 0x6f, 0x15, 0x28, 0xf6, + 0x83, 0x91, 0xc7, 0x4d, 0xea, 0x0d, 0x06, 0x98, 0xa2, 0x0e, 0xe4, 0x47, 0xf8, 0x39, 0xb7, 0x5c, + 0xcc, 0x1c, 0x79, 0xb4, 0xc2, 0xfd, 0xfa, 0x8a, 0x20, 0x19, 0xb6, 0x3f, 0xc0, 0x6d, 0xcc, 0x1c, + 0xea, 0x05, 0x9c, 0xd0, 0x28, 0x5c, 0xaa, 0x30, 0x15, 0x52, 0xf4, 0x10, 0x80, 0x7a, 0x83, 0x61, + 0x84, 0x93, 0xba, 0x26, 0x4e, 0x5e, 0xda, 0x0a, 0x71, 0x98, 0xdd, 0xc7, 0x19, 0x35, 0xad, 0x65, + 0xea, 0xbf, 0x4a, 0x41, 0xb1, 0x8b, 0xe9, 0x00, 0xff, 0x9f, 0x3a, 0x8b, 0x7c, 0xd0, 0x42, 0x20, + 0xc1, 0x4b, 0x8b, 0x71, 0x9b, 0x33, 0x49, 0x9c, 0xc2, 0xfd, 0x0f, 0x12, 0x70, 0x11, 0x99, 0x1b, + 0x21, 0x8b, 0x1b, 0x31, 0x99, 0x1b, 0xdd, 0x93, 0x56, 0xab, 0x2f, 0x6c, 0x9a, 0xb7, 0x04, 0xf0, + 0x7c, 0x56, 0x2b, 0x1b, 0x02, 0x6d, 0x21, 0x37, 0xca, 0x12, 0xbd, 0x7b, 0xe6, 0x38, 0x72, 0x7e, + 0x21, 0x38, 0x7f, 0x48, 0xc1, 0x4e, 0x6b, 0x28, 0x3c, 0x34, 0x70, 0x30, 0xf2, 0x1c, 0x9b, 0x2d, + 0xa3, 0x54, 0x70, 0xe4, 0x82, 0xc5, 0xa7, 0x01, 0x96, 0x71, 0x2a, 0xdf, 0xff, 0xda, 0xaa, 0xf3, + 0x85, 0x86, 0x21, 0x8a, 0x39, 0x0d, 0xb0, 0x01, 0xce, 0x62, 0x8c, 0xda, 0x90, 0xa3, 0xa1, 0x42, + 0x14, 0xa2, 0x2b, 0x20, 0x2e, 0x05, 0x29, 0x36, 0x45, 0xc7, 0xa0, 0x4d, 0x02, 0xd7, 0xe6, 0xd8, + 0xb5, 0x22, 0x11, 0xab, 0xa4, 0xef, 0xa4, 0xaf, 0x09, 0xb7, 0x15, 0x61, 0xc4, 0x47, 0x45, 0x1f, + 0xc1, 0x96, 0x8f, 0xcf, 0x79, 0x8c, 0x29, 0x18, 0x90, 0x91, 0x0c, 0xa8, 0xce, 0x67, 0xb5, 0x52, + 0x0f, 0x9f, 0xf3, 0x48, 0x55, 0xf2, 0x20, 0xbf, 0x98, 0x18, 0x25, 0x3f, 0xb1, 0xe6, 0xee, 0xa9, + 0xa2, 0x9f, 0xc9, 0x86, 0xf2, 0x99, 0x02, 0x37, 0xba, 0xc4, 0xf5, 0x9e, 0x7b, 0xd8, 0x15, 0x7d, + 0x32, 0x8e, 0xe6, 0x07, 0x80, 0xd8, 0x94, 0x71, 0x3c, 0xb6, 0x1c, 0xe2, 0x3f, 0xf7, 0x06, 0x16, + 0x0b, 0x6c, 0x5f, 0x06, 0x55, 0x35, 0xb4, 0x70, 0xa5, 0x25, 0x17, 0x64, 0x73, 0xed, 0x00, 0x92, + 0x0c, 0x1e, 0x79, 0x67, 0xd8, 0xc7, 0x8c, 0x85, 0xda, 0x61, 0xfc, 0xde, 0x59, 0x71, 0x60, 0x61, + 0x64, 0x68, 0xc2, 0xe4, 0x20, 0xb2, 0x10, 0x92, 0xa8, 0xc7, 0xfd, 0x33, 0x05, 0x3b, 0xba, 0xcf, + 0x31, 0xf5, 0xed, 0x51, 0x8b, 0x8c, 0xc7, 0x4b, 0xd6, 0xb6, 0xa1, 0xc4, 0x04, 0x8b, 0x2d, 0x1e, + 0x0a, 0x22, 0x32, 0xd4, 0x56, 0xee, 0xb0, 0x64, 0xbb, 0x51, 0x64, 0x49, 0xee, 0xb7, 0xa1, 0x34, + 0x16, 0xf4, 0x5a, 0xa0, 0xa4, 0xde, 0x8a, 0x92, 0xa4, 0xa1, 0x51, 0x1c, 0x27, 0x49, 0xf9, 0x13, + 0x78, 0x27, 0x2a, 0xb7, 0x38, 0xc1, 0x0b, 0xbc, 0xb4, 0xc4, 0xbb, 0xbb, 0x02, 0x6f, 0x65, 0xe5, + 0x1a, 0x3b, 0xce, 0xca, 0x82, 0x7e, 0x06, 0x3b, 0xe3, 0x28, 0x33, 0x32, 0x9e, 0x0b, 0xfc, 0x90, + 0x6b, 0x5f, 0x5f, 0xe5, 0xef, 0xe5, 0x4c, 0x1a, 0x37, 0xc6, 0x97, 0x85, 0x7b, 0xea, 0xa7, 0xd1, + 0x65, 0x56, 0xff, 0xb9, 0x02, 0xdb, 0x87, 0xa7, 0x0c, 0xd3, 0x33, 0xec, 0x2e, 0xae, 0x9f, 0x64, + 0x4b, 0x56, 0xd6, 0x68, 0xc9, 0xff, 0x83, 0xfb, 0x4d, 0x8d, 0xaf, 0xd7, 0xfa, 0x17, 0x9b, 0x50, + 0x30, 0xa9, 0xed, 0x33, 0xdb, 0xe1, 0x1e, 0xf1, 0xd1, 0x23, 0xc8, 0x88, 0xc7, 0x44, 0x94, 0xec, + 0x7b, 0x6b, 0xb4, 0x18, 0xf3, 0xdc, 0xef, 0x62, 0x6e, 0x37, 0x55, 0xb1, 0xc9, 0xe7, 0xb3, 0x9a, + 0x62, 0x48, 0x04, 0x84, 0x20, 0xe3, 0xdb, 0xe3, 0xf0, 0x56, 0xcc, 0x1b, 0x72, 0x8c, 0x7e, 0x00, + 0x9b, 0xa2, 0x83, 0x4d, 0xc2, 0x16, 0xb6, 0xba, 0x63, 0x24, 0xbc, 0xe9, 0x4b, 0x5d, 0x23, 0xb2, + 0x41, 0x8f, 0xa1, 0x3c, 0xb2, 0x19, 0xb7, 0x86, 0xd8, 0xa6, 0xfc, 0x14, 0xdb, 0xbc, 0x92, 0x5d, + 0xff, 0xf4, 0x25, 0x61, 0xfa, 0x28, 0xb6, 0x14, 0x58, 0x84, 0x7a, 0x03, 0x6b, 0x19, 0xc9, 0xcd, + 0x6b, 0x60, 0x09, 0xd3, 0x65, 0xfe, 0x1e, 0x41, 0x69, 0x6c, 0x9f, 0x27, 0xa0, 0x72, 0xeb, 0x43, + 0x15, 0xc7, 0xf6, 0xf9, 0x12, 0xe9, 0x63, 0xb8, 0x41, 0xa2, 0xf2, 0x58, 0xc2, 0xb1, 0x8a, 0xfa, + 0xd6, 0x66, 0x76, 0xa9, 0x98, 0x22, 0x58, 0x44, 0xfe, 0x73, 0x81, 0xa1, 0x0a, 0xe4, 0x5e, 0x52, + 0x8f, 0x7b, 0xfe, 0xa0, 0x92, 0x97, 0xad, 0x25, 0x9e, 0xa2, 0xef, 0x42, 0xce, 0xf3, 0x39, 0xf6, + 0x39, 0xab, 0x14, 0xe4, 0x56, 0x6f, 0x6b, 0x23, 0x71, 0xe7, 0x8d, 0xb4, 0x51, 0x1d, 0x4a, 0x02, + 0x03, 0x5b, 0x9c, 0x10, 0x8b, 0x8c, 0xdc, 0x4a, 0x51, 0x02, 0x17, 0xa4, 0xd0, 0x24, 0xe4, 0x70, + 0xe4, 0x0a, 0x1d, 0x8a, 0x39, 0x9d, 0x5a, 0xc4, 0xb7, 0x82, 0x09, 0x1b, 0x56, 0x4a, 0xa1, 0x8e, + 0x14, 0x1e, 0xfa, 0x47, 0x13, 0x36, 0x44, 0xc7, 0x70, 0x13, 0x07, 0xc4, 0x19, 0x5a, 0x3f, 0xc3, + 0x94, 0x24, 0x02, 0x59, 0x5e, 0x3f, 0x90, 0x48, 0x02, 0x3c, 0xc3, 0x94, 0x2c, 0xc3, 0x69, 0xc2, + 0x0d, 0x8a, 0x9f, 0x53, 0xcc, 0x86, 0xc9, 0x78, 0x56, 0xb6, 0xae, 0x81, 0xba, 0xb0, 0x5f, 0xa2, + 0xfe, 0x10, 0xde, 0xbd, 0x58, 0x3a, 0xd6, 0x4b, 0x9b, 0x59, 0x71, 0xc0, 0x2b, 0x9a, 0x3c, 0x5e, + 0xe5, 0x42, 0x89, 0x3c, 0xb5, 0x59, 0x9c, 0xa9, 0x8b, 0xcf, 0xdb, 0xfa, 0xef, 0x15, 0xd8, 0xd4, + 0x65, 0x34, 0xd1, 0xb7, 0x21, 0xb3, 0xe8, 0xfb, 0x57, 0xa4, 0x20, 0xc1, 0x33, 0xa1, 0x8e, 0x9a, + 0x90, 0xe6, 0xe7, 0x71, 0xff, 0xbf, 0x0e, 0x61, 0xc3, 0x13, 0x0a, 0xe3, 0x04, 0x2f, 0xd3, 0xd7, + 0xe7, 0x65, 0x74, 0x93, 0x3c, 0x84, 0x72, 0x1f, 0xff, 0x74, 0x82, 0x7d, 0x07, 0xbb, 0x4f, 0x45, + 0xfe, 0xaf, 0x7a, 0x33, 0xdf, 0x06, 0x95, 0x45, 0xca, 0xe1, 0x33, 0xd4, 0x58, 0xcc, 0xeb, 0x5f, + 0xa4, 0x21, 0x7b, 0x80, 0x6d, 0x86, 0xd1, 0xf7, 0x21, 0xcb, 0xb8, 0x4d, 0x79, 0x14, 0x92, 0xb5, + 0x32, 0x16, 0x5a, 0xa0, 0x4f, 0x00, 0xf0, 0x79, 0xe0, 0x51, 0x5b, 0xf8, 0xbb, 0x5e, 0x97, 0xac, + 0xfe, 0x6b, 0x56, 0xbb, 0x9d, 0x78, 0x5f, 0xef, 0xd5, 0xa9, 0xed, 0xbb, 0xfe, 0x64, 0x34, 0xb2, + 0x4f, 0x47, 0xb8, 0x6e, 0x24, 0x00, 0x93, 0x0f, 0x97, 0xf4, 0x7f, 0xff, 0x70, 0x99, 0xc0, 0x3b, + 0x2e, 0x0e, 0x28, 0x76, 0xe4, 0xdb, 0x45, 0x3a, 0x2e, 0xfe, 0x32, 0x2f, 0x7e, 0xe2, 0x7d, 0x49, + 0x8f, 0x77, 0x96, 0xe8, 0x7d, 0x01, 0xde, 0x97, 0xd8, 0xa8, 0x07, 0x85, 0x80, 0x92, 0x80, 0x30, + 0xc1, 0x0a, 0xb6, 0x5e, 0x13, 0x2d, 0xcf, 0x67, 0x35, 0x38, 0x8a, 0xac, 0xcc, 0xbe, 0x01, 0x31, + 0x82, 0xc9, 0xd0, 0x4d, 0xc8, 0x4a, 0xf2, 0xc9, 0x16, 0x9a, 0x36, 0xc2, 0x09, 0xfa, 0x30, 0x91, + 0x62, 0xd1, 0x10, 0xd3, 0xcd, 0xed, 0x37, 0xb3, 0x5a, 0x49, 0x66, 0x36, 0x2e, 0x94, 0x65, 0xd6, + 0xc3, 0x57, 0x92, 0xa0, 0x45, 0xfd, 0x17, 0x0a, 0x94, 0xf7, 0x4f, 0x09, 0xe5, 0xa2, 0xdc, 0x3b, + 0x3e, 0xa7, 0xd3, 0xab, 0x2a, 0xe9, 0xcb, 0xdf, 0x86, 0xa2, 0x18, 0x03, 0xea, 0x11, 0xea, 0xf1, + 0xf0, 0x93, 0x35, 0x6b, 0x2c, 0xe6, 0x89, 0x9b, 0xf2, 0x1f, 0x69, 0x28, 0x9a, 0xe7, 0x7e, 0x8b, + 0x10, 0xea, 0x0a, 0xfe, 0xa0, 0xef, 0x84, 0xc4, 0x0b, 0x6b, 0xb3, 0x7a, 0x35, 0x63, 0x92, 0x64, + 0x4b, 0x74, 0xdb, 0xd4, 0xb5, 0xba, 0xed, 0xfb, 0x50, 0x72, 0xc8, 0x78, 0x6c, 0xfb, 0xae, 0xe5, + 0x90, 0x89, 0xcf, 0x23, 0x67, 0x8b, 0x91, 0xb0, 0x25, 0x64, 0xa8, 0x29, 0xda, 0xad, 0xec, 0x59, + 0x16, 0xc5, 0xb6, 0x2b, 0x2a, 0x69, 0x8d, 0x3d, 0x8a, 0x91, 0x8d, 0x21, 0x4c, 0x50, 0x1b, 0xca, + 0x31, 0x86, 0xec, 0xe4, 0xa2, 0x46, 0xd6, 0x00, 0x89, 0x37, 0x96, 0xec, 0x67, 0xe8, 0x7b, 0x50, + 0x49, 0x54, 0x77, 0x0c, 0x78, 0x66, 0x8f, 0x3c, 0x57, 0x56, 0x8a, 0x6a, 0xdc, 0x5a, 0xae, 0x1b, + 0xe1, 0xf2, 0x89, 0x58, 0x45, 0xdf, 0x80, 0xad, 0x58, 0xdd, 0xf3, 0x43, 0x83, 0x9c, 0x34, 0x88, + 0xdd, 0xd2, 0x43, 0x29, 0x3a, 0x01, 0x44, 0x26, 0x9c, 0x71, 0xdb, 0x77, 0x3d, 0x7f, 0x10, 0x3b, + 0x1b, 0x5e, 0x97, 0xef, 0xad, 0x72, 0xf6, 0x42, 0x83, 0x8a, 0xdc, 0xde, 0x4e, 0x40, 0x84, 0xae, + 0xdf, 0xfb, 0x8d, 0x02, 0x79, 0xf9, 0xcd, 0x2e, 0xbf, 0x52, 0x0a, 0x90, 0x3b, 0xee, 0x3d, 0xe9, + 0x1d, 0x3e, 0xed, 0x69, 0x1b, 0x28, 0x07, 0x69, 0xbd, 0x67, 0x6a, 0x0a, 0xca, 0x43, 0xf6, 0xa3, + 0x83, 0xc3, 0x7d, 0x53, 0x4b, 0x89, 0x61, 0xf3, 0xc7, 0x66, 0xa7, 0xaf, 0xa5, 0xd1, 0x0d, 0xd8, + 0x6a, 0x77, 0x0e, 0xf4, 0xae, 0x6e, 0x76, 0xda, 0x56, 0x28, 0x54, 0x91, 0x0a, 0x19, 0x53, 0xef, + 0x76, 0xb4, 0x8c, 0x80, 0x6a, 0x77, 0x5a, 0x7a, 0x77, 0xff, 0x40, 0xcb, 0xa2, 0x1d, 0xd8, 0x5e, + 0xea, 0xc6, 0xe2, 0x3c, 0x2a, 0x82, 0xda, 0x3e, 0x36, 0xf6, 0x4d, 0xfd, 0xb0, 0xa7, 0x6d, 0x0a, + 0x6c, 0xf3, 0xf8, 0xe8, 0xa0, 0xa3, 0x01, 0x2a, 0x03, 0x08, 0x98, 0x7e, 0xc7, 0xd0, 0x3b, 0x7d, + 0xcd, 0xad, 0x67, 0xd4, 0x9c, 0x96, 0xbb, 0xf7, 0x23, 0xd8, 0xbe, 0xf4, 0x91, 0x85, 0xb6, 0xa0, + 0xb0, 0xdf, 0x6e, 0x5b, 0x46, 0xe7, 0xe8, 0x40, 0x6f, 0xed, 0x6b, 0x1b, 0x08, 0x41, 0xd9, 0xe8, + 0x74, 0x0f, 0x4f, 0x3a, 0x0b, 0x99, 0x72, 0x3b, 0xf3, 0xe9, 0xaf, 0xab, 0x1b, 0xf7, 0x9a, 0xb0, + 0x7d, 0xa9, 0xb5, 0x0b, 0x3f, 0x8f, 0x3a, 0xbd, 0xb6, 0xde, 0x7b, 0xa8, 0x6d, 0xa0, 0x12, 0xe4, + 0x5b, 0x87, 0xdd, 0xae, 0x6e, 0x9a, 0x9d, 0xb6, 0xa6, 0x88, 0xb5, 0xfd, 0xe6, 0xa1, 0x21, 0x26, + 0xa9, 0x10, 0xa3, 0xf9, 0xde, 0xeb, 0xbf, 0x56, 0x37, 0x5e, 0xcf, 0xab, 0xca, 0xe7, 0xf3, 0xaa, + 0xf2, 0xa7, 0x79, 0x55, 0xf9, 0xcb, 0xbc, 0xaa, 0x7c, 0xf6, 0xb7, 0xea, 0xc6, 0xb3, 0x5c, 0x14, + 0xfa, 0xd3, 0x4d, 0xf9, 0x0b, 0xd1, 0x83, 0x7f, 0x07, 0x00, 0x00, 0xff, 0xff, 0x6f, 0x61, 0x6f, + 0xbb, 0xdb, 0x12, 0x00, 0x00, } diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 3f8e9a36d17b..c2de1b3a521d 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -17,6 +17,7 @@ package cockroach.roachpb; option go_package = "roachpb"; import "roachpb/metadata.proto"; +import "storage/engine/enginepb/mvcc.proto"; import "storage/engine/enginepb/mvcc3.proto"; import "util/hlc/timestamp.proto"; import "gogoproto/gogo.proto"; @@ -133,9 +134,12 @@ message MergeTrigger { RangeDescriptor left_desc = 1 [(gogoproto.nullable) = false]; RangeDescriptor right_desc = 2 [(gogoproto.nullable) = false]; - // A snapshot of the replicated data from the right-hand side of the merge - // as a serialized RocksDB write batch. - bytes right_data = 3; + reserved 3; + + storage.engine.enginepb.MVCCStats right_mvcc_stats = 4 [ + (gogoproto.customname) = "RightMVCCStats", + (gogoproto.nullable) = false + ]; } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -487,7 +491,7 @@ message TxnCoordMeta { repeated Span refresh_writes = 5 [(gogoproto.nullable) = false]; // refresh_invalid indicates that spans were discarded or not collected // (i.e. because of a dist SQL processor running a version before refreshing - // was introduced). This is false if all spans encountered during the + // was introduced). This is false if all spans encountered during the // transaction which need refreshing have been collected to the refresh_reads // and refresh_writes span slices. bool refresh_invalid = 7; diff --git a/pkg/sql/opt/catalog.go b/pkg/sql/opt/catalog.go index a4c0ad3c6812..e7da3e710b8c 100644 --- a/pkg/sql/opt/catalog.go +++ b/pkg/sql/opt/catalog.go @@ -29,9 +29,6 @@ import ( // This file contains interfaces that are used by the query optimizer to avoid // including specifics of sqlbase structures in the opt code. -// ColumnName is the type of a column name. -type ColumnName string - // PrimaryIndex selects the primary index of a table when calling the // Table.Index method. Every table is guaranteed to have a unique primary // index, even if it meant adding a hidden unique rowid column. @@ -128,7 +125,7 @@ type View interface { // needed by the query optimizer. type Column interface { // ColName returns the name of the column. - ColName() ColumnName + ColName() tree.Name // DatumType returns the data type of the column. DatumType() types.T diff --git a/pkg/sql/opt/exec/execbuilder/testdata/aggregate b/pkg/sql/opt/exec/execbuilder/testdata/aggregate index 96edccf84587..48b83c0d6a85 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/aggregate +++ b/pkg/sql/opt/exec/execbuilder/testdata/aggregate @@ -49,18 +49,18 @@ render · · (min int, max i │ render 9 (agg8)[bool] · · │ render 10 (agg9)[bytes] · · └── group · · (agg0 int, agg1 int, agg2 int, agg3 int, agg4 decimal, agg5 decimal, agg6 decimal, agg7 decimal, agg8 bool, agg9 bytes) · - │ aggregate 0 min(kv.v) · · - │ aggregate 1 max(kv.v) · · - │ aggregate 2 count(kv.v) · · + │ aggregate 0 min(v) · · + │ aggregate 1 max(v) · · + │ aggregate 2 count(v) · · │ aggregate 3 sum_int(column8) · · - │ aggregate 4 avg(kv.v) · · - │ aggregate 5 sum(kv.v) · · - │ aggregate 6 stddev(kv.v) · · - │ aggregate 7 variance(kv.v) · · + │ aggregate 4 avg(v) · · + │ aggregate 5 sum(v) · · + │ aggregate 6 stddev(v) · · + │ aggregate 7 variance(v) · · │ aggregate 8 bool_and(column14) · · │ aggregate 9 xor_agg(column16) · · │ scalar · · · - └── render · · (column8 int, column14 bool, column16 bytes, "kv.v" int) · + └── render · · (column8 int, column14 bool, column16 bytes, v int) · │ render 0 (1)[int] · · │ render 1 ((v)[int] = (1)[int])[bool] · · │ render 2 ((s)[string]::BYTES)[bytes] · · @@ -541,18 +541,18 @@ sort · · (v int, count int) +count query TTTTT EXPLAIN (TYPES) SELECT v, count(1) FROM kv GROUP BY v ORDER BY count(1) ---- -sort · · (v int, count int) +count - │ order +count · · - └── group · · (v int, count int) · - │ aggregate 0 kv.v · · - │ aggregate 1 count(column5) · · - │ group by @2 · · - └── render · · (column5 int, "kv.v" int) · - │ render 0 (1)[int] · · - │ render 1 (v)[int] · · - └── scan · · (v int) · -· table kv@primary · · -· spans ALL · · +sort · · (v int, count int) +count + │ order +count · · + └── group · · (v int, count int) · + │ aggregate 0 v · · + │ aggregate 1 count(column5) · · + │ group by @2 · · + └── render · · (column5 int, v int) · + │ render 0 (1)[int] · · + │ render 1 (v)[int] · · + └── scan · · (v int) · +· table kv@primary · · +· spans ALL · · # TODO(radu): we don't propagate filters yet. ## Check that filters propagate through no-op aggregation. @@ -848,18 +848,18 @@ render · · query TTTTT EXPLAIN (TYPES) SELECT 1 a FROM kv GROUP BY v, w::DECIMAL HAVING w::DECIMAL > 1; ---- -render · · (a int) · - │ render 0 (1)[int] · · - └── distinct · · (column5 decimal, "kv.v" int) weak-key(column5,"kv.v") - │ distinct on column5, kv.v · · - └── filter · · (column5 decimal, "kv.v" int) · - │ filter ((column5)[decimal] > (1)[decimal])[bool] · · - └── render · · (column5 decimal, "kv.v" int) · - │ render 0 ((w)[int]::DECIMAL)[decimal] · · - │ render 1 (v)[int] · · - └── scan · · (v int, w int) · -· table kv@primary · · -· spans ALL · · +render · · (a int) · + │ render 0 (1)[int] · · + └── distinct · · (column5 decimal, v int) weak-key(column5,v) + │ distinct on column5, v · · + └── filter · · (column5 decimal, v int) · + │ filter ((column5)[decimal] > (1)[decimal])[bool] · · + └── render · · (column5 decimal, v int) · + │ render 0 ((w)[int]::DECIMAL)[decimal] · · + │ render 1 (v)[int] · · + └── scan · · (v int, w int) · +· table kv@primary · · +· spans ALL · · statement ok CREATE TABLE foo(a INT, b CHAR) @@ -868,34 +868,34 @@ CREATE TABLE foo(a INT, b CHAR) query TTTTT EXPLAIN (VERBOSE) SELECT min(a) AS m FROM foo GROUP BY @1 ---- -render · · (m) · - │ render 0 agg0 · · - └── group · · (column4, agg0) · - │ aggregate 0 column4 · · - │ aggregate 1 min(foo.a) · · - │ group by @1 · · - └── render · · (column4, "foo.a") · - │ render 0 a · · - │ render 1 a · · - └── scan · · (a) · -· table foo@primary · · -· spans ALL · · +render · · (m) · + │ render 0 agg0 · · + └── group · · (column4, agg0) · + │ aggregate 0 column4 · · + │ aggregate 1 min(a) · · + │ group by @1 · · + └── render · · (column4, a) · + │ render 0 a · · + │ render 1 a · · + └── scan · · (a) · +· table foo@primary · · +· spans ALL · · query TTTTT EXPLAIN (VERBOSE) SELECT min(a) AS m FROM foo GROUP BY @2 ---- -render · · (m) · - │ render 0 agg0 · · - └── group · · (column4, agg0) · - │ aggregate 0 column4 · · - │ aggregate 1 min(foo.a) · · - │ group by @1 · · - └── render · · (column4, "foo.a") · - │ render 0 b · · - │ render 1 a · · - └── scan · · (a, b) · -· table foo@primary · · -· spans ALL · · +render · · (m) · + │ render 0 agg0 · · + └── group · · (column4, agg0) · + │ aggregate 0 column4 · · + │ aggregate 1 min(a) · · + │ group by @1 · · + └── render · · (column4, a) · + │ render 0 b · · + │ render 1 a · · + └── scan · · (a, b) · +· table foo@primary · · +· spans ALL · · query TTTTT EXPLAIN (VERBOSE) SELECT array_agg(v) FROM (SELECT * FROM kv ORDER BY v) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_numtables b/pkg/sql/opt/exec/execbuilder/testdata/distsql_numtables index df7efd55c3a9..2a8d12b93768 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_numtables +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_numtables @@ -162,38 +162,38 @@ https://cockroachdb.github.io/distsqlplan/decode.html#eJyMj7FqxDAQRPt8RZhaELtVde query TTTTT EXPLAIN (VERBOSE) SELECT y, str, repeat('test', y) AS res FROM NumToStr ORDER BY res ---- -render · · (y, str, res) · - │ render 0 "numtostr.y" · · - │ render 1 "numtostr.str" · · - │ render 2 res · · - └── sort · · (res, "numtostr.y", "numtostr.str") +res - │ order +res · · - └── render · · (res, "numtostr.y", "numtostr.str") · - │ render 0 repeat('test', y) · · - │ render 1 y · · - │ render 2 str · · - └── scan · · (y, str) · -· table numtostr@primary · · -· spans ALL · · +render · · (y, str, res) · + │ render 0 y · · + │ render 1 str · · + │ render 2 res · · + └── sort · · (res, y, str) +res + │ order +res · · + └── render · · (res, y, str) · + │ render 0 repeat('test', y) · · + │ render 1 y · · + │ render 2 str · · + └── scan · · (y, str) · +· table numtostr@primary · · +· spans ALL · · query TTTTT EXPLAIN (VERBOSE) SELECT y, str, repeat('test', y) AS res FROM NumToStr ORDER BY res LIMIT 10 ---- -render · · (y, str, res) · - │ render 0 "numtostr.y" · · - │ render 1 "numtostr.str" · · - │ render 2 res · · - └── limit · · (res, "numtostr.y", "numtostr.str") +res - │ count 10 · · - └── sort · · (res, "numtostr.y", "numtostr.str") +res - │ order +res · · - └── render · · (res, "numtostr.y", "numtostr.str") · - │ render 0 repeat('test', y) · · - │ render 1 y · · - │ render 2 str · · - └── scan · · (y, str) · -· table numtostr@primary · · -· spans ALL · · +render · · (y, str, res) · + │ render 0 y · · + │ render 1 str · · + │ render 2 res · · + └── limit · · (res, y, str) +res + │ count 10 · · + └── sort · · (res, y, str) +res + │ order +res · · + └── render · · (res, y, str) · + │ render 0 repeat('test', y) · · + │ render 1 y · · + │ render 2 str · · + └── scan · · (y, str) · +· table numtostr@primary · · +· spans ALL · · # Regression test for #20481. query T diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 667d758da987..23680588c8ee 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -486,15 +486,15 @@ values · · (column1 int, column2 int, column3 i query TTTTT EXPLAIN (TYPES) SELECT 2*count(k) as z, v FROM t WHERE v>123 GROUP BY v HAVING v<2 AND count(k)>1 ---- -render · · (z int, v int) · - │ render 0 ((agg0)[int] * (2)[int])[int] · · - │ render 1 (agg1)[int] · · - └── filter · · (agg0 int, agg1 int) · - │ filter ((agg0)[int] > (1)[int])[bool] · · - └── group · · (agg0 int, agg1 int) · - │ aggregate 0 count(t.k) · · - │ aggregate 1 any_not_null(t.v) · · - └── norows · · ("t.k" int, "t.v" int) · +render · · (z int, v int) · + │ render 0 ((agg0)[int] * (2)[int])[int] · · + │ render 1 (agg1)[int] · · + └── filter · · (agg0 int, agg1 int) · + │ filter ((agg0)[int] > (1)[int])[bool] · · + └── group · · (agg0 int, agg1 int) · + │ aggregate 0 count(k) · · + │ aggregate 1 any_not_null(v) · · + └── norows · · (k int, v int) · query TTTTT EXPLAIN (TYPES) DELETE FROM t WHERE v > 1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/join b/pkg/sql/opt/exec/execbuilder/testdata/join index 7112de9cbfd6..ff4bfbbf1c0e 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/join +++ b/pkg/sql/opt/exec/execbuilder/testdata/join @@ -181,25 +181,25 @@ render · · (x) · query TTTTT EXPLAIN (VERBOSE) SELECT * FROM (SELECT x, 2 two FROM onecolumn) NATURAL FULL JOIN (SELECT x, y+1 plus1 FROM twocolumn) ---- -render · · (x, two, plus1) · - │ render 0 COALESCE("onecolumn.x", "twocolumn.x") · · - │ render 1 two · · - │ render 2 plus1 · · - └── join · · (two, "onecolumn.x", plus1, "twocolumn.x") · - │ type full outer · · - │ equality ("onecolumn.x") = ("twocolumn.x") · · - ├── render · · (two, "onecolumn.x") · - │ │ render 0 2 · · - │ │ render 1 x · · - │ └── scan · · (x) · - │ table onecolumn@primary · · - │ spans ALL · · - └── render · · (plus1, "twocolumn.x") · - │ render 0 y + 1 · · - │ render 1 x · · - └── scan · · (x, y) · -· table twocolumn@primary · · -· spans ALL · · +render · · (x, two, plus1) · + │ render 0 COALESCE(x, x) · · + │ render 1 two · · + │ render 2 plus1 · · + └── join · · (two, x, plus1, x) · + │ type full outer · · + │ equality (x) = (x) · · + ├── render · · (two, x) · + │ │ render 0 2 · · + │ │ render 1 x · · + │ └── scan · · (x) · + │ table onecolumn@primary · · + │ spans ALL · · + └── render · · (plus1, x) · + │ render 0 y + 1 · · + │ render 1 x · · + └── scan · · (x, y) · +· table twocolumn@primary · · +· spans ALL · · # Ensure that the ordering information for the result of joins is sane. (#12037) query TTTTT @@ -401,27 +401,27 @@ join · · (a, b, n, sq) · query TTTTT EXPLAIN (VERBOSE) SELECT a, b, n, sq FROM (SELECT a, b, a * b / 2 AS div, n, sq FROM pairs, square) WHERE div = sq ---- -render · · (a, b, n, sq) · - │ render 0 "pairs.a" · · - │ render 1 "pairs.b" · · - │ render 2 "square.n" · · - │ render 3 "square.sq" · · - └── filter · · (div, "pairs.a", "pairs.b", "square.n", "square.sq") · - │ filter div = "square.sq" · · - └── render · · (div, "pairs.a", "pairs.b", "square.n", "square.sq") · - │ render 0 (a * b) / 2 · · - │ render 1 a · · - │ render 2 b · · - │ render 3 n · · - │ render 4 sq · · - └── join · · (a, b, n, sq) · - │ type cross · · - ├── scan · · (a, b) · - │ table pairs@primary · · - │ spans ALL · · - └── scan · · (n, sq) · -· table square@primary · · -· spans ALL · · +render · · (a, b, n, sq) · + │ render 0 a · · + │ render 1 b · · + │ render 2 n · · + │ render 3 sq · · + └── filter · · (div, a, b, n, sq) · + │ filter div = sq · · + └── render · · (div, a, b, n, sq) · + │ render 0 (a * b) / 2 · · + │ render 1 a · · + │ render 2 b · · + │ render 3 n · · + │ render 4 sq · · + └── join · · (a, b, n, sq) · + │ type cross · · + ├── scan · · (a, b) · + │ table pairs@primary · · + │ spans ALL · · + └── scan · · (n, sq) · +· table square@primary · · +· spans ALL · · # The filter expression must stay on top of the outer join. query TTTTT diff --git a/pkg/sql/opt/exec/execbuilder/testdata/orderby b/pkg/sql/opt/exec/execbuilder/testdata/orderby index 584d48f7a88a..0b3e63a58208 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/orderby +++ b/pkg/sql/opt/exec/execbuilder/testdata/orderby @@ -581,34 +581,34 @@ CREATE TABLE foo(a INT, b CHAR) query TTTTT EXPLAIN (VERBOSE) SELECT b, a FROM foo ORDER BY @1 ---- -render · · (b, a) · - │ render 0 "foo.b" · · - │ render 1 "foo.a" · · - └── sort · · (column4, "foo.a", "foo.b") +column4 - │ order +column4 · · - └── render · · (column4, "foo.a", "foo.b") · - │ render 0 a · · - │ render 1 a · · - │ render 2 b · · - └── scan · · (a, b) · -· table foo@primary · · -· spans ALL · · +render · · (b, a) · + │ render 0 b · · + │ render 1 a · · + └── sort · · (column4, a, b) +column4 + │ order +column4 · · + └── render · · (column4, a, b) · + │ render 0 a · · + │ render 1 a · · + │ render 2 b · · + └── scan · · (a, b) · +· table foo@primary · · +· spans ALL · · query TTTTT EXPLAIN (VERBOSE) SELECT b, a FROM foo ORDER BY @2 ---- -render · · (b, a) · - │ render 0 "foo.b" · · - │ render 1 "foo.a" · · - └── sort · · (column4, "foo.a", "foo.b") +column4 - │ order +column4 · · - └── render · · (column4, "foo.a", "foo.b") · - │ render 0 b · · - │ render 1 a · · - │ render 2 b · · - └── scan · · (a, b) · -· table foo@primary · · -· spans ALL · · +render · · (b, a) · + │ render 0 b · · + │ render 1 a · · + └── sort · · (column4, a, b) +column4 + │ order +column4 · · + └── render · · (column4, a, b) · + │ render 0 b · · + │ render 1 a · · + │ render 2 b · · + └── scan · · (a, b) · +· table foo@primary · · +· spans ALL · · # ------------------------------------------------------------------------------ # Check star expansion in ORDER BY. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/srfs b/pkg/sql/opt/exec/execbuilder/testdata/srfs index fcc64754ce7c..03b9dbe03b25 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/srfs +++ b/pkg/sql/opt/exec/execbuilder/testdata/srfs @@ -89,31 +89,31 @@ query TTTTT EXPLAIN (VERBOSE) SELECT x, y, z, information_schema._pg_expandarray(ARRAY[x, y, z]) FROM xy NATURAL JOIN xz WHERE y < z ORDER BY 1, 2, 3 ---- -render · · (x, y, z, "information_schema._pg_expandarray") · - │ render 0 "xy.x" · · - │ render 1 "xy.y" · · - │ render 2 "xz.z" · · - │ render 3 "information_schema._pg_expandarray" · · - └── sort · · ("information_schema._pg_expandarray", "xy.x", "xy.y", "xz.z") +"xy.x" - │ order +"xy.x" · · - └── render · · ("information_schema._pg_expandarray", "xy.x", "xy.y", "xz.z") · - │ render 0 ((x, n) AS x, n) · · - │ render 1 x · · - │ render 2 y · · - │ render 3 z · · - └── project set · · (x, y, x, z, x, n) · - │ render 0 information_schema._pg_expandarray(ARRAY[@1, @2, @4]) · · - └── join · · (x, y, x, z) · - │ type inner · · - │ equality (x) = (x) · · - │ mergeJoinOrder +"(x=x)" · · - │ pred (x = x) AND (y < z) · · - ├── scan · · (x, y) +x - │ table xy@primary · · - │ spans ALL · · - └── scan · · (x, z) +x -· table xz@primary · · -· spans ALL · · +render · · (x, y, z, "information_schema._pg_expandarray") · + │ render 0 x · · + │ render 1 y · · + │ render 2 z · · + │ render 3 "information_schema._pg_expandarray" · · + └── sort · · ("information_schema._pg_expandarray", x, y, z) +x + │ order +x · · + └── render · · ("information_schema._pg_expandarray", x, y, z) · + │ render 0 ((x, n) AS x, n) · · + │ render 1 x · · + │ render 2 y · · + │ render 3 z · · + └── project set · · (x, y, x, z, x, n) · + │ render 0 information_schema._pg_expandarray(ARRAY[@1, @2, @4]) · · + └── join · · (x, y, x, z) · + │ type inner · · + │ equality (x) = (x) · · + │ mergeJoinOrder +"(x=x)" · · + │ pred (x = x) AND (y < z) · · + ├── scan · · (x, y) +x + │ table xy@primary · · + │ spans ALL · · + └── scan · · (x, z) +x +· table xz@primary · · +· spans ALL · · query TTTTT EXPLAIN (VERBOSE) SELECT generate_series(x, z) FROM xz WHERE z < ANY(SELECT generate_series(x, y) FROM xy) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/subquery b/pkg/sql/opt/exec/execbuilder/testdata/subquery index c74d24e42563..bfbddbf09e1b 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/subquery +++ b/pkg/sql/opt/exec/execbuilder/testdata/subquery @@ -122,35 +122,35 @@ WHERE OR (col4 IN (SELECT col1 FROM tab4 WHERE col1 > 8.27)) AND (col3 <= 5 AND (col3 BETWEEN 7 AND 9)) ---- -render · · (col0) · - │ render 0 "tab4.col0" · · - └── filter · · ("case", "tab4.col0", "tab4.col3", "tab4.col4") · - │ filter (("tab4.col0" <= 0) AND ("tab4.col4" <= 5.38)) OR ((("case" AND ("tab4.col3" <= 5)) AND ("tab4.col3" >= 7)) AND ("tab4.col3" <= 9)) · · - └── render · · ("case", "tab4.col0", "tab4.col3", "tab4.col4") · - │ render 0 CASE WHEN agg0 AND (agg3 IS NOT NULL) THEN true WHEN agg0 IS NULL THEN false END · · - │ render 1 agg1 · · - │ render 2 agg2 · · - │ render 3 agg3 · · - └── group · · (rowid[hidden], agg0, agg1, agg2, agg3) · - │ aggregate 0 rowid · · - │ aggregate 1 bool_or(notnull) · · - │ aggregate 2 any_not_null(col0) · · - │ aggregate 3 any_not_null(col3) · · - │ aggregate 4 any_not_null(col4) · · - │ group by @4 · · - └── join · · (col0, col3, col4, rowid[hidden], "notnull", "tab4.col1") · - │ type left outer · · - │ pred (col4 = "tab4.col1") IS NOT false · · - ├── scan · · (col0, col3, col4, rowid[hidden]) · - │ table tab4@primary · · - │ spans ALL · · - └── render · · ("notnull", "tab4.col1") · - │ render 0 col1 IS NOT NULL · · - │ render 1 col1 · · - └── scan · · (col1) · -· table tab4@primary · · -· spans ALL · · -· filter col1 > 8.27 · · +render · · (col0) · + │ render 0 col0 · · + └── filter · · ("case", col0, col3, col4) · + │ filter ((col0 <= 0) AND (col4 <= 5.38)) OR ((("case" AND (col3 <= 5)) AND (col3 >= 7)) AND (col3 <= 9)) · · + └── render · · ("case", col0, col3, col4) · + │ render 0 CASE WHEN agg0 AND (agg3 IS NOT NULL) THEN true WHEN agg0 IS NULL THEN false END · · + │ render 1 agg1 · · + │ render 2 agg2 · · + │ render 3 agg3 · · + └── group · · (rowid[hidden], agg0, agg1, agg2, agg3) · + │ aggregate 0 rowid · · + │ aggregate 1 bool_or(notnull) · · + │ aggregate 2 any_not_null(col0) · · + │ aggregate 3 any_not_null(col3) · · + │ aggregate 4 any_not_null(col4) · · + │ group by @4 · · + └── join · · (col0, col3, col4, rowid[hidden], "notnull", col1) · + │ type left outer · · + │ pred (col4 = col1) IS NOT false · · + ├── scan · · (col0, col3, col4, rowid[hidden]) · + │ table tab4@primary · · + │ spans ALL · · + └── render · · ("notnull", col1) · + │ render 0 col1 IS NOT NULL · · + │ render 1 col1 · · + └── scan · · (col1) · +· table tab4@primary · · +· spans ALL · · +· filter col1 > 8.27 · · # ------------------------------------------------------------------------------ # Correlated subqueries. diff --git a/pkg/sql/opt/format.go b/pkg/sql/opt/format.go deleted file mode 100644 index 39adc989764e..000000000000 --- a/pkg/sql/opt/format.go +++ /dev/null @@ -1,83 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or -// implied. See the License for the specific language governing -// permissions and limitations under the License. - -package opt - -// ExprFmtFlags controls which properties of the expression are shown in -// formatted output. -type ExprFmtFlags int - -const ( - // ExprFmtShowAll shows all properties of the expression. - ExprFmtShowAll ExprFmtFlags = 0 - - // ExprFmtHideMiscProps does not show outer columns, row cardinality, or - // side effects in the output. - ExprFmtHideMiscProps ExprFmtFlags = 1 << (iota - 1) - - // ExprFmtHideConstraints does not show inferred constraints in the output. - ExprFmtHideConstraints - - // ExprFmtHideFuncDeps does not show functional dependencies in the output. - ExprFmtHideFuncDeps - - // ExprFmtHideRuleProps does not show rule-specific properties in the output. - ExprFmtHideRuleProps - - // ExprFmtHideStats does not show statistics in the output. - ExprFmtHideStats - - // ExprFmtHideCost does not show expression cost in the output. - ExprFmtHideCost - - // ExprFmtHideQualifications removes the qualification from column labels - // (except when a shortened name would be ambiguous). - ExprFmtHideQualifications - - // ExprFmtHideScalars removes subtrees that contain only scalars and replaces - // them with the SQL expression (if possible). - ExprFmtHideScalars - - // ExprFmtHideAll shows only the most basic properties of the expression. - ExprFmtHideAll ExprFmtFlags = (1 << iota) - 1 -) - -// HasFlags tests whether the given flags are all set. -func (f ExprFmtFlags) HasFlags(subset ExprFmtFlags) bool { - return f&subset == subset -} - -// ExprFmtCtx contains data relevant to formatting routines. -type ExprFmtCtx struct { - md *Metadata - flags ExprFmtFlags -} - -// MakeExprFmtCtx creates an expression formatting context. -func MakeExprFmtCtx(md *Metadata, flags ExprFmtFlags) ExprFmtCtx { - return ExprFmtCtx{ - md: md, - flags: flags, - } -} - -// Metadata returns the metadata relevant to this expression tree. -func (f *ExprFmtCtx) Metadata() *Metadata { - return f.md -} - -// HasFlags tests whether the given flags are all set. -func (f *ExprFmtCtx) HasFlags(subset ExprFmtFlags) bool { - return f.flags.HasFlags(subset) -} diff --git a/pkg/sql/opt/memo/expr_view.go b/pkg/sql/opt/memo/expr_view.go index b19c8caa2b71..2cc840389109 100644 --- a/pkg/sql/opt/memo/expr_view.go +++ b/pkg/sql/opt/memo/expr_view.go @@ -16,7 +16,6 @@ package memo import ( "bytes" - "fmt" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" @@ -173,7 +172,7 @@ func (ev ExprView) Private() interface{} { // operator types refer to the metadata in their private fields. For example, // the Scan operator holds a metadata table index. func (ev ExprView) Metadata() *opt.Metadata { - return ev.mem.metadata + return ev.mem.Metadata() } // Cost returns the cost of executing this expression tree, as estimated by the @@ -223,335 +222,18 @@ func (ev ExprView) bestExprID() BestExprID { // String returns a string representation of this expression for testing and // debugging. The output shows all properties of the expression. func (ev ExprView) String() string { - return ev.FormatString(opt.ExprFmtShowAll) + return ev.FormatString(ExprFmtShowAll) } // FormatString returns a string representation of this expression for testing // and debugging. The given flags control which properties are shown. -func (ev ExprView) FormatString(flags opt.ExprFmtFlags) string { - f := opt.MakeExprFmtCtx(ev.Metadata(), flags) +func (ev ExprView) FormatString(flags ExprFmtFlags) string { + f := MakeExprFmtCtx(&bytes.Buffer{}, flags, ev.mem) tp := treeprinter.New() ev.format(&f, tp) return tp.String() } -// format constructs a treeprinter view of this expression for testing and -// debugging. The given flags control which properties are added. -func (ev ExprView) format(f *opt.ExprFmtCtx, tp treeprinter.Node) { - if ExprFmtInterceptor != nil && ExprFmtInterceptor(f, tp, ev) { - return - } - if ev.IsScalar() { - ev.formatScalar(f, tp) - } else { - ev.formatRelational(f, tp) - } -} - -func (ev ExprView) formatRelational(f *opt.ExprFmtCtx, tp treeprinter.Node) { - var buf bytes.Buffer - formatter := ev.mem.makeExprFormatter(&buf) - - logProps := ev.Logical() - var physProps *props.Physical - if ev.best == normBestOrdinal { - physProps = &props.Physical{} - } else { - physProps = ev.Physical() - } - - // Special cases for merge-join and lookup-join: we want the type of the join - // to show up first. - switch ev.Operator() { - case opt.MergeJoinOp: - def := ev.Child(2).Private().(*MergeOnDef) - fmt.Fprintf(&buf, "%v (merge)", def.JoinType) - - case opt.LookupJoinOp: - def := ev.Private().(*LookupJoinDef) - fmt.Fprintf(&buf, "%v (lookup", def.JoinType) - formatter.formatPrivate(def, physProps, formatNormal) - buf.WriteByte(')') - - case opt.ScanOp, opt.VirtualScanOp, opt.IndexJoinOp, opt.ShowTraceForSessionOp: - fmt.Fprintf(&buf, "%v", ev.op) - formatter.formatPrivate(ev.Private(), physProps, formatNormal) - - default: - fmt.Fprintf(&buf, "%v", ev.op) - } - - tp = tp.Child(buf.String()) - - // If a particular column presentation is required of the expression, then - // print columns using that information. - if !physProps.Presentation.Any() { - ev.formatPresentation(f, tp, physProps.Presentation) - } else { - // Special handling to improve the columns display for certain ops. - switch ev.Operator() { - case opt.ProjectOp: - // We want the synthesized column IDs to map 1-to-1 to the projections, - // and the pass-through columns at the end. - - // Get the list of columns from the ProjectionsOp, which has the natural - // order. - def := ev.Child(1).Private().(*ProjectionsOpDef) - colList := append(opt.ColList(nil), def.SynthesizedCols...) - // Add pass-through columns. - def.PassthroughCols.ForEach(func(i int) { - colList = append(colList, opt.ColumnID(i)) - }) - - logProps.FormatColList(f, tp, "columns:", colList) - - case opt.ValuesOp: - colList := ev.Private().(opt.ColList) - logProps.FormatColList(f, tp, "columns:", colList) - - case opt.UnionOp, opt.IntersectOp, opt.ExceptOp, - opt.UnionAllOp, opt.IntersectAllOp, opt.ExceptAllOp: - colMap := ev.Private().(*SetOpColMap) - logProps.FormatColList(f, tp, "columns:", colMap.Out) - - default: - // Fall back to writing output columns in column id order, with - // best guess label. - logProps.FormatColSet(f, tp, "columns:", logProps.Relational.OutputCols) - } - } - - switch ev.Operator() { - // Special-case handling for GroupBy private; print grouping columns - // and internal ordering in addition to full set of columns. - case opt.GroupByOp, opt.ScalarGroupByOp, opt.DistinctOnOp: - def := ev.Private().(*GroupByDef) - if !def.GroupingCols.Empty() { - logProps.FormatColSet(f, tp, "grouping columns:", def.GroupingCols) - } - if !def.Ordering.Any() { - tp.Childf("internal-ordering: %s", def.Ordering) - } - - case opt.LimitOp, opt.OffsetOp: - if ord := ev.Private().(*props.OrderingChoice); !ord.Any() { - tp.Childf("internal-ordering: %s", ord) - } - - // Special-case handling for set operators to show the left and right - // input columns that correspond to the output columns. - case opt.UnionOp, opt.IntersectOp, opt.ExceptOp, - opt.UnionAllOp, opt.IntersectAllOp, opt.ExceptAllOp: - colMap := ev.Private().(*SetOpColMap) - logProps.FormatColList(f, tp, "left columns:", colMap.Left) - logProps.FormatColList(f, tp, "right columns:", colMap.Right) - - case opt.ScanOp: - def := ev.Private().(*ScanOpDef) - if def.Constraint != nil { - tp.Childf("constraint: %s", def.Constraint) - } - if def.HardLimit.IsSet() { - tp.Childf("limit: %s", def.HardLimit) - } - if !def.Flags.Empty() { - if def.Flags.NoIndexJoin { - tp.Childf("flags: no-index-join") - } else if def.Flags.ForceIndex { - idx := ev.Metadata().Table(def.Table).Index(def.Flags.Index) - tp.Childf("flags: force-index=%s", idx.IdxName()) - } - } - - case opt.LookupJoinOp: - def := ev.Private().(*LookupJoinDef) - buf.Reset() - idxCols := make(opt.ColList, len(def.KeyCols)) - idx := ev.mem.metadata.Table(def.Table).Index(def.Index) - for i := range idxCols { - idxCols[i] = def.Table.ColumnID(idx.Column(i).Ordinal) - } - tp.Childf("key columns: %v = %v", def.KeyCols, idxCols) - } - - if !f.HasFlags(opt.ExprFmtHideMiscProps) { - if !logProps.Relational.OuterCols.Empty() { - tp.Childf("outer: %s", logProps.Relational.OuterCols.String()) - } - if logProps.Relational.Cardinality != props.AnyCardinality { - // Suppress cardinality for Scan ops if it's redundant with Limit field. - if !(ev.Operator() == opt.ScanOp && ev.Private().(*ScanOpDef).HardLimit.IsSet()) { - tp.Childf("cardinality: %s", logProps.Relational.Cardinality) - } - } - if logProps.Relational.CanHaveSideEffects { - tp.Child("side-effects") - } - } - - if !f.HasFlags(opt.ExprFmtHideStats) { - tp.Childf("stats: %s", &logProps.Relational.Stats) - } - - if !f.HasFlags(opt.ExprFmtHideCost) && ev.best != normBestOrdinal { - tp.Childf("cost: %.9g", ev.bestExpr().cost) - } - - // Format functional dependencies. - if !f.HasFlags(opt.ExprFmtHideFuncDeps) { - // Show the key separately from the rest of the FDs. Do this by copying - // the FD to the stack (fast shallow copy), and then calling ClearKey. - fd := logProps.Relational.FuncDeps - key, ok := fd.Key() - if ok { - tp.Childf("key: %s", key) - } - fd.ClearKey() - if !fd.Empty() { - tp.Childf("fd: %s", fd) - } - } - - if !physProps.Ordering.Any() { - tp.Childf("ordering: %s", physProps.Ordering.String()) - } - - if !f.HasFlags(opt.ExprFmtHideRuleProps) { - r := &logProps.Relational.Rule - if !r.PruneCols.Empty() { - tp.Childf("prune: %s", r.PruneCols.String()) - } - if !r.RejectNullCols.Empty() { - tp.Childf("reject-nulls: %s", r.RejectNullCols.String()) - } - if len(r.InterestingOrderings) > 0 { - tp.Childf("interesting orderings: %s", r.InterestingOrderings.String()) - } - } - - for i := 0; i < ev.ChildCount(); i++ { - ev.Child(i).format(f, tp) - } -} - -func (ev ExprView) formatScalar(f *opt.ExprFmtCtx, tp treeprinter.Node) { - // Omit empty ProjectionsOp and AggregationsOp. - if (ev.op == opt.ProjectionsOp || ev.op == opt.AggregationsOp) && - ev.ChildCount() == 0 { - return - } - if ev.op == opt.MergeOnOp { - tp = tp.Childf("%v", ev.op) - def := ev.Private().(*MergeOnDef) - tp.Childf("left ordering: %s", def.LeftEq) - tp.Childf("right ordering: %s", def.RightEq) - } else { - var buf bytes.Buffer - fmt.Fprintf(&buf, "%v", ev.op) - - ev.formatScalarPrivate(&buf, ev.Private()) - ev.FormatScalarProps(f, &buf) - tp = tp.Child(buf.String()) - } - for i := 0; i < ev.ChildCount(); i++ { - child := ev.Child(i) - child.format(f, tp) - } -} - -// FormatScalarProps writes out a string representation of the scalar -// properties (with a preceding space); for example: -// " [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)]" -func (ev ExprView) FormatScalarProps(f *opt.ExprFmtCtx, buf *bytes.Buffer) { - // Don't panic if scalar properties don't yet exist when printing - // expression. - scalar := ev.Logical().Scalar - if scalar == nil { - buf.WriteString(" [type=undefined]") - } else { - first := true - writeProp := func(format string, args ...interface{}) { - if first { - buf.WriteString(" [") - first = false - } else { - buf.WriteString(", ") - } - fmt.Fprintf(buf, format, args...) - } - - switch ev.Operator() { - case opt.ProjectionsOp, opt.AggregationsOp: - // Don't show the type of these ops because they are simply tuple - // types of their children's types, and the types of children are - // already listed. - - default: - writeProp("type=%s", scalar.Type) - } - - if !f.HasFlags(opt.ExprFmtHideMiscProps) { - if !scalar.OuterCols.Empty() { - writeProp("outer=%s", scalar.OuterCols) - } - if scalar.CanHaveSideEffects { - writeProp("side-effects") - } - } - - if !f.HasFlags(opt.ExprFmtHideConstraints) { - if scalar.Constraints != nil && !scalar.Constraints.IsUnconstrained() { - writeProp("constraints=(%s", scalar.Constraints) - if scalar.TightConstraints { - buf.WriteString("; tight") - } - buf.WriteString(")") - } - } - - if !f.HasFlags(opt.ExprFmtHideFuncDeps) && !scalar.FuncDeps.Empty() { - writeProp("fd=%s", scalar.FuncDeps) - } - - if !first { - buf.WriteString("]") - } - } -} - -func (ev ExprView) formatScalarPrivate(buf *bytes.Buffer, private interface{}) { - switch ev.op { - case opt.NullOp, opt.TupleOp: - // Private is redundant with logical type property. - private = nil - - case opt.ProjectionsOp, opt.AggregationsOp: - // The private data of these ops was already used to print the output - // columns for their containing op (Project or GroupBy), so no need to - // print again. - private = nil - } - - if private != nil { - buf.WriteRune(':') - formatter := ev.mem.makeExprFormatter(buf) - formatter.formatPrivate(private, &props.Physical{}, formatNormal) - } -} - -func (ev ExprView) formatPresentation( - f *opt.ExprFmtCtx, tp treeprinter.Node, presentation props.Presentation, -) { - logProps := ev.Logical() - - var buf bytes.Buffer - buf.WriteString("columns:") - for _, col := range presentation { - logProps.FormatCol(f, &buf, col.Label, col.ID) - } - tp.Child(buf.String()) -} - // HasOnlyConstChildren returns true if all children of ev are constant values // (tuples of constant values are considered constant values). func HasOnlyConstChildren(ev ExprView) bool { @@ -575,4 +257,4 @@ func MatchesTupleOfConstants(ev ExprView) bool { // ExprFmtInterceptor is a callback that can be set to a custom formatting // function. If the function returns true, the normal formatting code is bypassed. -var ExprFmtInterceptor func(f *opt.ExprFmtCtx, tp treeprinter.Node, ev ExprView) bool +var ExprFmtInterceptor func(f *ExprFmtCtx, tp treeprinter.Node, ev ExprView) bool diff --git a/pkg/sql/opt/memo/expr_view_format.go b/pkg/sql/opt/memo/expr_view_format.go new file mode 100644 index 000000000000..b3b8bfa1aa25 --- /dev/null +++ b/pkg/sql/opt/memo/expr_view_format.go @@ -0,0 +1,560 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package memo + +import ( + "bytes" + "fmt" + "unicode" + + "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/sql/opt/props" + "github.com/cockroachdb/cockroach/pkg/sql/sem/types" + "github.com/cockroachdb/cockroach/pkg/util/treeprinter" +) + +// ExprFmtFlags controls which properties of the expression are shown in +// formatted output. +type ExprFmtFlags int + +const ( + // ExprFmtShowAll shows all properties of the expression. + ExprFmtShowAll ExprFmtFlags = 0 + + // ExprFmtHideMiscProps does not show outer columns, row cardinality, or + // side effects in the output. + ExprFmtHideMiscProps ExprFmtFlags = 1 << (iota - 1) + + // ExprFmtHideConstraints does not show inferred constraints in the output. + ExprFmtHideConstraints + + // ExprFmtHideFuncDeps does not show functional dependencies in the output. + ExprFmtHideFuncDeps + + // ExprFmtHideRuleProps does not show rule-specific properties in the output. + ExprFmtHideRuleProps + + // ExprFmtHideStats does not show statistics in the output. + ExprFmtHideStats + + // ExprFmtHideCost does not show expression cost in the output. + ExprFmtHideCost + + // ExprFmtHideQualifications removes the qualification from column labels + // (except when a shortened name would be ambiguous). + ExprFmtHideQualifications + + // ExprFmtHideScalars removes subtrees that contain only scalars and replaces + // them with the SQL expression (if possible). + ExprFmtHideScalars + + // ExprFmtHideAll shows only the most basic properties of the expression. + ExprFmtHideAll ExprFmtFlags = (1 << iota) - 1 +) + +// HasFlags tests whether the given flags are all set. +func (f ExprFmtFlags) HasFlags(subset ExprFmtFlags) bool { + return f&subset == subset +} + +// ExprFmtCtx contains data relevant to formatting routines. +type ExprFmtCtx struct { + Buffer *bytes.Buffer + + // Flags controls how the expression is formatted. + Flags ExprFmtFlags + + // Memo must contain any expression that is formatted. + Memo *Memo +} + +// MakeExprFmtCtx creates an expression formatting context from an existing +// buffer. +func MakeExprFmtCtx(buf *bytes.Buffer, flags ExprFmtFlags, mem *Memo) ExprFmtCtx { + return ExprFmtCtx{Buffer: buf, Flags: flags, Memo: mem} +} + +// HasFlags tests whether the given flags are all set. +func (f *ExprFmtCtx) HasFlags(subset ExprFmtFlags) bool { + return f.Flags.HasFlags(subset) +} + +// format constructs a treeprinter view of this expression for testing and +// debugging. The given flags control which properties are added. +func (ev ExprView) format(f *ExprFmtCtx, tp treeprinter.Node) { + if ExprFmtInterceptor != nil && ExprFmtInterceptor(f, tp, ev) { + return + } + if ev.IsScalar() { + ev.formatScalar(f, tp) + } else { + ev.formatRelational(f, tp) + } +} + +func (ev ExprView) formatRelational(f *ExprFmtCtx, tp treeprinter.Node) { + logProps := ev.Logical() + var physProps *props.Physical + if ev.best == normBestOrdinal { + physProps = &props.Physical{} + } else { + physProps = ev.Physical() + } + + // Special cases for merge-join and lookup-join: we want the type of the join + // to show up first. + f.Buffer.Reset() + switch ev.Operator() { + case opt.MergeJoinOp: + def := ev.Child(2).Private().(*MergeOnDef) + fmt.Fprintf(f.Buffer, "%v (merge)", def.JoinType) + + case opt.LookupJoinOp: + def := ev.Private().(*LookupJoinDef) + fmt.Fprintf(f.Buffer, "%v (lookup", def.JoinType) + formatPrivate(f, def, physProps) + f.Buffer.WriteByte(')') + + case opt.ScanOp, opt.VirtualScanOp, opt.IndexJoinOp, opt.ShowTraceForSessionOp: + fmt.Fprintf(f.Buffer, "%v", ev.op) + formatPrivate(f, ev.Private(), physProps) + + default: + fmt.Fprintf(f.Buffer, "%v", ev.op) + } + + tp = tp.Child(f.Buffer.String()) + + // If a particular column presentation is required of the expression, then + // print columns using that information. + if !physProps.Presentation.Any() { + ev.formatPresentation(f, tp, physProps.Presentation) + } else { + // Special handling to improve the columns display for certain ops. + switch ev.Operator() { + case opt.ProjectOp: + // We want the synthesized column IDs to map 1-to-1 to the projections, + // and the pass-through columns at the end. + + // Get the list of columns from the ProjectionsOp, which has the natural + // order. + def := ev.Child(1).Private().(*ProjectionsOpDef) + colList := append(opt.ColList(nil), def.SynthesizedCols...) + // Add pass-through columns. + def.PassthroughCols.ForEach(func(i int) { + colList = append(colList, opt.ColumnID(i)) + }) + + ev.formatColList(f, tp, "columns:", colList) + + case opt.ValuesOp: + colList := ev.Private().(opt.ColList) + ev.formatColList(f, tp, "columns:", colList) + + case opt.UnionOp, opt.IntersectOp, opt.ExceptOp, + opt.UnionAllOp, opt.IntersectAllOp, opt.ExceptAllOp: + colMap := ev.Private().(*SetOpColMap) + ev.formatColList(f, tp, "columns:", colMap.Out) + + default: + // Fall back to writing output columns in column id order, with + // best guess label. + ev.formatColSet(f, tp, "columns:", logProps.Relational.OutputCols) + } + } + + switch ev.Operator() { + // Special-case handling for GroupBy private; print grouping columns + // and internal ordering in addition to full set of columns. + case opt.GroupByOp, opt.ScalarGroupByOp, opt.DistinctOnOp: + def := ev.Private().(*GroupByDef) + if !def.GroupingCols.Empty() { + ev.formatColSet(f, tp, "grouping columns:", def.GroupingCols) + } + if !def.Ordering.Any() { + tp.Childf("internal-ordering: %s", def.Ordering) + } + + case opt.LimitOp, opt.OffsetOp: + if ord := ev.Private().(*props.OrderingChoice); !ord.Any() { + tp.Childf("internal-ordering: %s", ord) + } + + // Special-case handling for set operators to show the left and right + // input columns that correspond to the output columns. + case opt.UnionOp, opt.IntersectOp, opt.ExceptOp, + opt.UnionAllOp, opt.IntersectAllOp, opt.ExceptAllOp: + colMap := ev.Private().(*SetOpColMap) + ev.formatColList(f, tp, "left columns:", colMap.Left) + ev.formatColList(f, tp, "right columns:", colMap.Right) + + case opt.ScanOp: + def := ev.Private().(*ScanOpDef) + if def.Constraint != nil { + tp.Childf("constraint: %s", def.Constraint) + } + if def.HardLimit.IsSet() { + tp.Childf("limit: %s", def.HardLimit) + } + if !def.Flags.Empty() { + if def.Flags.NoIndexJoin { + tp.Childf("flags: no-index-join") + } else if def.Flags.ForceIndex { + idx := ev.Metadata().Table(def.Table).Index(def.Flags.Index) + tp.Childf("flags: force-index=%s", idx.IdxName()) + } + } + + case opt.LookupJoinOp: + def := ev.Private().(*LookupJoinDef) + idxCols := make(opt.ColList, len(def.KeyCols)) + idx := ev.mem.metadata.Table(def.Table).Index(def.Index) + for i := range idxCols { + idxCols[i] = def.Table.ColumnID(idx.Column(i).Ordinal) + } + tp.Childf("key columns: %v = %v", def.KeyCols, idxCols) + } + + if !f.HasFlags(ExprFmtHideMiscProps) { + if !logProps.Relational.OuterCols.Empty() { + tp.Childf("outer: %s", logProps.Relational.OuterCols.String()) + } + if logProps.Relational.Cardinality != props.AnyCardinality { + // Suppress cardinality for Scan ops if it's redundant with Limit field. + if !(ev.Operator() == opt.ScanOp && ev.Private().(*ScanOpDef).HardLimit.IsSet()) { + tp.Childf("cardinality: %s", logProps.Relational.Cardinality) + } + } + if logProps.Relational.CanHaveSideEffects { + tp.Child("side-effects") + } + } + + if !f.HasFlags(ExprFmtHideStats) { + tp.Childf("stats: %s", &logProps.Relational.Stats) + } + + if !f.HasFlags(ExprFmtHideCost) && ev.best != normBestOrdinal { + tp.Childf("cost: %.9g", ev.bestExpr().cost) + } + + // Format functional dependencies. + if !f.HasFlags(ExprFmtHideFuncDeps) { + // Show the key separately from the rest of the FDs. Do this by copying + // the FD to the stack (fast shallow copy), and then calling ClearKey. + fd := logProps.Relational.FuncDeps + key, ok := fd.Key() + if ok { + tp.Childf("key: %s", key) + } + fd.ClearKey() + if !fd.Empty() { + tp.Childf("fd: %s", fd) + } + } + + if !physProps.Ordering.Any() { + tp.Childf("ordering: %s", physProps.Ordering.String()) + } + + if !f.HasFlags(ExprFmtHideRuleProps) { + r := &logProps.Relational.Rule + if !r.PruneCols.Empty() { + tp.Childf("prune: %s", r.PruneCols.String()) + } + if !r.RejectNullCols.Empty() { + tp.Childf("reject-nulls: %s", r.RejectNullCols.String()) + } + if len(r.InterestingOrderings) > 0 { + tp.Childf("interesting orderings: %s", r.InterestingOrderings.String()) + } + } + + for i := 0; i < ev.ChildCount(); i++ { + ev.Child(i).format(f, tp) + } +} + +func (ev ExprView) formatScalar(f *ExprFmtCtx, tp treeprinter.Node) { + // Omit empty ProjectionsOp and AggregationsOp. + if (ev.op == opt.ProjectionsOp || ev.op == opt.AggregationsOp) && + ev.ChildCount() == 0 { + return + } + if ev.op == opt.MergeOnOp { + tp = tp.Childf("%v", ev.op) + def := ev.Private().(*MergeOnDef) + tp.Childf("left ordering: %s", def.LeftEq) + tp.Childf("right ordering: %s", def.RightEq) + } else { + f.Buffer.Reset() + fmt.Fprintf(f.Buffer, "%v", ev.op) + + ev.formatScalarPrivate(f, ev.Private()) + ev.FormatScalarProps(f) + tp = tp.Child(f.Buffer.String()) + } + for i := 0; i < ev.ChildCount(); i++ { + child := ev.Child(i) + child.format(f, tp) + } +} + +// FormatScalarProps writes out a string representation of the scalar +// properties (with a preceding space); for example: +// " [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)]" +func (ev ExprView) FormatScalarProps(f *ExprFmtCtx) { + // Don't panic if scalar properties don't yet exist when printing + // expression. + scalar := ev.Logical().Scalar + if scalar == nil { + f.Buffer.WriteString(" [type=undefined]") + } else { + first := true + writeProp := func(format string, args ...interface{}) { + if first { + f.Buffer.WriteString(" [") + first = false + } else { + f.Buffer.WriteString(", ") + } + fmt.Fprintf(f.Buffer, format, args...) + } + + switch ev.Operator() { + case opt.ProjectionsOp, opt.AggregationsOp: + // Don't show the type of these ops because they are simply tuple + // types of their children's types, and the types of children are + // already listed. + + default: + writeProp("type=%s", scalar.Type) + } + + if !f.HasFlags(ExprFmtHideMiscProps) { + if !scalar.OuterCols.Empty() { + writeProp("outer=%s", scalar.OuterCols) + } + if scalar.CanHaveSideEffects { + writeProp("side-effects") + } + } + + if !f.HasFlags(ExprFmtHideConstraints) { + if scalar.Constraints != nil && !scalar.Constraints.IsUnconstrained() { + writeProp("constraints=(%s", scalar.Constraints) + if scalar.TightConstraints { + f.Buffer.WriteString("; tight") + } + f.Buffer.WriteString(")") + } + } + + if !f.HasFlags(ExprFmtHideFuncDeps) && !scalar.FuncDeps.Empty() { + writeProp("fd=%s", scalar.FuncDeps) + } + + if !first { + f.Buffer.WriteString("]") + } + } +} + +func (ev ExprView) formatScalarPrivate(f *ExprFmtCtx, private interface{}) { + switch ev.op { + case opt.NullOp, opt.TupleOp: + // Private is redundant with logical type property. + private = nil + + case opt.ProjectionsOp, opt.AggregationsOp: + // The private data of these ops was already used to print the output + // columns for their containing op (Project or GroupBy), so no need to + // print again. + private = nil + } + + if private != nil { + f.Buffer.WriteRune(':') + formatPrivate(f, private, &props.Physical{}) + } +} + +func (ev ExprView) formatPresentation( + f *ExprFmtCtx, tp treeprinter.Node, presentation props.Presentation, +) { + logProps := ev.Logical() + + f.Buffer.Reset() + f.Buffer.WriteString("columns:") + for _, col := range presentation { + formatCol(f, col.Label, col.ID, logProps.Relational.NotNullCols) + } + tp.Child(f.Buffer.String()) +} + +// formatColSet constructs a new treeprinter child containing the specified set +// of columns formatting using the formatCol method. +func (ev ExprView) formatColSet( + f *ExprFmtCtx, tp treeprinter.Node, heading string, colSet opt.ColSet, +) { + if !colSet.Empty() { + notNullCols := ev.Logical().Relational.NotNullCols + f.Buffer.Reset() + f.Buffer.WriteString(heading) + colSet.ForEach(func(i int) { + formatCol(f, "", opt.ColumnID(i), notNullCols) + }) + tp.Child(f.Buffer.String()) + } +} + +// formatColList constructs a new treeprinter child containing the specified +// list of columns formatted using the formatCol method. +func (ev ExprView) formatColList( + f *ExprFmtCtx, tp treeprinter.Node, heading string, colList opt.ColList, +) { + if len(colList) > 0 { + notNullCols := ev.Logical().Relational.NotNullCols + f.Buffer.Reset() + f.Buffer.WriteString(heading) + for _, col := range colList { + formatCol(f, "", col, notNullCols) + } + tp.Child(f.Buffer.String()) + } +} + +// formatCol outputs the specified column into the context's buffer using the +// following format: +// label:index(type) +// +// If the column is not nullable, then this is the format: +// label:index(type!null) +// +// If a label is given, then it is used. Otherwise, a "best effort" label is +// used from query metadata. +func formatCol(f *ExprFmtCtx, label string, id opt.ColumnID, notNullCols opt.ColSet) { + md := f.Memo.metadata + if label == "" { + fullyQualify := !f.HasFlags(ExprFmtHideQualifications) + label = md.QualifiedColumnLabel(id, fullyQualify) + } + + if !isSimpleColumnName(label) { + // Add quotations around the column name if it is not composed of simple + // ASCII characters. + label = "\"" + label + "\"" + } + + typ := md.ColumnType(id) + f.Buffer.WriteByte(' ') + f.Buffer.WriteString(label) + f.Buffer.WriteByte(':') + fmt.Fprintf(f.Buffer, "%d", id) + f.Buffer.WriteByte('(') + f.Buffer.WriteString(typ.String()) + + if notNullCols.Contains(int(id)) { + f.Buffer.WriteString("!null") + } + f.Buffer.WriteByte(')') +} + +func formatPrivate(f *ExprFmtCtx, private interface{}, physProps *props.Physical) { + if private == nil { + return + } + switch t := private.(type) { + case *ScanOpDef: + // Don't output name of index if it's the primary index. + tab := f.Memo.metadata.Table(t.Table) + if t.Index == opt.PrimaryIndex { + fmt.Fprintf(f.Buffer, " %s", tab.Name().TableName) + } else { + fmt.Fprintf(f.Buffer, " %s@%s", tab.Name().TableName, tab.Index(t.Index).IdxName()) + } + if _, reverse := t.CanProvideOrdering(f.Memo.Metadata(), &physProps.Ordering); reverse { + f.Buffer.WriteString(",rev") + } + + case *VirtualScanOpDef: + tab := f.Memo.metadata.Table(t.Table) + fmt.Fprintf(f.Buffer, " %s", tab.Name()) + + case *RowNumberDef: + if !t.Ordering.Any() { + fmt.Fprintf(f.Buffer, " ordering=%s", t.Ordering) + } + + case *GroupByDef: + fmt.Fprintf(f.Buffer, " cols=%s", t.GroupingCols.String()) + if !t.Ordering.Any() { + fmt.Fprintf(f.Buffer, ",ordering=%s", t.Ordering) + } + + case opt.ColumnID: + fullyQualify := !f.HasFlags(ExprFmtHideQualifications) + label := f.Memo.metadata.QualifiedColumnLabel(t, fullyQualify) + fmt.Fprintf(f.Buffer, " %s", label) + + case *IndexJoinDef: + tab := f.Memo.metadata.Table(t.Table) + fmt.Fprintf(f.Buffer, " %s", tab.Name().TableName) + + case *LookupJoinDef: + tab := f.Memo.metadata.Table(t.Table) + if t.Index == opt.PrimaryIndex { + fmt.Fprintf(f.Buffer, " %s", tab.Name().TableName) + } else { + fmt.Fprintf(f.Buffer, " %s@%s", tab.Name().TableName, tab.Index(t.Index).IdxName()) + } + + case *MergeOnDef: + fmt.Fprintf(f.Buffer, " %s,%s,%s", t.JoinType, t.LeftEq, t.RightEq) + + case *props.OrderingChoice: + if !t.Any() { + fmt.Fprintf(f.Buffer, " ordering=%s", t) + } + + case *ExplainOpDef, *ProjectionsOpDef, opt.ColSet, opt.ColList, *SetOpColMap, types.T: + // Don't show anything, because it's mostly redundant. + + default: + fmt.Fprintf(f.Buffer, " %v", private) + } +} + +// isSimpleColumnName returns true if the given label consists of only ASCII +// letters, numbers, underscores, quotation marks, and periods ("."). It is +// used to determine whether to enclose a column name in quotation marks for +// nicer display. +func isSimpleColumnName(label string) bool { + for i, r := range label { + if r > unicode.MaxASCII { + return false + } + + if i == 0 { + if r != '"' && !unicode.IsLetter(r) { + // The first character must be a letter or quotation mark. + return false + } + } else if r != '.' && r != '_' && r != '"' && !unicode.IsNumber(r) && !unicode.IsLetter(r) { + return false + } + } + return true +} diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 7bbd9bb85453..df3f4415ab22 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -33,6 +33,7 @@ var fdAnnID = opt.NewTableAnnID() type logicalPropsBuilder struct { evalCtx *tree.EvalContext sb statisticsBuilder + kb keyBuffer } // buildProps is called by the memo group construction code in order to @@ -42,7 +43,8 @@ type logicalPropsBuilder struct { // NOTE: The parent expression is passed as an ExprView for convenient access // to children, but certain properties on it are not yet defined (like // its logical properties!). -func (b *logicalPropsBuilder) buildProps(ev ExprView) props.Logical { +func (b *logicalPropsBuilder) buildProps(evalCtx *tree.EvalContext, ev ExprView) props.Logical { + b.evalCtx = evalCtx if ev.IsRelational() { return b.buildRelationalProps(ev) } @@ -170,7 +172,7 @@ func (b *logicalPropsBuilder) buildScanProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildScan(ev, relational) return logical @@ -206,7 +208,7 @@ func (b *logicalPropsBuilder) buildVirtualScanProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildVirtualScan(ev, relational) return logical @@ -268,7 +270,7 @@ func (b *logicalPropsBuilder) buildSelectProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildSelect(ev, relational) return logical @@ -358,7 +360,7 @@ func (b *logicalPropsBuilder) buildProjectProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildProject(ev, relational) return logical @@ -519,7 +521,7 @@ func (b *logicalPropsBuilder) buildJoinProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildJoin(ev, relational) return logical @@ -564,7 +566,7 @@ func (b *logicalPropsBuilder) buildIndexJoinProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildIndexJoin(ev, relational) return logical @@ -627,7 +629,7 @@ func (b *logicalPropsBuilder) buildGroupByProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildGroupBy(ev, relational) return logical @@ -685,7 +687,7 @@ func (b *logicalPropsBuilder) buildSetProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildSetOp(ev, relational) return logical @@ -726,7 +728,7 @@ func (b *logicalPropsBuilder) buildValuesProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildValues(ev, relational) return logical @@ -857,7 +859,7 @@ func (b *logicalPropsBuilder) buildLimitProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildLimit(ev, relational) return logical @@ -912,7 +914,7 @@ func (b *logicalPropsBuilder) buildOffsetProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildOffset(ev, relational) return logical @@ -951,7 +953,7 @@ func (b *logicalPropsBuilder) buildMax1RowProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildMax1Row(ev, relational) return logical @@ -1000,7 +1002,7 @@ func (b *logicalPropsBuilder) buildRowNumberProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildRowNumber(ev, relational) return logical @@ -1037,7 +1039,7 @@ func (b *logicalPropsBuilder) buildZipProps(ev ExprView) props.Logical { // Statistics // ---------- - b.sb.init(b.evalCtx, &keyBuffer{}) + b.sb.init(b.evalCtx, &b.kb) b.sb.buildZip(ev, relational) return logical diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index 665a9e12e33e..f31e1da0b2cf 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -15,6 +15,8 @@ package memo import ( + "bytes" + "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -106,7 +108,7 @@ const ( type Memo struct { // metadata provides information about the columns and tables used in this // particular query. - metadata *opt.Metadata + metadata opt.Metadata // exprMap maps from expression fingerprint (Expr.fingerprint()) to // that expression's group. Multiple different fingerprints can map to the @@ -119,6 +121,10 @@ type Memo struct { // expression to indicate that it did not originate from the memo. groups []group + // logPropsBuilder is inlined in the memo so that it can be reused each time + // scalar or relational properties need to be built. + logPropsBuilder logicalPropsBuilder + // Some memoExprs have a variable number of children. The Expr stores // the list as a ListID struct, which is a slice of an array maintained by // listStorage. Note that ListID 0 is invalid in order to indicate an @@ -141,9 +147,8 @@ func New() *Memo { // index 0 for private data, index 0 for physical properties, and index 0 // for lists are all reserved. m := &Memo{ - metadata: opt.NewMetadata(), - exprMap: make(map[Fingerprint]GroupID), - groups: make([]group, 1, 8), + exprMap: make(map[Fingerprint]GroupID), + groups: make([]group, 1, 8), } m.privateStorage.init() @@ -152,7 +157,7 @@ func New() *Memo { // Metadata returns the metadata instance associated with the memo. func (m *Memo) Metadata() *opt.Metadata { - return m.metadata + return &m.metadata } // Root returns the root of the memo's lowest cost expression tree. It can only @@ -258,8 +263,7 @@ func (m *Memo) MemoizeNormExpr(evalCtx *tree.EvalContext, norm Expr) GroupID { } mgrp := m.newGroup(norm) ev := MakeNormExprView(m, mgrp.id) - logPropsFactory := logicalPropsBuilder{evalCtx: evalCtx} - mgrp.logical = logPropsFactory.buildProps(ev) + mgrp.logical = m.logPropsBuilder.buildProps(evalCtx, ev) return mgrp.id } @@ -393,6 +397,5 @@ func (m *Memo) String() string { // FormatString returns a string representation of this memo for testing // and debugging. The given flags control which properties are shown. func (m *Memo) FormatString(flags FmtFlags) string { - formatter := m.makeMemoFormatter(flags) - return formatter.format() + return m.format(&memoFmtCtx{buf: &bytes.Buffer{}, flags: flags}) } diff --git a/pkg/sql/opt/memo/memo_format.go b/pkg/sql/opt/memo/memo_format.go index 4071c6b2e5e7..db1028c091d0 100644 --- a/pkg/sql/opt/memo/memo_format.go +++ b/pkg/sql/opt/memo/memo_format.go @@ -22,66 +22,33 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" - "github.com/cockroachdb/cockroach/pkg/sql/sem/types" "github.com/cockroachdb/cockroach/pkg/util/treeprinter" ) -type exprFormatter struct { - mem *Memo - buf *bytes.Buffer -} - -type memoFormatter struct { - exprFormatter +type memoFmtCtx struct { + buf *bytes.Buffer flags FmtFlags ordering []GroupID numbering []GroupID } -// formatMode controls the formatting depending on the context. -type formatMode int8 - -const ( - // formatNormal is used when we are printing expressions. - formatNormal formatMode = iota - // formatMemo is used when we are printing the memo. - formatMemo -) - -func (m *Memo) makeExprFormatter(buf *bytes.Buffer) exprFormatter { - return exprFormatter{ - mem: m, - buf: buf, - } -} - -func (m *Memo) makeMemoFormatter(flags FmtFlags) memoFormatter { - return memoFormatter{ - exprFormatter: exprFormatter{ - mem: m, - buf: &bytes.Buffer{}, - }, - flags: flags, - } -} - -func (f *memoFormatter) format() string { +func (m *Memo) format(f *memoFmtCtx) string { // If requested, we topological sort the memo with respect to its root group. // Otherwise, we simply print out all the groups in the order and with the // names they're referred to physically. - if f.flags.HasFlags(FmtRaw) || !f.mem.isOptimized() { + if f.flags.HasFlags(FmtRaw) || !m.isOptimized() { // In this case we renumber with the identity mapping, so the groups have // the same numbers as they're represented with internally. - f.ordering = make([]GroupID, len(f.mem.groups)-1) - for i := range f.mem.groups[1:] { + f.ordering = make([]GroupID, len(m.groups)-1) + for i := range m.groups[1:] { f.ordering[i] = GroupID(i + 1) } } else { - f.ordering = f.sortGroups(f.mem.root.group) + f.ordering = m.sortGroups(m.root.group) } // We renumber the groups so that they're still printed in order from 1..N. - f.numbering = make([]GroupID, len(f.mem.groups)) + f.numbering = make([]GroupID, len(m.groups)) for i := range f.ordering { f.numbering[f.ordering[i]] = GroupID(i + 1) } @@ -89,51 +56,93 @@ func (f *memoFormatter) format() string { tp := treeprinter.New() var root treeprinter.Node - if f.mem.isOptimized() { + if m.isOptimized() { root = tp.Child("memo (optimized)") } else { root = tp.Child("memo (not optimized)") } for i := range f.ordering { - mgrp := &f.mem.groups[f.ordering[i]] + mgrp := &m.groups[f.ordering[i]] f.buf.Reset() for ord := 0; ord < mgrp.exprCount(); ord++ { if ord != 0 { f.buf.WriteByte(' ') } - f.formatExpr(mgrp.expr(ExprOrdinal(ord))) + m.formatExpr(f, mgrp.expr(ExprOrdinal(ord))) } child := root.Childf("G%d: %s", i+1, f.buf.String()) - f.formatBestExprSet(child, mgrp) + m.formatBestExprSet(f, child, mgrp) } // If showing raw memo, then add header text to point to root expression if // it's available. - if f.flags.HasFlags(FmtRaw) && f.mem.isOptimized() { - ev := f.mem.Root() + if f.flags.HasFlags(FmtRaw) && m.isOptimized() { + ev := m.Root() return fmt.Sprintf("root: G%d, %s\n%s", f.numbering[ev.Group()], ev.Physical(), tp.String()) } return tp.String() } -func (f *memoFormatter) formatExpr(e *Expr) { +func (m *Memo) formatExpr(f *memoFmtCtx, e *Expr) { fmt.Fprintf(f.buf, "(%s", e.op) for i := 0; i < e.ChildCount(); i++ { - fmt.Fprintf(f.buf, " G%d", f.numbering[e.ChildGroup(f.mem, i)]) + fmt.Fprintf(f.buf, " G%d", f.numbering[e.ChildGroup(m, i)]) } - f.formatPrivate(e.Private(f.mem), &props.Physical{}, formatMemo) + m.formatPrivate(f, e.Private(m), &props.Physical{}) f.buf.WriteString(")") } +func (m *Memo) formatPrivate(f *memoFmtCtx, private interface{}, physProps *props.Physical) { + if private == nil { + return + } + + // Start by using private expression formatting. + ef := MakeExprFmtCtx(f.buf, ExprFmtHideAll, m) + formatPrivate(&ef, private, physProps) + + // Now append additional information that's useful in the memo case. + switch t := private.(type) { + case *ScanOpDef: + tab := m.metadata.Table(t.Table) + if tab.ColumnCount() != t.Cols.Len() { + fmt.Fprintf(f.buf, ",cols=%s", t.Cols) + } + if t.Constraint != nil { + fmt.Fprintf(f.buf, ",constrained") + } + if t.HardLimit.IsSet() { + fmt.Fprintf(f.buf, ",lim=%s", t.HardLimit) + } + + case *IndexJoinDef: + fmt.Fprintf(f.buf, ",cols=%s", t.Cols) + + case *LookupJoinDef: + fmt.Fprintf(f.buf, ",keyCols=%v,lookupCols=%s", t.KeyCols, t.LookupCols) + + case *ExplainOpDef: + propsStr := t.Props.String() + if propsStr != "" { + fmt.Fprintf(f.buf, " %s", propsStr) + } + + case *ProjectionsOpDef: + t.PassthroughCols.ForEach(func(i int) { + fmt.Fprintf(f.buf, " %s", m.metadata.ColumnLabel(opt.ColumnID(i))) + }) + } +} + type bestExprSort struct { required PhysicalPropsID display string best *BestExpr } -func (f *memoFormatter) formatBestExprSet(tp treeprinter.Node, mgrp *group) { +func (m *Memo) formatBestExprSet(f *memoFmtCtx, tp treeprinter.Node, mgrp *group) { // Don't show best expressions for scalar groups because they're not too // interesting. if mgrp.isScalarGroup() { @@ -152,14 +161,14 @@ func (f *memoFormatter) formatBestExprSet(tp treeprinter.Node, mgrp *group) { best := mgrp.bestExpr(bestOrdinal(i)) beSort = append(beSort, bestExprSort{ required: best.required, - display: f.mem.LookupPhysicalProps(best.required).String(), + display: m.LookupPhysicalProps(best.required).String(), best: best, }) } sort.Slice(beSort, func(i, j int) bool { // Always order the root required properties first. - if mgrp.id == f.mem.root.group { + if mgrp.id == m.root.group { if beSort[i].required == beSort[i].best.required { return true } @@ -170,13 +179,13 @@ func (f *memoFormatter) formatBestExprSet(tp treeprinter.Node, mgrp *group) { for _, sort := range beSort { f.buf.Reset() child := tp.Childf("\"%s\"", sort.display) - f.formatBestExpr(sort.best) + m.formatBestExpr(f, sort.best) child.Childf("best: %s", f.buf.String()) child.Childf("cost: %.2f", sort.best.cost) } } -func (f *memoFormatter) formatBestExpr(be *BestExpr) { +func (m *Memo) formatBestExpr(f *memoFmtCtx, be *BestExpr) { fmt.Fprintf(f.buf, "(%s", be.op) for i := 0; i < be.ChildCount(); i++ { @@ -184,32 +193,22 @@ func (f *memoFormatter) formatBestExpr(be *BestExpr) { fmt.Fprintf(f.buf, " G%d", f.numbering[bestChild.group]) // Print properties required of the child if they are interesting. - required := f.mem.bestExpr(bestChild).required + required := m.bestExpr(bestChild).required if required != MinPhysPropsID { - fmt.Fprintf(f.buf, "=\"%s\"", f.mem.LookupPhysicalProps(required).String()) + fmt.Fprintf(f.buf, "=\"%s\"", m.LookupPhysicalProps(required).String()) } } - f.formatPrivate(be.Private(f.mem), f.mem.LookupPhysicalProps(be.Required()), formatMemo) + m.formatPrivate(f, be.Private(m), m.LookupPhysicalProps(be.Required())) f.buf.WriteString(")") } -// forEachDependency runs f for each child group of g. -func (f *memoFormatter) forEachDependency(g *group, fn func(GroupID)) { - for i := 0; i < g.exprCount(); i++ { - e := g.expr(ExprOrdinal(i)) - for c := 0; c < e.ChildCount(); c++ { - fn(e.ChildGroup(f.mem, c)) - } - } -} - // sortGroups sorts groups reachable from the root by doing a BFS topological // sort. -func (f *memoFormatter) sortGroups(root GroupID) (groups []GroupID) { - indegrees := f.getIndegrees(root) +func (m *Memo) sortGroups(root GroupID) (groups []GroupID) { + indegrees := m.getIndegrees(root) - res := make([]GroupID, 0, len(f.mem.groups)) + res := make([]GroupID, 0, len(m.groups)) queue := []GroupID{root} for len(queue) > 0 { @@ -221,7 +220,7 @@ func (f *memoFormatter) sortGroups(root GroupID) (groups []GroupID) { // graph, so all of its dependencies have their indegree reduced by one. // Any dependencies which have no more dependents can now be visited and // are added to the queue. - f.forEachDependency(&f.mem.groups[next], func(dep GroupID) { + m.forEachDependency(&m.groups[next], func(dep GroupID) { indegrees[dep]-- if indegrees[dep] == 0 { queue = append(queue, dep) @@ -240,126 +239,34 @@ func (f *memoFormatter) sortGroups(root GroupID) (groups []GroupID) { return res } +// forEachDependency runs fn for each child group of g. +func (m *Memo) forEachDependency(g *group, fn func(GroupID)) { + for i := 0; i < g.exprCount(); i++ { + e := g.expr(ExprOrdinal(i)) + for c := 0; c < e.ChildCount(); c++ { + fn(e.ChildGroup(m, c)) + } + } +} + // getIndegrees returns the indegree of each group reachable from the root. -func (f *memoFormatter) getIndegrees(root GroupID) (indegrees []int) { - indegrees = make([]int, len(f.mem.groups)) - f.computeIndegrees(root, make([]bool, len(f.mem.groups)), indegrees) +func (m *Memo) getIndegrees(root GroupID) (indegrees []int) { + indegrees = make([]int, len(m.groups)) + m.computeIndegrees(root, make([]bool, len(m.groups)), indegrees) return indegrees } // computedIndegrees computes the indegree (number of dependents) of each group // reachable from id. It also populates reachable with true for all reachable // ids. -func (f *memoFormatter) computeIndegrees(id GroupID, reachable []bool, indegrees []int) { +func (m *Memo) computeIndegrees(id GroupID, reachable []bool, indegrees []int) { if id <= 0 || reachable[id] { return } reachable[id] = true - f.forEachDependency(&f.mem.groups[id], func(dep GroupID) { + m.forEachDependency(&m.groups[id], func(dep GroupID) { indegrees[dep]++ - f.computeIndegrees(dep, reachable, indegrees) + m.computeIndegrees(dep, reachable, indegrees) }) } - -func (f exprFormatter) formatPrivate( - private interface{}, physProps *props.Physical, mode formatMode, -) { - if private == nil { - return - } - switch t := private.(type) { - case *ScanOpDef: - // Don't output name of index if it's the primary index. - tab := f.mem.metadata.Table(t.Table) - if t.Index == opt.PrimaryIndex { - fmt.Fprintf(f.buf, " %s", tab.Name().TableName) - } else { - fmt.Fprintf(f.buf, " %s@%s", tab.Name().TableName, tab.Index(t.Index).IdxName()) - } - if _, reverse := t.CanProvideOrdering(f.mem.metadata, &physProps.Ordering); reverse { - f.buf.WriteString(",rev") - } - if mode == formatMemo { - if tab.ColumnCount() != t.Cols.Len() { - fmt.Fprintf(f.buf, ",cols=%s", t.Cols) - } - if t.Constraint != nil { - fmt.Fprintf(f.buf, ",constrained") - } - if t.HardLimit.IsSet() { - fmt.Fprintf(f.buf, ",lim=%s", t.HardLimit) - } - } - - case *VirtualScanOpDef: - tab := f.mem.metadata.Table(t.Table) - fmt.Fprintf(f.buf, " %s", tab.Name()) - - case *RowNumberDef: - if !t.Ordering.Any() { - fmt.Fprintf(f.buf, " ordering=%s", t.Ordering) - } - - case *GroupByDef: - fmt.Fprintf(f.buf, " cols=%s", t.GroupingCols.String()) - if !t.Ordering.Any() { - fmt.Fprintf(f.buf, ",ordering=%s", t.Ordering) - } - - case opt.ColumnID: - fmt.Fprintf(f.buf, " %s", f.mem.metadata.ColumnLabel(t)) - - case *IndexJoinDef: - tab := f.mem.metadata.Table(t.Table) - fmt.Fprintf(f.buf, " %s", tab.Name().TableName) - if mode == formatMemo { - fmt.Fprintf(f.buf, ",cols=%s", t.Cols) - } - - case *LookupJoinDef: - tab := f.mem.metadata.Table(t.Table) - if t.Index == opt.PrimaryIndex { - fmt.Fprintf(f.buf, " %s", tab.Name().TableName) - } else { - fmt.Fprintf(f.buf, " %s@%s", tab.Name().TableName, tab.Index(t.Index).IdxName()) - } - if mode == formatMemo { - fmt.Fprintf(f.buf, ",keyCols=%v,lookupCols=%s", t.KeyCols, t.LookupCols) - } - - case *ExplainOpDef: - if mode == formatMemo { - propsStr := t.Props.String() - if propsStr != "" { - fmt.Fprintf(f.buf, " %s", propsStr) - } - } - - case *MergeOnDef: - fmt.Fprintf(f.buf, " %s,%s,%s", t.JoinType, t.LeftEq, t.RightEq) - - case opt.ColSet, opt.ColList: - // Don't show anything, because it's mostly redundant. - - case *ProjectionsOpDef: - // In normal mode, the information is mostly redundant. It is helpful to - // display these columns in memo mode though. - if mode == formatMemo { - t.PassthroughCols.ForEach(func(i int) { - fmt.Fprintf(f.buf, " %s", f.mem.metadata.ColumnLabel(opt.ColumnID(i))) - }) - } - - case *props.OrderingChoice: - if !t.Any() { - fmt.Fprintf(f.buf, " ordering=%s", t) - } - - case *SetOpColMap, types.T: - // Don't show anything, because it's mostly redundant. - - default: - fmt.Fprintf(f.buf, " %v", private) - } -} diff --git a/pkg/sql/opt/memo/memo_test.go b/pkg/sql/opt/memo/memo_test.go index a0ecf3790146..6745265de96b 100644 --- a/pkg/sql/opt/memo/memo_test.go +++ b/pkg/sql/opt/memo/memo_test.go @@ -17,24 +17,24 @@ package memo_test import ( "testing" - "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat" "github.com/cockroachdb/cockroach/pkg/testutils/datadriven" ) func TestMemo(t *testing.T) { - flags := opt.ExprFmtHideCost | opt.ExprFmtHideRuleProps | opt.ExprFmtHideQualifications + flags := memo.ExprFmtHideCost | memo.ExprFmtHideRuleProps | memo.ExprFmtHideQualifications runDataDrivenTest(t, "testdata/memo", flags) } func TestLogicalProps(t *testing.T) { - flags := opt.ExprFmtHideCost | opt.ExprFmtHideQualifications + flags := memo.ExprFmtHideCost | memo.ExprFmtHideQualifications runDataDrivenTest(t, "testdata/logprops/", flags) } func TestStats(t *testing.T) { - flags := opt.ExprFmtHideCost | opt.ExprFmtHideRuleProps | opt.ExprFmtHideQualifications + flags := memo.ExprFmtHideCost | memo.ExprFmtHideRuleProps | memo.ExprFmtHideQualifications runDataDrivenTest(t, "testdata/stats/", flags) } @@ -45,7 +45,7 @@ func TestStats(t *testing.T) { // // // See OptTester.Handle for supported commands. -func runDataDrivenTest(t *testing.T, path string, fmtFlags opt.ExprFmtFlags) { +func runDataDrivenTest(t *testing.T, path string, fmtFlags memo.ExprFmtFlags) { datadriven.Walk(t, path, func(t *testing.T, path string) { catalog := testcat.New() datadriven.RunTest(t, path, func(d *datadriven.TestData) string { diff --git a/pkg/sql/opt/memo/private_defs_test.go b/pkg/sql/opt/memo/private_defs_test.go index cd1cfc7f8ffd..d310d265426e 100644 --- a/pkg/sql/opt/memo/private_defs_test.go +++ b/pkg/sql/opt/memo/private_defs_test.go @@ -39,7 +39,7 @@ func TestScanCanProvideOrdering(t *testing.T) { t.Fatal(err) } - md := opt.NewMetadata() + var md opt.Metadata a := md.AddTable(cat.Table(tree.NewUnqualifiedTableName("a"))) // PRIMARY KEY (k) @@ -73,7 +73,7 @@ func TestScanCanProvideOrdering(t *testing.T) { for _, tc := range testcases { def := &memo.ScanOpDef{Table: a, Index: tc.index} required := props.ParseOrderingChoice(tc.ordering) - ok, reverse := def.CanProvideOrdering(md, &required) + ok, reverse := def.CanProvideOrdering(&md, &required) res := "no" if ok { if reverse { diff --git a/pkg/sql/opt/memo/statistics_builder_test.go b/pkg/sql/opt/memo/statistics_builder_test.go index 62f191a51461..7e11c70e863b 100644 --- a/pkg/sql/opt/memo/statistics_builder_test.go +++ b/pkg/sql/opt/memo/statistics_builder_test.go @@ -93,7 +93,7 @@ func TestGetStatsFromConstraint(t *testing.T) { mem := New() tab := catalog.Table(tree.NewUnqualifiedTableName("sel")) - tabID := mem.metadata.AddTableWithName(tab, tab.Name().String()) + tabID := mem.metadata.AddTable(tab) // Test that applyConstraintSet correctly updates the statistics from // constraint set cs, and selectivity is calculated correctly. diff --git a/pkg/sql/opt/memo/testdata/logprops/constraints b/pkg/sql/opt/memo/testdata/logprops/constraints index fe379f8813a3..7cad21fd718e 100644 --- a/pkg/sql/opt/memo/testdata/logprops/constraints +++ b/pkg/sql/opt/memo/testdata/logprops/constraints @@ -33,7 +33,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] └── gt [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 1 [type=int] opt @@ -51,7 +51,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: [/1 - ]; tight)] └── ge [type=bool, outer=(1), constraints=(/1: [/1 - ]; tight)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 1 [type=int] opt @@ -69,7 +69,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: (/NULL - /0]; tight)] └── lt [type=bool, outer=(1), constraints=(/1: (/NULL - /0]; tight)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 1 [type=int] opt @@ -87,7 +87,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: (/NULL - /1]; tight)] └── le [type=bool, outer=(1), constraints=(/1: (/NULL - /1]; tight)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 1 [type=int] opt @@ -106,7 +106,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] └── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 1 [type=int] opt @@ -124,10 +124,10 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: [/2 - /4]; tight)] ├── gt [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] └── lt [type=bool, outer=(1), constraints=(/1: (/NULL - /4]; tight)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 5 [type=int] opt @@ -145,10 +145,10 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: [/1 - /1]; /2: [/5 - /5]; tight), fd=()-->(1,2)] ├── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] └── eq [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] - ├── variable: a.y [type=int, outer=(2)] + ├── variable: y [type=int, outer=(2)] └── const: 5 [type=int] opt @@ -165,16 +165,16 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: [/2 - /4]; /2: [/7 - /9]; tight)] ├── gt [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] ├── lt [type=bool, outer=(1), constraints=(/1: (/NULL - /4]; tight)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 5 [type=int] ├── ge [type=bool, outer=(2), constraints=(/2: [/7 - ]; tight)] - │ ├── variable: a.y [type=int, outer=(2)] + │ ├── variable: y [type=int, outer=(2)] │ └── const: 7 [type=int] └── le [type=bool, outer=(2), constraints=(/2: (/NULL - /9]; tight)] - ├── variable: a.y [type=int, outer=(2)] + ├── variable: y [type=int, outer=(2)] └── const: 9 [type=int] # Verify the resulting constraints are not tight. @@ -192,15 +192,15 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: [/2 - /4])] ├── gt [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] ├── lt [type=bool, outer=(1), constraints=(/1: (/NULL - /4]; tight)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 5 [type=int] └── eq [type=bool, outer=(1,2)] ├── plus [type=int, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── const: 5 [type=int] opt @@ -217,17 +217,17 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: [/2 - ])] ├── gt [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] ├── ge [type=bool, outer=(1,2)] │ ├── plus [type=int, outer=(1,2)] - │ │ ├── variable: a.x [type=int, outer=(1)] - │ │ └── variable: a.y [type=int, outer=(2)] + │ │ ├── variable: x [type=int, outer=(1)] + │ │ └── variable: y [type=int, outer=(2)] │ └── const: 5 [type=int] └── le [type=bool, outer=(1,2)] ├── plus [type=int, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── const: 7 [type=int] # Verify that we ignore mixed-type comparisons. @@ -246,7 +246,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1)] └── gt [type=bool, outer=(1)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 1.5 [type=decimal] opt @@ -268,7 +268,7 @@ select │ └── interesting orderings: (+1) └── filters [type=bool, outer=(2)] └── gt [type=bool, outer=(2)] - ├── variable: kuv.u [type=float, outer=(2)] + ├── variable: u [type=float, outer=(2)] └── const: 1 [type=int] opt @@ -290,10 +290,10 @@ select │ └── interesting orderings: (+1) └── filters [type=bool, outer=(3), constraints=(/3: [/'bar' - /'foo']; tight)] ├── le [type=bool, outer=(3), constraints=(/3: (/NULL - /'foo']; tight)] - │ ├── variable: kuv.v [type=string, outer=(3)] + │ ├── variable: v [type=string, outer=(3)] │ └── const: 'foo' [type=string] └── ge [type=bool, outer=(3), constraints=(/3: [/'bar' - ]; tight)] - ├── variable: kuv.v [type=string, outer=(3)] + ├── variable: v [type=string, outer=(3)] └── const: 'bar' [type=string] # Test IN. @@ -312,7 +312,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: [/1 - /1] [/2 - /2] [/3 - /3]; tight)] └── in [type=bool, outer=(1), constraints=(/1: [/1 - /1] [/2 - /2] [/3 - /3]; tight)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── tuple [type=tuple{unknown, int, int, int}] ├── null [type=unknown] ├── const: 1 [type=int] @@ -356,7 +356,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: [/7 - /7] [/9 - /9]; tight)] ├── in [type=bool, outer=(1), constraints=(/1: [/1 - /1] [/3 - /3] [/5 - /5] [/7 - /7] [/9 - /9]; tight)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── tuple [type=tuple{int, int, int, int, int}] │ ├── const: 1 [type=int] │ ├── const: 3 [type=int] @@ -364,7 +364,7 @@ select │ ├── const: 7 [type=int] │ └── const: 9 [type=int] └── gt [type=bool, outer=(1), constraints=(/1: [/7 - ]; tight)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 6 [type=int] # Test IN in combination with a condition on another column. @@ -382,12 +382,12 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: [/1 - /1] [/3 - /3]; /2: [/5 - ]; tight)] ├── in [type=bool, outer=(1), constraints=(/1: [/1 - /1] [/3 - /3]; tight)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── tuple [type=tuple{int, int}] │ ├── const: 1 [type=int] │ └── const: 3 [type=int] └── gt [type=bool, outer=(2), constraints=(/2: [/5 - ]; tight)] - ├── variable: a.y [type=int, outer=(2)] + ├── variable: y [type=int, outer=(2)] └── const: 4 [type=int] # Test tuple inequality. @@ -406,8 +406,8 @@ select └── filters [type=bool, outer=(1,2), constraints=(/1/2: [/1/3 - ]; tight)] └── gt [type=bool, outer=(1,2), constraints=(/1/2: [/1/3 - ]; tight)] ├── tuple [type=tuple{int, int}, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] └── const: 2 [type=int] @@ -427,8 +427,8 @@ select └── filters [type=bool, outer=(1,2), constraints=(/1/2: [/1/2 - ]; tight)] └── ge [type=bool, outer=(1,2), constraints=(/1/2: [/1/2 - ]; tight)] ├── tuple [type=tuple{int, int}, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] └── const: 2 [type=int] @@ -448,8 +448,8 @@ select └── filters [type=bool, outer=(1,2), constraints=(/1/2: (/NULL - /1/1]; tight)] └── lt [type=bool, outer=(1,2), constraints=(/1/2: (/NULL - /1/1]; tight)] ├── tuple [type=tuple{int, int}, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] └── const: 2 [type=int] @@ -469,8 +469,8 @@ select └── filters [type=bool, outer=(1,2), constraints=(/1/2: (/NULL - /1/2]; tight)] └── le [type=bool, outer=(1,2), constraints=(/1/2: (/NULL - /1/2]; tight)] ├── tuple [type=tuple{int, int}, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] └── const: 2 [type=int] @@ -491,8 +491,8 @@ select └── filters [type=bool, outer=(1,2)] └── ge [type=bool, outer=(1,2)] ├── tuple [type=tuple{int, int}, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── tuple [type=tuple{int, decimal}] ├── const: 1 [type=int] └── const: 2.5 [type=decimal] @@ -513,8 +513,8 @@ select └── filters [type=bool, outer=(1,2)] └── ge [type=bool, outer=(1,2)] ├── tuple [type=tuple{int, int}, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── tuple [type=tuple{int, unknown}] ├── const: 1 [type=int] └── null [type=unknown] @@ -537,7 +537,7 @@ select └── filters [type=bool, outer=(1)] └── ge [type=bool, outer=(1)] ├── tuple [type=tuple{int, int}, outer=(1)] - │ ├── variable: a.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] @@ -569,7 +569,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: (/NULL - /4] [/6 - ]; tight)] └── ne [type=bool, outer=(1), constraints=(/1: (/NULL - /4] [/6 - ]; tight)] - ├── variable: abc.a [type=int, outer=(1)] + ├── variable: a [type=int, outer=(1)] └── const: 5 [type=int] opt @@ -587,7 +587,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: [ - /4] [/6 - ]; tight)] └── is-not [type=bool, outer=(1), constraints=(/1: [ - /4] [/6 - ]; tight)] - ├── variable: abc.a [type=int, outer=(1)] + ├── variable: a [type=int, outer=(1)] └── const: 5 [type=int] opt @@ -605,7 +605,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - /false]; tight)] └── ne [type=bool, outer=(2), constraints=(/2: (/NULL - /false]; tight)] - ├── variable: abc.b [type=bool, outer=(2)] + ├── variable: b [type=bool, outer=(2)] └── true [type=bool] opt @@ -623,7 +623,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(2), constraints=(/2: [/true - ]; tight)] └── ne [type=bool, outer=(2), constraints=(/2: [/true - ]; tight)] - ├── variable: abc.b [type=bool, outer=(2)] + ├── variable: b [type=bool, outer=(2)] └── false [type=bool] opt @@ -641,7 +641,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(2), constraints=(/2: [ - /false]; tight)] └── is-not [type=bool, outer=(2), constraints=(/2: [ - /false]; tight)] - ├── variable: abc.b [type=bool, outer=(2)] + ├── variable: b [type=bool, outer=(2)] └── true [type=bool] opt @@ -659,7 +659,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(2), constraints=(/2: [ - /false) [/true - ]; tight)] └── is-not [type=bool, outer=(2), constraints=(/2: [ - /false) [/true - ]; tight)] - ├── variable: abc.b [type=bool, outer=(2)] + ├── variable: b [type=bool, outer=(2)] └── false [type=bool] opt @@ -677,7 +677,7 @@ select │ ├── prune: (1-3) │ └── interesting orderings: () └── filters [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] - └── variable: abc.b [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight)] + └── variable: b [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight)] opt SELECT * FROM abc WHERE NOT b @@ -695,7 +695,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(2), constraints=(/2: [/false - /false]; tight), fd=()-->(2)] └── not [type=bool, outer=(2), constraints=(/2: [/false - /false]; tight)] - └── variable: abc.b [type=bool, outer=(2)] + └── variable: b [type=bool, outer=(2)] opt SELECT * FROM abc WHERE a > 5 AND b @@ -713,9 +713,9 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: [/6 - ]; /2: [/true - /true]; tight), fd=()-->(2)] ├── gt [type=bool, outer=(1), constraints=(/1: [/6 - ]; tight)] - │ ├── variable: abc.a [type=int, outer=(1)] + │ ├── variable: a [type=int, outer=(1)] │ └── const: 5 [type=int] - └── variable: abc.b [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight)] + └── variable: b [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight)] opt SELECT * FROM abc WHERE c != 'foo' @@ -732,7 +732,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(3), constraints=(/3: (/NULL - /'foo') [/e'foo\x00' - ]; tight)] └── ne [type=bool, outer=(3), constraints=(/3: (/NULL - /'foo') [/e'foo\x00' - ]; tight)] - ├── variable: abc.c [type=string, outer=(3)] + ├── variable: c [type=string, outer=(3)] └── const: 'foo' [type=string] opt @@ -750,7 +750,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(3), constraints=(/3: [ - /'foo') [/e'foo\x00' - ]; tight)] └── is-not [type=bool, outer=(3), constraints=(/3: [ - /'foo') [/e'foo\x00' - ]; tight)] - ├── variable: abc.c [type=string, outer=(3)] + ├── variable: c [type=string, outer=(3)] └── const: 'foo' [type=string] opt @@ -770,8 +770,8 @@ select │ │ └── interesting orderings: () │ └── projections [outer=(1,2)] │ └── tuple [type=tuple{int, int}, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── filters [type=bool, outer=(4), constraints=(/4: [/(1, 3) - ]; tight)] └── gt [type=bool, outer=(4), constraints=(/4: [/(1, 3) - ]; tight)] ├── variable: col [type=tuple{int, int}, outer=(4)] @@ -811,7 +811,7 @@ scan c@v ├── prune: (1) └── interesting orderings: (+1) (+3,+2,+1) -opt format=show-all +opt format=hide-qual SELECT * FROM c WHERE (v, u) IN ((1, 2), (1, 3), (1, 4)) ---- scan c@v @@ -824,7 +824,7 @@ scan c@v ├── prune: (1) └── interesting orderings: (+1) (+3,+2,+1) -opt format=show-all +opt format=hide-qual SELECT * FROM c WHERE (v, u) IN ((1, 2), (3, 2), (5, 2)) ---- scan c@v @@ -837,7 +837,7 @@ scan c@v ├── prune: (1) └── interesting orderings: (+1) (+3,+2,+1) -opt format=show-all +opt format=hide-qual SELECT * FROM c WHERE (v, u) IN ((1, 2), (1, 2), (1, 2)) ---- scan c@v @@ -850,7 +850,7 @@ scan c@v ├── prune: (1) └── interesting orderings: (+1) (+3,+2,+1) -opt format=show-all +opt format=hide-qual SELECT * FROM c WHERE (v, u) IN ((1, 2), (1, 3), (1, 4)) ---- scan c@v @@ -863,7 +863,7 @@ scan c@v ├── prune: (1) └── interesting orderings: (+1) (+3,+2,+1) -opt format=show-all +opt format=hide-qual SELECT * FROM c WHERE (v, u) IN ((1, 2), (3, 2), (5, 2)) ---- scan c@v @@ -876,7 +876,7 @@ scan c@v ├── prune: (1) └── interesting orderings: (+1) (+3,+2,+1) -opt format=show-all +opt format=hide-qual SELECT * FROM c WHERE (v, u) IN ((1, 2), (1, 2), (1, 2)) ---- scan c@v @@ -925,7 +925,7 @@ select └── filters [type=bool, outer=(3), constraints=(/3: [/1 - /1] [/3 - /3] [/5 - /5])] └── in [type=bool, outer=(3), constraints=(/3: [/1 - /1] [/3 - /3] [/5 - /5])] ├── tuple [type=tuple{int, int}, outer=(3)] - │ ├── variable: c.v [type=int, outer=(3)] + │ ├── variable: v [type=int, outer=(3)] │ └── const: 2 [type=int] └── tuple [type=tuple{tuple{int, int}, tuple{int, int}, tuple{int, int}}] ├── tuple [type=tuple{int, int}] @@ -962,10 +962,10 @@ select └── filters [type=bool, outer=(2,3), constraints=(/3: [/1 - /1] [/3 - /3] [/5 - /5])] └── in [type=bool, outer=(2,3), constraints=(/3: [/1 - /1] [/3 - /3] [/5 - /5])] ├── tuple [type=tuple{int, int}, outer=(2,3)] - │ ├── variable: c.v [type=int, outer=(3)] + │ ├── variable: v [type=int, outer=(3)] │ └── plus [type=int, outer=(2,3)] - │ ├── variable: c.u [type=int, outer=(2)] - │ └── variable: c.v [type=int, outer=(3)] + │ ├── variable: u [type=int, outer=(2)] + │ └── variable: v [type=int, outer=(3)] └── tuple [type=tuple{tuple{int, int}, tuple{int, int}, tuple{int, int}}] ├── tuple [type=tuple{int, int}] │ ├── const: 1 [type=int] @@ -996,14 +996,14 @@ select └── filters [type=bool, outer=(1-3), constraints=(/2: [/2 - /2] [/50 - /50] [/100 - /100])] └── in [type=bool, outer=(1-3), constraints=(/2: [/2 - /2] [/50 - /50] [/100 - /100])] ├── tuple [type=tuple{int, int}, outer=(2,3)] - │ ├── variable: c.v [type=int, outer=(3)] - │ └── variable: c.u [type=int, outer=(2)] + │ ├── variable: v [type=int, outer=(3)] + │ └── variable: u [type=int, outer=(2)] └── tuple [type=tuple{tuple{int, int}, tuple{int, int}, tuple{int, int}}, outer=(1)] ├── tuple [type=tuple{int, int}] │ ├── const: 1 [type=int] │ └── const: 2 [type=int] ├── tuple [type=tuple{int, int}, outer=(1)] - │ ├── variable: c.k [type=int, outer=(1)] + │ ├── variable: k [type=int, outer=(1)] │ └── const: 50 [type=int] └── tuple [type=tuple{int, int}] ├── const: 5 [type=int] @@ -1024,7 +1024,7 @@ TABLE d └── INDEX primary └── k int not null -opt format=show-all +opt format=hide-qual SELECT * FROM d WHERE (p, q) IN ((1, 2), (1, 3), (1, 4)) ---- select @@ -1036,7 +1036,7 @@ select ├── prune: (1) ├── interesting orderings: (+1) ├── scan d - │ ├── columns: d.k:1(int!null) d.p:2(int) d.q:3(int) + │ ├── columns: k:1(int!null) p:2(int) q:3(int) │ ├── stats: [rows=1000, distinct(2)=700, distinct(3)=700] │ ├── cost: 1060 │ ├── key: (1) @@ -1046,8 +1046,8 @@ select └── filters [type=bool, outer=(2,3), constraints=(/2/3: [/1/2 - /1/2] [/1/3 - /1/3] [/1/4 - /1/4]; /3: [/2 - /2] [/3 - /3] [/4 - /4]; tight), fd=()-->(2)] └── in [type=bool, outer=(2,3), constraints=(/2/3: [/1/2 - /1/2] [/1/3 - /1/3] [/1/4 - /1/4]; /3: [/2 - /2] [/3 - /3] [/4 - /4]; tight)] ├── tuple [type=tuple{int, int}, outer=(2,3)] - │ ├── variable: d.p [type=int, outer=(2)] - │ └── variable: d.q [type=int, outer=(3)] + │ ├── variable: p [type=int, outer=(2)] + │ └── variable: q [type=int, outer=(3)] └── tuple [type=tuple{tuple{int, int}, tuple{int, int}, tuple{int, int}}] ├── tuple [type=tuple{int, int}] │ ├── const: 1 [type=int] @@ -1059,7 +1059,7 @@ select ├── const: 1 [type=int] └── const: 4 [type=int] -opt format=show-all +opt format=hide-qual SELECT * FROM d WHERE (p, q) IN ((2, 1), (3, 1), (4, 1)) ---- select @@ -1071,7 +1071,7 @@ select ├── prune: (1) ├── interesting orderings: (+1) ├── scan d - │ ├── columns: d.k:1(int!null) d.p:2(int) d.q:3(int) + │ ├── columns: k:1(int!null) p:2(int) q:3(int) │ ├── stats: [rows=1000, distinct(2)=700, distinct(3)=700] │ ├── cost: 1060 │ ├── key: (1) @@ -1081,8 +1081,8 @@ select └── filters [type=bool, outer=(2,3), constraints=(/2/3: [/2/1 - /2/1] [/3/1 - /3/1] [/4/1 - /4/1]; /3: [/1 - /1]; tight), fd=()-->(3)] └── in [type=bool, outer=(2,3), constraints=(/2/3: [/2/1 - /2/1] [/3/1 - /3/1] [/4/1 - /4/1]; /3: [/1 - /1]; tight)] ├── tuple [type=tuple{int, int}, outer=(2,3)] - │ ├── variable: d.p [type=int, outer=(2)] - │ └── variable: d.q [type=int, outer=(3)] + │ ├── variable: p [type=int, outer=(2)] + │ └── variable: q [type=int, outer=(3)] └── tuple [type=tuple{tuple{int, int}, tuple{int, int}, tuple{int, int}}] ├── tuple [type=tuple{int, int}] │ ├── const: 2 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/constraints-null b/pkg/sql/opt/memo/testdata/logprops/constraints-null index 56d77d04cb5e..66797f587091 100644 --- a/pkg/sql/opt/memo/testdata/logprops/constraints-null +++ b/pkg/sql/opt/memo/testdata/logprops/constraints-null @@ -47,7 +47,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: [/NULL - /NULL]; tight), fd=()-->(1)] └── is [type=bool, outer=(1), constraints=(/1: [/NULL - /NULL]; tight)] - ├── variable: t.a [type=int, outer=(1)] + ├── variable: a [type=int, outer=(1)] └── null [type=unknown] opt @@ -65,7 +65,7 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] - ├── variable: t.a [type=int, outer=(1)] + ├── variable: a [type=int, outer=(1)] └── null [type=unknown] opt @@ -84,10 +84,10 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(2,3), constraints=(/2: [/NULL - /NULL]; /3: [/NULL - /NULL]; tight), fd=()-->(2,3)] ├── is [type=bool, outer=(2), constraints=(/2: [/NULL - /NULL]; tight)] - │ ├── variable: t.b [type=bool, outer=(2)] + │ ├── variable: b [type=bool, outer=(2)] │ └── null [type=unknown] └── is [type=bool, outer=(3), constraints=(/3: [/NULL - /NULL]; tight)] - ├── variable: t.c [type=string, outer=(3)] + ├── variable: c [type=string, outer=(3)] └── null [type=unknown] opt @@ -105,10 +105,10 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]; tight)] ├── is-not [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] - │ ├── variable: t.b [type=bool, outer=(2)] + │ ├── variable: b [type=bool, outer=(2)] │ └── null [type=unknown] └── is-not [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] - ├── variable: t.c [type=string, outer=(3)] + ├── variable: c [type=string, outer=(3)] └── null [type=unknown] exec-ddl @@ -139,9 +139,9 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: (/NULL - ])] └── gt [type=bool, outer=(1,2), constraints=(/1: (/NULL - ])] - ├── variable: xy.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── function: abs [type=int, outer=(2)] - └── variable: xy.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # Test that we get a not-NULL constraint on x. opt @@ -159,11 +159,11 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1), constraints=(/1: (/NULL - ])] └── gt [type=bool, outer=(1), constraints=(/1: (/NULL - ])] - ├── variable: xy.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── cast: INT [type=int, outer=(1)] └── function: sin [type=float, outer=(1)] └── cast: FLOAT [type=float, outer=(1)] - └── variable: xy.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] # Test that we get a not-NULL constraint on x and y. opt @@ -180,8 +180,8 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] └── gt [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - ├── variable: xy.x [type=int, outer=(1)] - └── variable: xy.y [type=int, outer=(2)] + ├── variable: x [type=int, outer=(1)] + └── variable: y [type=int, outer=(2)] # Test that we get a not-NULL constraint on x and y. opt @@ -199,5 +199,5 @@ select │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] └── eq [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - ├── variable: xy.x [type=int, outer=(1)] - └── variable: xy.y [type=int, outer=(2)] + ├── variable: x [type=int, outer=(1)] + └── variable: y [type=int, outer=(2)] diff --git a/pkg/sql/opt/memo/testdata/logprops/groupby b/pkg/sql/opt/memo/testdata/logprops/groupby index 66de2176c3ae..e620ea988516 100644 --- a/pkg/sql/opt/memo/testdata/logprops/groupby +++ b/pkg/sql/opt/memo/testdata/logprops/groupby @@ -57,7 +57,7 @@ project │ │ └── interesting orderings: (+1) (-4,+3,+1) │ └── aggregations [outer=(3)] │ └── sum [type=float, outer=(3)] - │ └── variable: xyzs.z [type=float, outer=(3)] + │ └── variable: z [type=float, outer=(3)] └── projections [outer=(1,2,5)] └── false [type=bool] @@ -88,9 +88,9 @@ scalar-group-by │ └── interesting orderings: (+1) (-4,+3,+1) └── aggregations [outer=(1,2)] ├── sum [type=decimal, outer=(1)] - │ └── variable: xyzs.x [type=int, outer=(1)] + │ └── variable: x [type=int, outer=(1)] └── max [type=int, outer=(2)] - └── variable: xyzs.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # Group by unique index columns. build @@ -148,7 +148,7 @@ project │ └── interesting orderings: (+1) (-4,+3,+1) └── aggregations [outer=(3)] └── sum [type=float, outer=(3)] - └── variable: xyzs.z [type=float, outer=(3)] + └── variable: z [type=float, outer=(3)] # Group by column that is subset of unique index. build @@ -175,7 +175,7 @@ group-by │ └── interesting orderings: (+1) (-4,+3,+1) └── aggregations [outer=(4)] └── max [type=string, outer=(4)] - └── variable: xyzs.s [type=string, outer=(4)] + └── variable: s [type=string, outer=(4)] # Group by all columns. build @@ -316,6 +316,6 @@ group-by │ └── interesting orderings: (+1) └── aggregations [outer=(1,3)] ├── count [type=int, outer=(3)] - │ └── variable: xyzs.z [type=float, outer=(3)] + │ └── variable: z [type=float, outer=(3)] └── const-agg [type=int, outer=(1)] - └── variable: xyzs.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] diff --git a/pkg/sql/opt/memo/testdata/logprops/index-join b/pkg/sql/opt/memo/testdata/logprops/index-join index 28145d23ded5..3b13976ed2da 100644 --- a/pkg/sql/opt/memo/testdata/logprops/index-join +++ b/pkg/sql/opt/memo/testdata/logprops/index-join @@ -76,8 +76,8 @@ select └── filters [type=bool, outer=(1,2)] └── eq [type=bool, outer=(1,2)] ├── plus [type=int, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── const: 10 [type=int] opt @@ -110,8 +110,8 @@ project └── filters [type=bool, outer=(1,2)] └── eq [type=bool, outer=(1,2)] ├── plus [type=int, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── const: 10 [type=int] # Use secondary index to join to multi-valued primary index, but project only @@ -140,5 +140,5 @@ select │ └── interesting orderings: (+1,+2) (+3,+1,+2) └── filters [type=bool, outer=(4), constraints=(/4: [/2 - /2]; tight), fd=()-->(4)] └── eq [type=bool, outer=(4), constraints=(/4: [/2 - /2]; tight)] - ├── variable: abc.d [type=int, outer=(4)] + ├── variable: d [type=int, outer=(4)] └── const: 2 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/join b/pkg/sql/opt/memo/testdata/logprops/join index af48b249a710..76ea0ac68641 100644 --- a/pkg/sql/opt/memo/testdata/logprops/join +++ b/pkg/sql/opt/memo/testdata/logprops/join @@ -62,8 +62,8 @@ inner-join │ └── interesting orderings: (+7) └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── variable: xysd.x [type=int, outer=(1)] - └── variable: uv.u [type=int, outer=(5)] + ├── variable: x [type=int, outer=(1)] + └── variable: u [type=int, outer=(5)] # Inner-join-apply. opt @@ -100,9 +100,9 @@ project │ │ ├── stats: [rows=2] │ │ ├── prune: (5) │ │ ├── tuple [type=tuple{int}, outer=(1)] - │ │ │ └── variable: xysd.x [type=int, outer=(1)] + │ │ │ └── variable: x [type=int, outer=(1)] │ │ └── tuple [type=tuple{int}, outer=(2)] - │ │ └── variable: xysd.y [type=int, outer=(2)] + │ │ └── variable: y [type=int, outer=(2)] │ └── true [type=bool] └── projections [outer=(5)] └── variable: column1 [type=int, outer=(5)] @@ -178,16 +178,16 @@ project │ │ │ └── interesting orderings: (+9) │ │ └── filters [type=bool, outer=(6,9), constraints=(/6: (/NULL - ]; /9: (/NULL - ]), fd=(6)==(9), (9)==(6)] │ │ └── eq [type=bool, outer=(6,9), constraints=(/6: (/NULL - ]; /9: (/NULL - ])] - │ │ ├── variable: mn.n [type=int, outer=(9)] - │ │ └── variable: uv.v [type=int, outer=(6)] + │ │ ├── variable: n [type=int, outer=(9)] + │ │ └── variable: v [type=int, outer=(6)] │ └── filters [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] │ └── eq [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] - │ ├── variable: xysd.x [type=int, outer=(1)] - │ └── variable: mn.n [type=int, outer=(9)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: n [type=int, outer=(9)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - ├── variable: xysd.x [type=int, outer=(1)] - └── variable: uv.v [type=int, outer=(6)] + ├── variable: x [type=int, outer=(1)] + └── variable: v [type=int, outer=(6)] # Inner-join nested in inner-join-apply with outer column reference to top-level # inner-join-apply. @@ -247,15 +247,15 @@ project │ │ └── interesting orderings: (+8) │ └── filters [type=bool, outer=(1,2,8), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(1,8), (8)==(1,2), (1)==(2,8)] │ ├── eq [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] - │ │ ├── variable: mn.m [type=int, outer=(8)] - │ │ └── variable: xysd.y [type=int, outer=(2)] + │ │ ├── variable: m [type=int, outer=(8)] + │ │ └── variable: y [type=int, outer=(2)] │ └── eq [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] - │ ├── variable: xysd.x [type=int, outer=(1)] - │ └── variable: mn.m [type=int, outer=(8)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: m [type=int, outer=(8)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - ├── variable: xysd.x [type=int, outer=(1)] - └── variable: uv.v [type=int, outer=(6)] + ├── variable: x [type=int, outer=(1)] + └── variable: v [type=int, outer=(6)] # Left-join. build @@ -285,8 +285,8 @@ left-join │ └── interesting orderings: (+7) └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── variable: xysd.x [type=int, outer=(1)] - └── variable: uv.u [type=int, outer=(5)] + ├── variable: x [type=int, outer=(1)] + └── variable: u [type=int, outer=(5)] # Left-join-apply. opt @@ -341,12 +341,12 @@ project │ │ │ └── interesting orderings: () │ │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ │ └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - │ │ ├── variable: uv.u [type=int, outer=(5)] - │ │ └── variable: xysd.x [type=int, outer=(1)] + │ │ ├── variable: u [type=int, outer=(5)] + │ │ └── variable: x [type=int, outer=(1)] │ └── true [type=bool] └── filters [type=bool, outer=(5), constraints=(/5: [/NULL - /NULL]; tight), fd=()-->(5)] └── is [type=bool, outer=(5), constraints=(/5: [/NULL - /NULL]; tight)] - ├── variable: uv.u [type=int, outer=(5)] + ├── variable: u [type=int, outer=(5)] └── null [type=unknown] # Right-join. @@ -377,8 +377,8 @@ right-join │ └── interesting orderings: (+7) └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── variable: xysd.x [type=int, outer=(1)] - └── variable: uv.u [type=int, outer=(5)] + ├── variable: x [type=int, outer=(1)] + └── variable: u [type=int, outer=(5)] # Right-join-apply. opt @@ -447,7 +447,7 @@ right-join │ │ │ │ └── filters [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ │ │ │ └── eq [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] │ │ │ │ ├── variable: uv.u [type=int, outer=(8)] - │ │ │ │ └── variable: xysd.x [type=int, outer=(1)] + │ │ │ │ └── variable: x [type=int, outer=(1)] │ │ │ └── const: 1 [type=int] │ │ └── true [type=bool] │ └── filters [type=bool, outer=(8), constraints=(/8: [/NULL - /NULL]; tight), fd=()-->(8)] @@ -489,8 +489,8 @@ full-join │ └── interesting orderings: (+7) └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── variable: xysd.x [type=int, outer=(1)] - └── variable: uv.u [type=int, outer=(5)] + ├── variable: x [type=int, outer=(1)] + └── variable: u [type=int, outer=(5)] # Full-join-apply. opt @@ -556,7 +556,7 @@ project │ │ │ └── filters [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ │ │ └── eq [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] │ │ │ ├── variable: uv.u [type=int, outer=(8)] - │ │ │ └── variable: xysd.x [type=int, outer=(1)] + │ │ │ └── variable: x [type=int, outer=(1)] │ │ └── const: 1 [type=int] │ └── true [type=bool] └── filters [type=bool, outer=(8), constraints=(/8: [/NULL - /NULL]; tight), fd=()-->(8)] @@ -589,8 +589,8 @@ semi-join │ └── interesting orderings: () └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── variable: xysd.x [type=int, outer=(1)] - └── variable: uv.u [type=int, outer=(5)] + ├── variable: x [type=int, outer=(1)] + └── variable: u [type=int, outer=(5)] # Semi-join-apply. opt @@ -631,8 +631,8 @@ semi-join-apply │ │ │ └── interesting orderings: () │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] │ │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - │ │ ├── variable: uv.v [type=int, outer=(6)] - │ │ └── variable: xysd.x [type=int, outer=(1)] + │ │ ├── variable: v [type=int, outer=(6)] + │ │ └── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] └── true [type=bool] @@ -675,11 +675,11 @@ semi-join-apply │ │ └── interesting orderings: (+8) (+9,+8) │ └── filters [type=bool, outer=(1,6,8), constraints=(/1: (/NULL - ]; /6: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(6,8), (8)==(1,6), (6)==(1,8)] │ ├── eq [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] - │ │ ├── variable: xysd.x [type=int, outer=(1)] - │ │ └── variable: mn.m [type=int, outer=(8)] + │ │ ├── variable: x [type=int, outer=(1)] + │ │ └── variable: m [type=int, outer=(8)] │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - │ ├── variable: xysd.x [type=int, outer=(1)] - │ └── variable: uv.v [type=int, outer=(6)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: v [type=int, outer=(6)] └── true [type=bool] # Anti-join. @@ -707,8 +707,8 @@ anti-join │ └── interesting orderings: () └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── variable: xysd.x [type=int, outer=(1)] - └── variable: uv.u [type=int, outer=(5)] + ├── variable: x [type=int, outer=(1)] + └── variable: u [type=int, outer=(5)] # Anti-join-apply. opt @@ -749,8 +749,8 @@ anti-join-apply │ │ │ └── interesting orderings: () │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] │ │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - │ │ ├── variable: uv.v [type=int, outer=(6)] - │ │ └── variable: xysd.x [type=int, outer=(1)] + │ │ ├── variable: v [type=int, outer=(6)] + │ │ └── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] └── true [type=bool] @@ -876,7 +876,7 @@ select └── eq [type=bool, outer=(3,5)] ├── cast: STRING [type=string, outer=(5)] │ └── variable: x [type=int, outer=(5)] - └── variable: xysd.s [type=string, outer=(3)] + └── variable: s [type=string, outer=(3)] # Calculate semi-join cardinality when left side has non-zero cardinality. opt @@ -959,8 +959,8 @@ semi-join-apply │ │ │ └── interesting orderings: () │ │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ │ └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - │ │ ├── variable: xysd.x [type=int, outer=(1)] - │ │ └── variable: uv.u [type=int, outer=(5)] + │ │ ├── variable: x [type=int, outer=(1)] + │ │ └── variable: u [type=int, outer=(5)] │ └── const: 5 [type=int] └── true [type=bool] @@ -990,7 +990,7 @@ anti-join │ └── interesting orderings: () └── filters [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] └── eq [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - ├── variable: uv.u [type=int, outer=(2)] + ├── variable: u [type=int, outer=(2)] └── variable: column1 [type=int, outer=(1)] # Calculate anti-join-apply cardinality. @@ -1035,8 +1035,8 @@ anti-join-apply │ │ │ └── interesting orderings: () │ │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ │ └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - │ │ ├── variable: xysd.x [type=int, outer=(1)] - │ │ └── variable: uv.u [type=int, outer=(5)] + │ │ ├── variable: x [type=int, outer=(1)] + │ │ └── variable: u [type=int, outer=(5)] │ └── const: 5 [type=int] └── true [type=bool] @@ -1340,10 +1340,10 @@ left-join │ │ └── interesting orderings: (+7) │ └── aggregations [outer=(6)] │ └── sum [type=decimal, outer=(6)] - │ └── variable: uv.v [type=int, outer=(6)] + │ └── variable: v [type=int, outer=(6)] └── filters [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] └── is-not [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] - ├── variable: uv.u [type=int, outer=(5)] + ├── variable: u [type=int, outer=(5)] └── null [type=unknown] # Not-null FD determinant on right side of left-join stays strict. @@ -1392,11 +1392,11 @@ left-join │ │ │ └── interesting orderings: (+7) │ │ └── filters [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] │ │ └── is-not [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] - │ │ ├── variable: uv.u [type=int, outer=(5)] + │ │ ├── variable: u [type=int, outer=(5)] │ │ └── null [type=unknown] │ └── aggregations [outer=(6)] │ └── sum [type=decimal, outer=(6)] - │ └── variable: uv.v [type=int, outer=(6)] + │ └── variable: v [type=int, outer=(6)] └── filters [type=bool] └── true [type=bool] @@ -1432,7 +1432,7 @@ right-join │ │ └── interesting orderings: (+3) │ └── aggregations [outer=(2)] │ └── sum [type=decimal, outer=(2)] - │ └── variable: uv.v [type=int, outer=(2)] + │ └── variable: v [type=int, outer=(2)] ├── scan xysd │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) │ ├── stats: [rows=1000] @@ -1442,7 +1442,7 @@ right-join │ └── interesting orderings: (+5) (-7,+8,+5) └── filters [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] - ├── variable: uv.u [type=int, outer=(1)] + ├── variable: u [type=int, outer=(1)] └── null [type=unknown] # Not-null FD determinant on left side of right-join stays strict. @@ -1484,11 +1484,11 @@ right-join │ │ │ └── interesting orderings: (+3) │ │ └── filters [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] │ │ └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] - │ │ ├── variable: uv.u [type=int, outer=(1)] + │ │ ├── variable: u [type=int, outer=(1)] │ │ └── null [type=unknown] │ └── aggregations [outer=(2)] │ └── sum [type=decimal, outer=(2)] - │ └── variable: uv.v [type=int, outer=(2)] + │ └── variable: v [type=int, outer=(2)] ├── scan xysd │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) │ ├── stats: [rows=1000] @@ -1538,10 +1538,10 @@ full-join │ │ └── interesting orderings: (+7) │ └── aggregations [outer=(6)] │ └── sum [type=decimal, outer=(6)] - │ └── variable: uv.v [type=int, outer=(6)] + │ └── variable: v [type=int, outer=(6)] └── filters [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] └── is-not [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] - ├── variable: uv.u [type=int, outer=(5)] + ├── variable: u [type=int, outer=(5)] └── null [type=unknown] # Nullable FD determinant on left side of full-join becomes lax. @@ -1576,7 +1576,7 @@ full-join │ │ └── interesting orderings: (+3) │ └── aggregations [outer=(2)] │ └── sum [type=decimal, outer=(2)] - │ └── variable: uv.v [type=int, outer=(2)] + │ └── variable: v [type=int, outer=(2)] ├── scan xysd │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) │ ├── stats: [rows=1000] @@ -1586,5 +1586,5 @@ full-join │ └── interesting orderings: (+5) (-7,+8,+5) └── filters [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] - ├── variable: uv.u [type=int, outer=(1)] + ├── variable: u [type=int, outer=(1)] └── null [type=unknown] diff --git a/pkg/sql/opt/memo/testdata/logprops/limit b/pkg/sql/opt/memo/testdata/logprops/limit index 21ae877d22eb..13762bc25d31 100644 --- a/pkg/sql/opt/memo/testdata/logprops/limit +++ b/pkg/sql/opt/memo/testdata/logprops/limit @@ -177,7 +177,7 @@ project │ │ └── interesting orderings: (+5) │ └── projections [outer=(1)] │ └── variable: xyzs.x [type=int, outer=(1)] - └── variable: xyzs.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # Test very high limit (> max uint32). opt diff --git a/pkg/sql/opt/memo/testdata/logprops/offset b/pkg/sql/opt/memo/testdata/logprops/offset index dbc179c552c4..f0f947306c04 100644 --- a/pkg/sql/opt/memo/testdata/logprops/offset +++ b/pkg/sql/opt/memo/testdata/logprops/offset @@ -145,7 +145,7 @@ project │ │ └── interesting orderings: (+5) │ └── projections [outer=(1)] │ └── variable: xyzs.x [type=int, outer=(1)] - └── variable: xyzs.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # Reduce cardinality of input set. build @@ -227,6 +227,6 @@ offset │ │ └── interesting orderings: (+1) (-4) │ └── filters [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] │ └── eq [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] - │ ├── variable: xyzs.s [type=string, outer=(4)] + │ ├── variable: s [type=string, outer=(4)] │ └── const: 'foo' [type=string] └── const: 4294967296 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/project b/pkg/sql/opt/memo/testdata/logprops/project index a218ffb8d16f..edf7dc247252 100644 --- a/pkg/sql/opt/memo/testdata/logprops/project +++ b/pkg/sql/opt/memo/testdata/logprops/project @@ -42,7 +42,7 @@ project │ └── interesting orderings: (+1) (-3,+4,+1) └── projections [outer=(1,2)] ├── plus [type=int, outer=(1)] - │ ├── variable: xysd.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] └── const: 1 [type=int] @@ -116,8 +116,8 @@ select │ │ │ └── interesting orderings: (+5) │ │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ │ └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - │ │ ├── variable: kuv.k [type=int, outer=(5)] - │ │ └── variable: xysd.x [type=int, outer=(1)] + │ │ ├── variable: k [type=int, outer=(5)] + │ │ └── variable: x [type=int, outer=(1)] │ └── projections [outer=(2)] │ └── subquery [type=int, outer=(2)] │ └── max1-row @@ -265,8 +265,8 @@ project │ └── interesting orderings: (+4) (-6,+7,+4) └── filters [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] - ├── variable: xysd.x [type=int, outer=(4)] - └── variable: kuv.k [type=int, outer=(1)] + ├── variable: x [type=int, outer=(4)] + └── variable: k [type=int, outer=(1)] # Project nested correlated subquery. build @@ -330,7 +330,7 @@ project └── filters [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] └── eq [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] ├── variable: xysd.x [type=int, outer=(8)] - └── variable: kuv.k [type=int, outer=(1)] + └── variable: k [type=int, outer=(1)] # We have the FD: y --> y::TEXT. build diff --git a/pkg/sql/opt/memo/testdata/logprops/scalar b/pkg/sql/opt/memo/testdata/logprops/scalar index 6269ecabb972..edf0dc90112b 100644 --- a/pkg/sql/opt/memo/testdata/logprops/scalar +++ b/pkg/sql/opt/memo/testdata/logprops/scalar @@ -36,7 +36,7 @@ select │ └── interesting orderings: (+1) └── filters [type=bool, outer=(1), constraints=(/1: (/NULL - /4]; tight)] └── lt [type=bool, outer=(1), constraints=(/1: (/NULL - /4]; tight)] - ├── variable: xy.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 5 [type=int] build @@ -71,15 +71,15 @@ project └── projections [outer=(1,2,5)] ├── eq [type=bool, outer=(1,2)] │ ├── plus [type=int, outer=(1)] - │ │ ├── variable: xy.x [type=int, outer=(1)] + │ │ ├── variable: x [type=int, outer=(1)] │ │ └── const: 1 [type=int] │ └── plus [type=int, outer=(2)] │ ├── function: length [type=int] │ │ └── const: 'foo' [type=string] - │ └── variable: xy.y [type=int, outer=(2)] + │ └── variable: y [type=int, outer=(2)] └── mult [type=int, outer=(1,5)] - ├── variable: uv.rowid [type=int, outer=(5)] - └── variable: xy.x [type=int, outer=(1)] + ├── variable: rowid [type=int, outer=(5)] + └── variable: x [type=int, outer=(1)] build SELECT * FROM xy WHERE EXISTS(SELECT * FROM uv WHERE u=x) @@ -123,8 +123,8 @@ select │ └── interesting orderings: (+5) └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] - ├── variable: uv.u [type=int, outer=(3)] - └── variable: xy.x [type=int, outer=(1)] + ├── variable: u [type=int, outer=(3)] + └── variable: x [type=int, outer=(1)] build SELECT * FROM xy WHERE y IN (SELECT v FROM uv WHERE u=x) @@ -166,9 +166,9 @@ select │ │ └── interesting orderings: (+5) │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] │ └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] - │ ├── variable: uv.u [type=int, outer=(3)] - │ └── variable: xy.x [type=int, outer=(1)] - └── variable: xy.y [type=int, outer=(2)] + │ ├── variable: u [type=int, outer=(3)] + │ └── variable: x [type=int, outer=(1)] + └── variable: y [type=int, outer=(2)] # Side-effects: test DivOp and impure FuncOp. build @@ -217,8 +217,8 @@ group-by │ │ │ └── interesting orderings: (+1) │ │ └── projections [outer=(1,2), side-effects] │ │ └── div [type=decimal, outer=(1,2), side-effects] - │ │ ├── variable: xy.x [type=int, outer=(1)] - │ │ └── variable: xy.y [type=int, outer=(2)] + │ │ ├── variable: x [type=int, outer=(1)] + │ │ └── variable: y [type=int, outer=(2)] │ ├── project │ │ ├── columns: u:4(int) v:5(int!null) │ │ ├── side-effects @@ -245,8 +245,8 @@ group-by │ │ └── const: '2018-01-01 00:00:00+00:00' [type=timestamptz] │ └── filters [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] - │ ├── variable: xy.x [type=int, outer=(1)] - │ └── variable: uv.u [type=int, outer=(4)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: u [type=int, outer=(4)] └── aggregations [outer=(1)] └── sum [type=decimal, outer=(1)] - └── variable: xy.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] diff --git a/pkg/sql/opt/memo/testdata/logprops/select b/pkg/sql/opt/memo/testdata/logprops/select index b05fa30b147a..19f130962c84 100644 --- a/pkg/sql/opt/memo/testdata/logprops/select +++ b/pkg/sql/opt/memo/testdata/logprops/select @@ -37,7 +37,7 @@ select │ └── interesting orderings: (+1) └── filters [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] └── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] - ├── variable: xy.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 1 [type=int] build @@ -74,8 +74,8 @@ select │ └── true [type=bool] └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] - ├── variable: xy.x [type=int, outer=(1)] - └── variable: kuv.k [type=int, outer=(3)] + ├── variable: x [type=int, outer=(1)] + └── variable: k [type=int, outer=(3)] # Propagate outer columns. build @@ -123,12 +123,12 @@ select │ │ └── interesting orderings: (+3) │ └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] │ └── eq [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] - │ ├── variable: kuv.k [type=int, outer=(3)] - │ └── variable: xy.y [type=int, outer=(2)] + │ ├── variable: k [type=int, outer=(3)] + │ └── variable: y [type=int, outer=(2)] └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] - ├── variable: kuv.k [type=int, outer=(3)] - └── variable: xy.x [type=int, outer=(1)] + ├── variable: k [type=int, outer=(3)] + └── variable: x [type=int, outer=(1)] # Reduce min cardinality. build @@ -240,7 +240,7 @@ project │ └── interesting orderings: (+5) └── filters [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] └── is-not [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] - ├── variable: abcd.c [type=int, outer=(3)] + ├── variable: c [type=int, outer=(3)] └── null [type=unknown] build @@ -267,8 +267,8 @@ project │ └── interesting orderings: (+5) └── filters [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ]), fd=(3)==(4), (4)==(3)] └── eq [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ])] - ├── variable: abcd.c [type=int, outer=(3)] - └── variable: abcd.d [type=int, outer=(4)] + ├── variable: c [type=int, outer=(3)] + └── variable: d [type=int, outer=(4)] build SELECT * FROM abcd WHERE a > c @@ -293,8 +293,8 @@ project │ └── interesting orderings: (+5) └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] └── gt [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] - ├── variable: abcd.a [type=int, outer=(1)] - └── variable: abcd.c [type=int, outer=(3)] + ├── variable: a [type=int, outer=(1)] + └── variable: c [type=int, outer=(3)] build SELECT * FROM (SELECT * FROM abcd WHERE a = c) WHERE b < d @@ -325,12 +325,12 @@ select │ │ └── interesting orderings: (+5) │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] │ └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] - │ ├── variable: abcd.a [type=int, outer=(1)] - │ └── variable: abcd.c [type=int, outer=(3)] + │ ├── variable: a [type=int, outer=(1)] + │ └── variable: c [type=int, outer=(3)] └── filters [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ])] └── lt [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ])] - ├── variable: abcd.b [type=int, outer=(2)] - └── variable: abcd.d [type=int, outer=(4)] + ├── variable: b [type=int, outer=(2)] + └── variable: d [type=int, outer=(4)] # Test outer column in select filter that is part of a not-null constraint. build @@ -392,8 +392,8 @@ project │ │ │ └── interesting orderings: (+6) │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] │ │ └── eq [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - │ │ ├── variable: xy.y [type=int, outer=(7)] - │ │ └── variable: abcd.b [type=int, outer=(2)] + │ │ ├── variable: y [type=int, outer=(7)] + │ │ └── variable: b [type=int, outer=(2)] │ └── aggregations │ └── count-rows [type=int] └── const: 0 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/set b/pkg/sql/opt/memo/testdata/logprops/set index 8c241333f09c..8332ba690abb 100644 --- a/pkg/sql/opt/memo/testdata/logprops/set +++ b/pkg/sql/opt/memo/testdata/logprops/set @@ -77,7 +77,7 @@ intersect │ └── interesting orderings: (+5) └── filters [type=bool, outer=(3), constraints=(/3: [/1 - /1]; tight), fd=()-->(3)] └── eq [type=bool, outer=(3), constraints=(/3: [/1 - /1]; tight)] - ├── variable: uv.u [type=int, outer=(3)] + ├── variable: u [type=int, outer=(3)] └── const: 1 [type=int] build @@ -117,7 +117,7 @@ except │ └── interesting orderings: (+5) └── filters [type=bool, outer=(3), constraints=(/3: [/1 - /1]; tight), fd=()-->(3)] └── eq [type=bool, outer=(3), constraints=(/3: [/1 - /1]; tight)] - ├── variable: uv.u [type=int, outer=(3)] + ├── variable: u [type=int, outer=(3)] └── const: 1 [type=int] # Propagate outer columns. diff --git a/pkg/sql/opt/memo/testdata/logprops/values b/pkg/sql/opt/memo/testdata/logprops/values index 247b57037c71..29d305930e51 100644 --- a/pkg/sql/opt/memo/testdata/logprops/values +++ b/pkg/sql/opt/memo/testdata/logprops/values @@ -56,10 +56,10 @@ project ├── stats: [rows=2] ├── prune: (3) ├── tuple [type=tuple{int}, outer=(1)] - │ └── variable: xy.x [type=int, outer=(1)] + │ └── variable: x [type=int, outer=(1)] └── tuple [type=tuple{int}, outer=(2)] └── plus [type=int, outer=(2)] - ├── variable: xy.y [type=int, outer=(2)] + ├── variable: y [type=int, outer=(2)] └── const: 1 [type=int] # Single row. diff --git a/pkg/sql/opt/memo/testdata/logprops/virtual-scan b/pkg/sql/opt/memo/testdata/logprops/virtual-scan index d2205f518d3e..50575b9d0682 100644 --- a/pkg/sql/opt/memo/testdata/logprops/virtual-scan +++ b/pkg/sql/opt/memo/testdata/logprops/virtual-scan @@ -54,7 +54,7 @@ project │ │ └── stats: [rows=1000, distinct(2)=700] │ └── filters [type=bool, outer=(2), constraints=(/2: [/'public' - /'public']; tight), fd=()-->(2)] │ └── eq [type=bool, outer=(2), constraints=(/2: [/'public' - /'public']; tight)] - │ ├── variable: system.information_schema.schemata.schema_name [type=string, outer=(2)] + │ ├── variable: schema_name [type=string, outer=(2)] │ └── const: 'public' [type=string] ├── virtual-scan system.information_schema.tables │ ├── columns: table_catalog:5(string) table_schema:6(string) table_name:7(string) table_type:8(string) is_insertable_into:9(string) version:10(int) @@ -62,8 +62,8 @@ project └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ])] └── and [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ])] ├── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - │ ├── variable: system.information_schema.schemata.catalog_name [type=string, outer=(1)] - │ └── variable: system.information_schema.tables.table_catalog [type=string, outer=(5)] + │ ├── variable: catalog_name [type=string, outer=(1)] + │ └── variable: table_catalog [type=string, outer=(5)] └── eq [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] - ├── variable: system.information_schema.schemata.schema_name [type=string, outer=(2)] - └── variable: system.information_schema.tables.table_schema [type=string, outer=(6)] + ├── variable: schema_name [type=string, outer=(2)] + └── variable: table_schema [type=string, outer=(6)] diff --git a/pkg/sql/opt/memo/testdata/memo b/pkg/sql/opt/memo/testdata/memo index 7930826f0ddd..6e05e5a9bb88 100644 --- a/pkg/sql/opt/memo/testdata/memo +++ b/pkg/sql/opt/memo/testdata/memo @@ -63,7 +63,7 @@ limit │ │ └── filters [type=bool, outer=(1-3), constraints=(/2: [/2 - ])] │ │ └── and [type=bool, outer=(1-3), constraints=(/2: [/2 - ])] │ │ ├── gt [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] - │ │ │ ├── variable: a.y [type=int, outer=(2)] + │ │ │ ├── variable: y [type=int, outer=(2)] │ │ │ └── const: 1 [type=int] │ │ └── eq [type=bool, outer=(1,3)] │ │ ├── cast: STRING [type=string, outer=(1)] @@ -71,7 +71,7 @@ limit │ │ └── variable: b.x [type=string, outer=(3)] │ └── projections [outer=(2,3)] │ └── plus [type=int, outer=(2)] - │ ├── variable: a.y [type=int, outer=(2)] + │ ├── variable: y [type=int, outer=(2)] │ └── const: 1 [type=int] └── const: 10 [type=int] @@ -123,7 +123,7 @@ project │ │ │ │ └── fd: (1)-->(2) │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] │ │ │ └── gt [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] - │ │ │ ├── variable: a.y [type=int, outer=(2)] + │ │ │ ├── variable: y [type=int, outer=(2)] │ │ │ └── const: 1 [type=int] │ │ └── filters [type=bool, outer=(1,3), constraints=(/3: (/NULL - ])] │ │ └── eq [type=bool, outer=(1,3), constraints=(/3: (/NULL - ])] @@ -133,7 +133,7 @@ project │ └── const: 10 [type=int] └── projections [outer=(2,3)] └── plus [type=int, outer=(2)] - ├── variable: a.y [type=int, outer=(2)] + ├── variable: y [type=int, outer=(2)] └── const: 1 [type=int] memo @@ -158,7 +158,7 @@ memo (optimized) │ └── "[ordering: +2]" │ ├── best: (limit G4="[ordering: +2]" G5 ordering=+2) │ └── cost: 41568.75 - ├── G3: (projections G6 a.y b.x) + ├── G3: (projections G6 y x) ├── G4: (inner-join G8 G7 G9) (inner-join G7 G8 G9) │ ├── "" │ │ ├── best: (inner-join G7 G8 G9) @@ -186,7 +186,7 @@ memo (optimized) ├── G13: (gt G16 G17) ├── G14: (variable b.x) ├── G15: (cast G18 STRING) - ├── G16: (variable a.y) + ├── G16: (variable y) ├── G17: (const 1) └── G18: (variable a.x) @@ -218,12 +218,12 @@ memo (optimized) ├── G10: (eq G13 G14) ├── G11: (eq G15 G15) ├── G12: (function G18 G16 left) - ├── G13: (variable b.z) + ├── G13: (variable z) ├── G14: (const 1) ├── G15: (function G18 G17 G18 concat) ├── G16: (const 10) ├── G17: (const 'foo') - └── G18: (variable b.x) + └── G18: (variable x) # Test topological sorting memo @@ -238,7 +238,7 @@ memo (optimized) │ └── "" │ ├── best: (select G6 G7) │ └── cost: 1.05 - ├── G3: (projections a.x) + ├── G3: (projections x) ├── G4: (scan a) │ └── "" │ ├── best: (scan a) @@ -253,8 +253,8 @@ memo (optimized) ├── G9: (eq G10 G11) ├── G10: (plus G12 G13) ├── G11: (const 1) - ├── G12: (variable a.x) - └── G13: (variable a.y) + ├── G12: (variable x) + └── G13: (variable y) memo raw-memo SELECT x FROM a WHERE x = 1 AND x+y = 1 @@ -265,10 +265,10 @@ memo (optimized) │ └── "" │ ├── best: (scan a) │ └── cost: 1040.00 - ├── G2: (variable a.x) + ├── G2: (variable x) ├── G3: (const 1) ├── G4: (eq G2 G3) - ├── G5: (variable a.y) + ├── G5: (variable y) ├── G6: (plus G2 G5) ├── G7: (eq G6 G3) ├── G8: (and G4 G7) @@ -277,7 +277,7 @@ memo (optimized) │ └── "" │ ├── best: (select G15 G14) │ └── cost: 1.05 - ├── G11: (projections a.x) + ├── G11: (projections x) ├── G12: (project G10 G11) │ └── "[presentation: x:1]" │ ├── best: (project G10 G11) @@ -330,7 +330,7 @@ memo (optimized) │ └── cost: 1030.00 ├── G3: (aggregations G4) ├── G4: (array-agg G5) - └── G5: (variable a.x) + └── G5: (variable x) memo SELECT array_agg(x) FROM (SELECT * FROM a) GROUP BY y @@ -351,7 +351,7 @@ memo (optimized) │ └── cost: 1040.00 ├── G5: (aggregations G6) ├── G6: (array-agg G7) - └── G7: (variable a.x) + └── G7: (variable x) memo SELECT array_agg(x) FROM (SELECT * FROM a ORDER BY y) @@ -370,7 +370,7 @@ memo (optimized) │ └── cost: 1249.32 ├── G3: (aggregations G4) ├── G4: (array-agg G5) - └── G5: (variable a.x) + └── G5: (variable x) memo SELECT DISTINCT field FROM [EXPLAIN SELECT 123 AS k] diff --git a/pkg/sql/opt/memo/testdata/stats/groupby b/pkg/sql/opt/memo/testdata/stats/groupby index 92c5184fdb74..bdb8eb64b637 100644 --- a/pkg/sql/opt/memo/testdata/stats/groupby +++ b/pkg/sql/opt/memo/testdata/stats/groupby @@ -92,7 +92,7 @@ project │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) └── aggregations [outer=(2)] └── max [type=int, outer=(2)] - └── variable: a.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # Group by non-key. build @@ -114,7 +114,7 @@ group-by │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) └── aggregations [outer=(3)] └── sum [type=float, outer=(3)] - └── variable: a.z [type=float, outer=(3)] + └── variable: z [type=float, outer=(3)] build SELECT max(x) FROM a GROUP BY y, z, s @@ -135,7 +135,7 @@ project │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) └── aggregations [outer=(1)] └── max [type=int, outer=(1)] - └── variable: a.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] build SELECT min(x) FROM a GROUP BY y, z @@ -161,7 +161,7 @@ project │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) └── aggregations [outer=(1)] └── min [type=int, outer=(1)] - └── variable: a.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] build SELECT max(x) FROM a GROUP BY y, z, s HAVING s IN ('a', 'b') @@ -187,10 +187,10 @@ project │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── aggregations [outer=(1)] │ └── max [type=int, outer=(1)] - │ └── variable: a.x [type=int, outer=(1)] + │ └── variable: x [type=int, outer=(1)] └── filters [type=bool, outer=(4), constraints=(/4: [/'a' - /'a'] [/'b' - /'b']; tight)] └── in [type=bool, outer=(4), constraints=(/4: [/'a' - /'a'] [/'b' - /'b']; tight)] - ├── variable: a.s [type=string, outer=(4)] + ├── variable: s [type=string, outer=(4)] └── tuple [type=tuple{string, string}] ├── const: 'a' [type=string] └── const: 'b' [type=string] @@ -222,7 +222,7 @@ select │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── aggregations [outer=(1)] │ └── sum [type=decimal, outer=(1)] - │ └── variable: a.x [type=int, outer=(1)] + │ └── variable: x [type=int, outer=(1)] └── filters [type=bool, outer=(5), constraints=(/5: [/5 - /5]; tight), fd=()-->(5)] └── eq [type=bool, outer=(5), constraints=(/5: [/5 - /5]; tight)] ├── variable: column5 [type=decimal, outer=(5)] @@ -254,9 +254,9 @@ select │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── aggregations [outer=(2,3)] │ ├── sum [type=float, outer=(3)] - │ │ └── variable: a.z [type=float, outer=(3)] + │ │ └── variable: z [type=float, outer=(3)] │ └── max [type=int, outer=(2)] - │ └── variable: a.y [type=int, outer=(2)] + │ └── variable: y [type=int, outer=(2)] └── filters [type=bool, outer=(5), constraints=(/5: [/5.0 - /5.0]; tight), fd=()-->(5)] └── eq [type=bool, outer=(5), constraints=(/5: [/5.0 - /5.0]; tight)] ├── variable: column5 [type=float, outer=(5)] diff --git a/pkg/sql/opt/memo/testdata/stats/index-join b/pkg/sql/opt/memo/testdata/stats/index-join index 79e026d75343..940862680ef7 100644 --- a/pkg/sql/opt/memo/testdata/stats/index-join +++ b/pkg/sql/opt/memo/testdata/stats/index-join @@ -76,8 +76,8 @@ project │ └── filters [type=bool, outer=(1,2)] │ └── eq [type=bool, outer=(1,2)] │ ├── plus [type=int, outer=(1,2)] - │ │ ├── variable: a.x [type=int, outer=(1)] - │ │ └── variable: a.y [type=int, outer=(2)] + │ │ ├── variable: x [type=int, outer=(1)] + │ │ └── variable: y [type=int, outer=(2)] │ └── const: 10 [type=int] └── aggregations └── count-rows [type=int] diff --git a/pkg/sql/opt/memo/testdata/stats/join b/pkg/sql/opt/memo/testdata/stats/join index 716ce43a2aee..eab541df3437 100644 --- a/pkg/sql/opt/memo/testdata/stats/join +++ b/pkg/sql/opt/memo/testdata/stats/join @@ -264,7 +264,7 @@ project │ └── true [type=bool] └── filters [type=bool, outer=(6), constraints=(/6: [/5 - /5]; tight), fd=()-->(6)] └── eq [type=bool, outer=(6), constraints=(/6: [/5 - /5]; tight)] - ├── variable: b.z [type=int, outer=(6)] + ├── variable: z [type=int, outer=(6)] └── const: 5 [type=int] # Force calculation of the distinct count for the column set spanning both @@ -299,7 +299,7 @@ group-by │ └── true [type=bool] └── aggregations [outer=(6)] └── sum [type=decimal, outer=(6)] - └── variable: b.z [type=int, outer=(6)] + └── variable: z [type=int, outer=(6)] # Join selectivity: 1/max(distinct(a.x), distinct(b.x)) = 1/5000. norm @@ -328,7 +328,7 @@ group-by │ └── variable: b.x [type=int, outer=(5)] └── aggregations [outer=(6)] └── sum [type=decimal, outer=(6)] - └── variable: b.z [type=int, outer=(6)] + └── variable: z [type=int, outer=(6)] # Semi-join. norm @@ -395,8 +395,8 @@ inner-join │ ├── variable: a.x [type=int, outer=(1)] │ └── variable: b.x [type=int, outer=(5)] └── eq [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] - ├── variable: a.y [type=int, outer=(2)] - └── variable: b.z [type=int, outer=(6)] + ├── variable: y [type=int, outer=(2)] + └── variable: z [type=int, outer=(6)] # Equality condition + extra filters. norm @@ -418,10 +418,10 @@ inner-join │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── filters [type=bool, outer=(2), constraints=(/2: [/1 - /299]; tight)] │ ├── gt [type=bool, outer=(2), constraints=(/2: [/1 - ]; tight)] - │ │ ├── variable: a.y [type=int, outer=(2)] + │ │ ├── variable: y [type=int, outer=(2)] │ │ └── const: 0 [type=int] │ └── lt [type=bool, outer=(2), constraints=(/2: (/NULL - /299]; tight)] - │ ├── variable: a.y [type=int, outer=(2)] + │ ├── variable: y [type=int, outer=(2)] │ └── const: 300 [type=int] ├── scan b │ ├── columns: b.x:5(int) z:6(int!null) @@ -432,8 +432,8 @@ inner-join │ └── variable: b.x [type=int, outer=(5)] └── eq [type=bool, outer=(2,6)] ├── plus [type=int, outer=(2,6)] - │ ├── variable: a.y [type=int, outer=(2)] - │ └── variable: b.z [type=int, outer=(6)] + │ ├── variable: y [type=int, outer=(2)] + │ └── variable: z [type=int, outer=(6)] └── const: 5 [type=int] # Force column statistics calculation for semi-join. @@ -470,8 +470,8 @@ project │ │ └── variable: b.x [type=int, outer=(5)] │ └── eq [type=bool, outer=(2,6)] │ ├── plus [type=int, outer=(2,6)] - │ │ ├── variable: a.y [type=int, outer=(2)] - │ │ └── variable: b.z [type=int, outer=(6)] + │ │ ├── variable: y [type=int, outer=(2)] + │ │ └── variable: z [type=int, outer=(6)] │ └── const: 5 [type=int] └── aggregations └── count-rows [type=int] @@ -510,8 +510,8 @@ project │ │ └── variable: b.x [type=int, outer=(5)] │ └── eq [type=bool, outer=(2,6)] │ ├── plus [type=int, outer=(2,6)] - │ │ ├── variable: a.y [type=int, outer=(2)] - │ │ └── variable: b.z [type=int, outer=(6)] + │ │ ├── variable: y [type=int, outer=(2)] + │ │ └── variable: z [type=int, outer=(6)] │ └── const: 5 [type=int] └── aggregations └── count-rows [type=int] @@ -549,8 +549,8 @@ project │ │ └── variable: b.x [type=int, outer=(5)] │ └── eq [type=bool, outer=(2,6)] │ ├── plus [type=int, outer=(2,6)] - │ │ ├── variable: a.y [type=int, outer=(2)] - │ │ └── variable: b.z [type=int, outer=(6)] + │ │ ├── variable: y [type=int, outer=(2)] + │ │ └── variable: z [type=int, outer=(6)] │ └── const: 5 [type=int] └── aggregations └── count-rows [type=int] @@ -588,8 +588,8 @@ project │ │ └── variable: b.x [type=int, outer=(5)] │ └── eq [type=bool, outer=(2,6)] │ ├── plus [type=int, outer=(2,6)] - │ │ ├── variable: a.y [type=int, outer=(2)] - │ │ └── variable: b.z [type=int, outer=(6)] + │ │ ├── variable: y [type=int, outer=(2)] + │ │ └── variable: z [type=int, outer=(6)] │ └── const: 5 [type=int] └── aggregations └── count-rows [type=int] @@ -627,8 +627,8 @@ project │ │ └── variable: b.x [type=int, outer=(5)] │ └── eq [type=bool, outer=(2,6)] │ ├── plus [type=int, outer=(2,6)] - │ │ ├── variable: a.y [type=int, outer=(2)] - │ │ └── variable: b.z [type=int, outer=(6)] + │ │ ├── variable: y [type=int, outer=(2)] + │ │ └── variable: z [type=int, outer=(6)] │ └── const: 5 [type=int] └── aggregations └── count-rows [type=int] @@ -705,8 +705,8 @@ inner-join (lookup def) │ └── fd: (1,3)-->(2) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - ├── variable: abc.a [type=int, outer=(1)] - └── variable: def.f [type=int, outer=(6)] + ├── variable: a [type=int, outer=(1)] + └── variable: f [type=int, outer=(6)] # The filter a=e is not very selective, so we do not expect a lookup join. opt @@ -734,5 +734,5 @@ inner-join (merge) ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── variable: abc.a [type=int, outer=(1)] - └── variable: def.e [type=int, outer=(5)] + ├── variable: a [type=int, outer=(1)] + └── variable: e [type=int, outer=(5)] diff --git a/pkg/sql/opt/memo/testdata/stats/limit b/pkg/sql/opt/memo/testdata/stats/limit index 765138ce45e7..16817d44fd61 100644 --- a/pkg/sql/opt/memo/testdata/stats/limit +++ b/pkg/sql/opt/memo/testdata/stats/limit @@ -63,7 +63,7 @@ limit │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── filters [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] │ └── eq [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight)] - │ ├── variable: a.s [type=string, outer=(3)] + │ ├── variable: s [type=string, outer=(3)] │ └── const: 'foo' [type=string] └── const: 5 [type=int] @@ -87,7 +87,7 @@ limit │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── filters [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] │ └── eq [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight)] - │ ├── variable: a.s [type=string, outer=(3)] + │ ├── variable: s [type=string, outer=(3)] │ └── const: 'foo' [type=string] └── subquery [type=int] └── max1-row @@ -140,5 +140,5 @@ select │ └── const: 5 [type=int] └── filters [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] └── eq [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight)] - ├── variable: a.s [type=string, outer=(3)] + ├── variable: s [type=string, outer=(3)] └── const: 'foo' [type=string] diff --git a/pkg/sql/opt/memo/testdata/stats/ordinality b/pkg/sql/opt/memo/testdata/stats/ordinality index 1590af3847df..229ea7dad83d 100644 --- a/pkg/sql/opt/memo/testdata/stats/ordinality +++ b/pkg/sql/opt/memo/testdata/stats/ordinality @@ -70,10 +70,10 @@ select │ └── fd: (1)-->(2) └── filters [type=bool, outer=(2), constraints=(/2: [/1 - /10]; tight)] ├── gt [type=bool, outer=(2), constraints=(/2: [/1 - ]; tight)] - │ ├── variable: a.y [type=int, outer=(2)] + │ ├── variable: y [type=int, outer=(2)] │ └── const: 0 [type=int] └── le [type=bool, outer=(2), constraints=(/2: (/NULL - /10]; tight)] - ├── variable: a.y [type=int, outer=(2)] + ├── variable: y [type=int, outer=(2)] └── const: 10 [type=int] norm diff --git a/pkg/sql/opt/memo/testdata/stats/project b/pkg/sql/opt/memo/testdata/stats/project index 275b23caf496..3dead52bb95c 100644 --- a/pkg/sql/opt/memo/testdata/stats/project +++ b/pkg/sql/opt/memo/testdata/stats/project @@ -110,9 +110,9 @@ select │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── projections [outer=(2,3)] │ └── function: concat [type=string, outer=(2,3)] - │ ├── variable: a.s [type=string, outer=(3)] + │ ├── variable: s [type=string, outer=(3)] │ └── cast: STRING [type=string, outer=(2)] - │ └── variable: a.y [type=int, outer=(2)] + │ └── variable: y [type=int, outer=(2)] └── filters [type=bool, outer=(5), constraints=(/5: [/'foo' - /'foo']; tight), fd=()-->(5)] └── eq [type=bool, outer=(5), constraints=(/5: [/'foo' - /'foo']; tight)] ├── variable: concat [type=string, outer=(5)] @@ -140,9 +140,9 @@ group-by │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) └── projections [outer=(1-3)] └── function: concat [type=string, outer=(2,3)] - ├── variable: a.s [type=string, outer=(3)] + ├── variable: s [type=string, outer=(3)] └── cast: STRING [type=string, outer=(2)] - └── variable: a.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # No available stats for column y. build @@ -161,7 +161,7 @@ select │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── projections [outer=(2)] │ └── plus [type=int, outer=(2)] - │ ├── variable: a.y [type=int, outer=(2)] + │ ├── variable: y [type=int, outer=(2)] │ └── const: 3 [type=int] └── filters [type=bool, outer=(5), constraints=(/5: [/1 - /100]; tight)] └── and [type=bool, outer=(5), constraints=(/5: [/1 - /100]; tight)] @@ -232,5 +232,5 @@ select │ └── fd: (5)-->(6,7) └── projections [outer=(3,7)] └── lt [type=bool, outer=(3,7)] - ├── variable: a.s [type=string, outer=(3)] - └── variable: kuv.v [type=string, outer=(7)] + ├── variable: s [type=string, outer=(3)] + └── variable: v [type=string, outer=(7)] diff --git a/pkg/sql/opt/memo/testdata/stats/scan b/pkg/sql/opt/memo/testdata/stats/scan index 1937fe02b67f..96198820450a 100644 --- a/pkg/sql/opt/memo/testdata/stats/scan +++ b/pkg/sql/opt/memo/testdata/stats/scan @@ -50,7 +50,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── filters [type=bool, outer=(5), constraints=(/5: [/true - /true]; tight), fd=()-->(5)] - └── variable: a.b [type=bool, outer=(5), constraints=(/5: [/true - /true]; tight)] + └── variable: b [type=bool, outer=(5), constraints=(/5: [/true - /true]; tight)] exec-ddl ALTER TABLE a INJECT STATISTICS '[ @@ -137,7 +137,7 @@ group-by └── aggregations [outer=(2)] ├── count-rows [type=int] └── const-agg [type=int, outer=(2)] - └── variable: a.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # Test calculation of multi-column stats. opt diff --git a/pkg/sql/opt/memo/testdata/stats/select b/pkg/sql/opt/memo/testdata/stats/select index 7fee8539edda..fa1fd45cd43a 100644 --- a/pkg/sql/opt/memo/testdata/stats/select +++ b/pkg/sql/opt/memo/testdata/stats/select @@ -96,16 +96,16 @@ project │ └── fd: (3)-->(1,2) └── filters [type=bool, outer=(1-3), constraints=(/1: [/1 - /1]; /2: [/2 - /2]; /3: [/5 - /8]; tight), fd=()-->(1,2)] ├── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] - │ ├── variable: b.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 1 [type=int] ├── eq [type=bool, outer=(2), constraints=(/2: [/2 - /2]; tight)] - │ ├── variable: b.z [type=int, outer=(2)] + │ ├── variable: z [type=int, outer=(2)] │ └── const: 2 [type=int] ├── ge [type=bool, outer=(3), constraints=(/3: [/5 - ]; tight)] - │ ├── variable: b.rowid [type=int, outer=(3)] + │ ├── variable: rowid [type=int, outer=(3)] │ └── const: 5 [type=int] └── le [type=bool, outer=(3), constraints=(/3: (/NULL - /8]; tight)] - ├── variable: b.rowid [type=int, outer=(3)] + ├── variable: rowid [type=int, outer=(3)] └── const: 8 [type=int] # Can't determine stats from filter. @@ -125,8 +125,8 @@ select └── filters [type=bool, outer=(1,2)] └── lt [type=bool, outer=(1,2)] ├── plus [type=int, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── const: 10 [type=int] # Remaining filter. @@ -145,12 +145,12 @@ select │ └── fd: (1)-->(2) └── filters [type=bool, outer=(1,2), constraints=(/2: [/5 - /5]), fd=()-->(2)] ├── eq [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] - │ ├── variable: a.y [type=int, outer=(2)] + │ ├── variable: y [type=int, outer=(2)] │ └── const: 5 [type=int] └── lt [type=bool, outer=(1,2)] ├── plus [type=int, outer=(1,2)] - │ ├── variable: a.x [type=int, outer=(1)] - │ └── variable: a.y [type=int, outer=(2)] + │ ├── variable: x [type=int, outer=(1)] + │ └── variable: y [type=int, outer=(2)] └── const: 10 [type=int] # Contradiction. @@ -170,7 +170,7 @@ select │ └── fd: (1)-->(2) └── filters [type=bool, outer=(1), constraints=(/1: [/NULL - /NULL]; tight), fd=()-->(1)] └── is [type=bool, outer=(1), constraints=(/1: [/NULL - /NULL]; tight)] - ├── variable: a.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── null [type=unknown] norm @@ -193,14 +193,14 @@ project │ │ └── stats: [rows=10000, distinct(1)=5000, distinct(2)=100] │ └── filters [type=bool, outer=(1), constraints=(/1: [/1001 - /2000]; tight)] │ ├── gt [type=bool, outer=(1), constraints=(/1: [/1001 - ]; tight)] - │ │ ├── variable: b.x [type=int, outer=(1)] + │ │ ├── variable: x [type=int, outer=(1)] │ │ └── const: 1000 [type=int] │ └── le [type=bool, outer=(1), constraints=(/1: (/NULL - /2000]; tight)] - │ ├── variable: b.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 2000 [type=int] └── aggregations [outer=(1)] └── sum [type=decimal, outer=(1)] - └── variable: b.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] # Regression: statistics builder panics when end key is NULL when it's trying # to compute start/end int boundaries. @@ -233,7 +233,7 @@ project │ └── stats: [rows=333.333333] └── filters [type=bool, outer=(3), constraints=(/3: (/NULL - /9]; tight)] └── lt [type=bool, outer=(3), constraints=(/3: (/NULL - /9]; tight)] - ├── variable: idx.z [type=int, outer=(3)] + ├── variable: z [type=int, outer=(3)] └── const: 10 [type=int] # Regression: certain queries could cause a NaN expected number of rows via a divide-by-zero. @@ -342,10 +342,10 @@ select │ └── fd: (1,2)-->(3) └── filters [type=bool, outer=(1,3), constraints=(/1: [/1 - /1]; /3: [/'bobs_burgers' - /'bobs_burgers']; tight), fd=()-->(1,3)] ├── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] - │ ├── variable: district.d_id [type=int, outer=(1)] + │ ├── variable: d_id [type=int, outer=(1)] │ └── const: 1 [type=int] └── eq [type=bool, outer=(3), constraints=(/3: [/'bobs_burgers' - /'bobs_burgers']; tight)] - ├── variable: district.d_name [type=string, outer=(3)] + ├── variable: d_name [type=string, outer=(3)] └── const: 'bobs_burgers' [type=string] # In this case we expect to use unknownFilterSelectivity @@ -366,10 +366,10 @@ select │ └── fd: (1,2)-->(3) └── filters [type=bool, outer=(1,3), constraints=(/1: [/1 - /1]), fd=()-->(1)] ├── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] - │ ├── variable: district.d_id [type=int, outer=(1)] + │ ├── variable: d_id [type=int, outer=(1)] │ └── const: 1 [type=int] └── like [type=bool, outer=(3)] - ├── variable: district.d_name [type=string, outer=(3)] + ├── variable: d_name [type=string, outer=(3)] └── const: 'bob' [type=string] # This tests selectivityFromReducedCols. @@ -392,16 +392,16 @@ select │ └── fd: (1,2)-->(3) └── filters [type=bool, outer=(1-3), constraints=(/1: [/2 - /9]; /2: [/10 - /10]; /3: [/'bobs_burgers' - /'bobs_burgers']; tight), fd=()-->(2,3)] ├── gt [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] - │ ├── variable: district.d_id [type=int, outer=(1)] + │ ├── variable: d_id [type=int, outer=(1)] │ └── const: 1 [type=int] ├── lt [type=bool, outer=(1), constraints=(/1: (/NULL - /9]; tight)] - │ ├── variable: district.d_id [type=int, outer=(1)] + │ ├── variable: d_id [type=int, outer=(1)] │ └── const: 10 [type=int] ├── eq [type=bool, outer=(2), constraints=(/2: [/10 - /10]; tight)] - │ ├── variable: district.d_w_id [type=int, outer=(2)] + │ ├── variable: d_w_id [type=int, outer=(2)] │ └── const: 10 [type=int] └── eq [type=bool, outer=(3), constraints=(/3: [/'bobs_burgers' - /'bobs_burgers']; tight)] - ├── variable: district.d_name [type=string, outer=(3)] + ├── variable: d_name [type=string, outer=(3)] └── const: 'bobs_burgers' [type=string] # This tests selectivityFromReducedCols @@ -422,13 +422,13 @@ select │ └── fd: (1,2)-->(3) └── filters [type=bool, outer=(1-3), constraints=(/1: [/1 - /1]; /2: [/10 - /10]; /3: [/'hello' - /'hello']; tight), fd=()-->(1-3)] ├── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] - │ ├── variable: district.d_id [type=int, outer=(1)] + │ ├── variable: d_id [type=int, outer=(1)] │ └── const: 1 [type=int] ├── eq [type=bool, outer=(2), constraints=(/2: [/10 - /10]; tight)] - │ ├── variable: district.d_w_id [type=int, outer=(2)] + │ ├── variable: d_w_id [type=int, outer=(2)] │ └── const: 10 [type=int] └── eq [type=bool, outer=(3), constraints=(/3: [/'hello' - /'hello']; tight)] - ├── variable: district.d_name [type=string, outer=(3)] + ├── variable: d_name [type=string, outer=(3)] └── const: 'hello' [type=string] exec-ddl @@ -493,11 +493,11 @@ select │ │ └── true [type=bool] │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - │ ├── variable: customers.id [type=int, outer=(1)] - │ └── variable: order_history.customer_id [type=int, outer=(6)] + │ ├── variable: id [type=int, outer=(1)] + │ └── variable: customer_id [type=int, outer=(6)] └── filters [type=bool, outer=(2), constraints=(/2: [/'andy' - /'andy']; tight), fd=()-->(2)] └── eq [type=bool, outer=(2), constraints=(/2: [/'andy' - /'andy']; tight)] - ├── variable: customers.name [type=string, outer=(2)] + ├── variable: name [type=string, outer=(2)] └── const: 'andy' [type=string] # This tests selectivityFromReducedCols @@ -545,15 +545,15 @@ select │ │ └── true [type=bool] │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - │ ├── variable: customers.id [type=int, outer=(1)] - │ └── variable: order_history.customer_id [type=int, outer=(6)] + │ ├── variable: id [type=int, outer=(1)] + │ └── variable: customer_id [type=int, outer=(6)] └── filters [type=bool, outer=(1,2), constraints=(/1: [/1 - /1]; /2: [/'andy' - /'andy']; tight), fd=()-->(1,2)] └── and [type=bool, outer=(1,2), constraints=(/1: [/1 - /1]; /2: [/'andy' - /'andy']; tight)] ├── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] - │ ├── variable: customers.id [type=int, outer=(1)] + │ ├── variable: id [type=int, outer=(1)] │ └── const: 1 [type=int] └── eq [type=bool, outer=(2), constraints=(/2: [/'andy' - /'andy']; tight)] - ├── variable: customers.name [type=string, outer=(2)] + ├── variable: name [type=string, outer=(2)] └── const: 'andy' [type=string] # Test equality conditions where all have distinct count 1. @@ -569,13 +569,13 @@ select │ └── stats: [rows=1000, distinct(1)=700, distinct(2)=700, distinct(3)=700] └── filters [type=bool, outer=(1-3), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /3: [/5 - /5]), fd=()-->(1-3), (1)==(2,3), (2)==(1,3), (3)==(1,2)] ├── eq [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - │ ├── variable: order_history.item_id [type=int, outer=(2)] - │ └── variable: order_history.order_id [type=int, outer=(1)] + │ ├── variable: item_id [type=int, outer=(2)] + │ └── variable: order_id [type=int, outer=(1)] ├── eq [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] - │ ├── variable: order_history.item_id [type=int, outer=(2)] - │ └── variable: order_history.customer_id [type=int, outer=(3)] + │ ├── variable: item_id [type=int, outer=(2)] + │ └── variable: customer_id [type=int, outer=(3)] └── eq [type=bool, outer=(3), constraints=(/3: [/5 - /5]; tight)] - ├── variable: order_history.customer_id [type=int, outer=(3)] + ├── variable: customer_id [type=int, outer=(3)] └── const: 5 [type=int] # Test equality condition with another condition on one of the attributes. @@ -591,13 +591,13 @@ select │ └── stats: [rows=1000, distinct(1)=700, distinct(2)=700] └── filters [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: [/1 - /4]), fd=(1)==(2), (2)==(1)] ├── eq [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - │ ├── variable: order_history.item_id [type=int, outer=(2)] - │ └── variable: order_history.order_id [type=int, outer=(1)] + │ ├── variable: item_id [type=int, outer=(2)] + │ └── variable: order_id [type=int, outer=(1)] ├── lt [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] - │ ├── variable: order_history.item_id [type=int, outer=(2)] + │ ├── variable: item_id [type=int, outer=(2)] │ └── const: 5 [type=int] └── gt [type=bool, outer=(2), constraints=(/2: [/1 - ]; tight)] - ├── variable: order_history.item_id [type=int, outer=(2)] + ├── variable: item_id [type=int, outer=(2)] └── const: 0 [type=int] # Test equality condition with another condition on a different attribute. @@ -613,13 +613,13 @@ select │ └── stats: [rows=1000, distinct(1)=700, distinct(2)=700] └── filters [type=bool, outer=(1-3), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /3: [/1 - /4]), fd=(1)==(2), (2)==(1)] ├── eq [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - │ ├── variable: order_history.item_id [type=int, outer=(2)] - │ └── variable: order_history.order_id [type=int, outer=(1)] + │ ├── variable: item_id [type=int, outer=(2)] + │ └── variable: order_id [type=int, outer=(1)] ├── lt [type=bool, outer=(3), constraints=(/3: (/NULL - /4]; tight)] - │ ├── variable: order_history.customer_id [type=int, outer=(3)] + │ ├── variable: customer_id [type=int, outer=(3)] │ └── const: 5 [type=int] └── gt [type=bool, outer=(3), constraints=(/3: [/1 - ]; tight)] - ├── variable: order_history.customer_id [type=int, outer=(3)] + ├── variable: customer_id [type=int, outer=(3)] └── const: 0 [type=int] # Test equality condition with another filter condition without a constraint. @@ -635,11 +635,11 @@ select │ └── stats: [rows=1000, distinct(1)=700, distinct(2)=700] └── filters [type=bool, outer=(1-3), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] ├── eq [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - │ ├── variable: order_history.item_id [type=int, outer=(2)] - │ └── variable: order_history.order_id [type=int, outer=(1)] + │ ├── variable: item_id [type=int, outer=(2)] + │ └── variable: order_id [type=int, outer=(1)] └── eq [type=bool, outer=(3)] ├── mod [type=int, outer=(3)] - │ ├── variable: order_history.customer_id [type=int, outer=(3)] + │ ├── variable: customer_id [type=int, outer=(3)] │ └── const: 2 [type=int] └── const: 0 [type=int] @@ -671,8 +671,8 @@ select │ └── fd: (1)~~>(2) └── filters [type=bool, outer=(1), constraints=(/1: [/0 - /99]; tight)] ├── ge [type=bool, outer=(1), constraints=(/1: [/0 - ]; tight)] - │ ├── variable: c.x [type=int, outer=(1)] + │ ├── variable: x [type=int, outer=(1)] │ └── const: 0 [type=int] └── lt [type=bool, outer=(1), constraints=(/1: (/NULL - /99]; tight)] - ├── variable: c.x [type=int, outer=(1)] + ├── variable: x [type=int, outer=(1)] └── const: 100 [type=int] diff --git a/pkg/sql/opt/memo/testdata/stats/set b/pkg/sql/opt/memo/testdata/stats/set index 3dec4e33bbf9..33b2e8275b74 100644 --- a/pkg/sql/opt/memo/testdata/stats/set +++ b/pkg/sql/opt/memo/testdata/stats/set @@ -577,5 +577,5 @@ select │ └── fd: (7)-->(4-6) └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] └── eq [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] - ├── variable: a.y [type=int, outer=(2)] + ├── variable: y [type=int, outer=(2)] └── const: 5 [type=int] diff --git a/pkg/sql/opt/memo/testdata/typing b/pkg/sql/opt/memo/testdata/typing index fcafa6b2a70c..43c7c806b518 100644 --- a/pkg/sql/opt/memo/testdata/typing +++ b/pkg/sql/opt/memo/testdata/typing @@ -57,7 +57,7 @@ select ├── scan a │ └── columns: x:1(int!null) y:2(int) └── filters [type=bool] - └── a.x = $1 [type=bool] + └── x = $1 [type=bool] # Tuple, Projections build @@ -68,7 +68,7 @@ project ├── scan a │ └── columns: x:1(int!null) y:2(int) └── projections - └── (a.x, 1.5) [type=tuple{int, decimal}] + └── (x, 1.5) [type=tuple{int, decimal}] # And, Or, Not build @@ -79,7 +79,7 @@ select ├── scan a │ └── columns: x:1(int!null) y:2(int) └── filters [type=bool] - └── (a.x = 1) AND (NOT ((a.y = 2) OR (a.y = 3.5))) [type=bool] + └── (x = 1) AND (NOT ((y = 2) OR (y = 3.5))) [type=bool] # Eq, Ne build @@ -90,7 +90,7 @@ select ├── scan a │ └── columns: x:1(int!null) y:2(int) └── filters [type=bool] - └── (a.x = 1) AND (a.x != 2) [type=bool] + └── (x = 1) AND (x != 2) [type=bool] # Le, Ge, Lt, Gt build @@ -101,7 +101,7 @@ select ├── scan a │ └── columns: x:1(int!null) y:2(int) └── filters [type=bool] - └── (((a.x >= 1) AND (a.x <= 10)) AND (a.y > 1)) AND (a.y < 10) [type=bool] + └── (((x >= 1) AND (x <= 10)) AND (y > 1)) AND (y < 10) [type=bool] # In, NotIn build @@ -112,7 +112,7 @@ select ├── scan a │ └── columns: x:1(int!null) y:2(int) └── filters [type=bool] - └── (a.x IN (1, 2)) AND (a.y NOT IN (3, 4)) [type=bool] + └── (x IN (1, 2)) AND (y NOT IN (3, 4)) [type=bool] # Like, NotLike build @@ -123,7 +123,7 @@ select ├── scan b │ └── columns: x:1(string!null) z:2(decimal!null) └── filters [type=bool] - └── (b.x LIKE '%foo%') AND (b.x NOT LIKE '%bar%') [type=bool] + └── (x LIKE '%foo%') AND (x NOT LIKE '%bar%') [type=bool] # ILike, INotLike build @@ -134,7 +134,7 @@ select ├── scan b │ └── columns: x:1(string!null) z:2(decimal!null) └── filters [type=bool] - └── (b.x ILIKE '%foo%') AND (b.x NOT ILIKE '%bar%') [type=bool] + └── (x ILIKE '%foo%') AND (x NOT ILIKE '%bar%') [type=bool] # RegMatch, NotRegMatch, RegIMatch, NotRegIMatch build @@ -145,7 +145,7 @@ select ├── scan b │ └── columns: x:1(string!null) z:2(decimal!null) └── filters [type=bool] - └── (((b.x ~ 'foo') AND (b.x !~ 'bar')) AND (b.x ~* 'foo')) AND (b.x !~* 'bar') [type=bool] + └── (((x ~ 'foo') AND (x !~ 'bar')) AND (x ~* 'foo')) AND (x !~* 'bar') [type=bool] # Is, IsNot build @@ -156,7 +156,7 @@ select ├── scan a │ └── columns: x:1(int!null) y:2(int) └── filters [type=bool] - └── (a.x IS DISTINCT FROM a.y) AND (a.x IS NULL) [type=bool] + └── (x IS DISTINCT FROM y) AND (x IS NULL) [type=bool] # Bitand, Bitor, Bitxor build @@ -167,9 +167,9 @@ project ├── scan a │ └── columns: x:1(int!null) y:2(int) └── projections - ├── a.x & a.y [type=int] - ├── a.x | a.y [type=int] - └── a.x # a.y [type=int] + ├── x & y [type=int] + ├── x | y [type=int] + └── x # y [type=int] # Plus, Minus, Mult, Div, FloorDiv build @@ -185,11 +185,11 @@ project ├── scan a │ └── columns: x:1(int!null) y:2(int) └── projections - ├── a.x + 1.5 [type=decimal] + ├── x + 1.5 [type=decimal] ├── '2000-01-01' - 15 [type=date] - ├── 10.10 * a.x [type=decimal] - ├── 1 / a.y [type=decimal] - └── a.x // 1.5 [type=decimal] + ├── 10.10 * x [type=decimal] + ├── 1 / y [type=decimal] + └── x // 1.5 [type=decimal] # Mod, Pow, LShift, RShift build @@ -204,10 +204,10 @@ project ├── scan a │ └── columns: x:1(int!null) y:2(int) └── projections - ├── 100.1 % a.x [type=decimal] - ├── a.x ^ 2.5 [type=decimal] - ├── a.x << 3 [type=int] - └── a.y >> 2 [type=int] + ├── 100.1 % x [type=decimal] + ├── x ^ 2.5 [type=decimal] + ├── x << 3 [type=int] + └── y >> 2 [type=int] # FetchVal, FetchText, FetchValPath, FetchTextPath build @@ -236,7 +236,7 @@ project ├── scan b │ └── columns: x:1(string!null) z:2(decimal!null) └── projections - └── b.x || 'more' [type=string] + └── x || 'more' [type=string] # UnaryMinus, UnaryComplement build @@ -247,8 +247,8 @@ project ├── scan a │ └── columns: x:1(int!null) y:2(int) └── projections - ├── -a.y [type=int] - └── ~a.x [type=int] + ├── -y [type=int] + └── ~x [type=int] # Array Concat build @@ -259,9 +259,9 @@ project ├── scan unusual │ └── columns: x:1(int!null) arr:2(int[]) └── projections - ├── unusual.arr || unusual.arr [type=int[]] - ├── unusual.arr || NULL::INT[] [type=int[]] - └── NULL::INT[] || unusual.arr [type=int[]] + ├── arr || arr [type=int[]] + ├── arr || NULL::INT[] [type=int[]] + └── NULL::INT[] || arr [type=int[]] # Array Element Concat build @@ -272,10 +272,10 @@ project ├── scan unusual │ └── columns: x:1(int!null) arr:2(int[]) └── projections - ├── unusual.x || unusual.arr [type=int[]] - ├── unusual.arr || unusual.x [type=int[]] - ├── unusual.x || NULL::INT[] [type=int[]] - └── NULL::INT[] || unusual.x [type=int[]] + ├── x || arr [type=int[]] + ├── arr || x [type=int[]] + ├── x || NULL::INT[] [type=int[]] + └── NULL::INT[] || x [type=int[]] # Function with fixed return type. build @@ -336,38 +336,38 @@ scalar-group-by │ ├── scan b │ │ └── columns: x:1(string!null) z:2(decimal!null) │ └── projections - │ ├── b.z = 0 [type=bool] - │ ├── b.x::INT [type=int] - │ ├── b.x::JSON [type=jsonb] - │ └── b.x::JSONB [type=jsonb] + │ ├── z = 0 [type=bool] + │ ├── x::INT [type=int] + │ ├── x::JSON [type=jsonb] + │ └── x::JSONB [type=jsonb] └── aggregations ├── array-agg [type=decimal[]] - │ └── variable: b.z [type=decimal] + │ └── variable: z [type=decimal] ├── avg [type=decimal] - │ └── variable: b.z [type=decimal] + │ └── variable: z [type=decimal] ├── bool-and [type=bool] │ └── variable: column5 [type=bool] ├── bool-or [type=bool] │ └── variable: column5 [type=bool] ├── concat-agg [type=string] - │ └── variable: b.x [type=string] + │ └── variable: x [type=string] ├── count [type=int] - │ └── variable: b.z [type=decimal] + │ └── variable: z [type=decimal] ├── count-rows [type=int] ├── max [type=string] - │ └── variable: b.x [type=string] + │ └── variable: x [type=string] ├── max [type=decimal] - │ └── variable: b.z [type=decimal] + │ └── variable: z [type=decimal] ├── sum-int [type=int] │ └── variable: column13 [type=int] ├── sum [type=decimal] - │ └── variable: b.z [type=decimal] + │ └── variable: z [type=decimal] ├── sqr-diff [type=decimal] - │ └── variable: b.z [type=decimal] + │ └── variable: z [type=decimal] ├── variance [type=decimal] │ └── variable: column13 [type=int] ├── std-dev [type=decimal] - │ └── variable: b.z [type=decimal] + │ └── variable: z [type=decimal] ├── xor-agg [type=int] │ └── variable: column13 [type=int] ├── json-agg [type=jsonb] @@ -393,12 +393,12 @@ project │ │ │ ├── scan b │ │ │ │ └── columns: b.x:4(string!null) z:5(decimal!null) │ │ │ └── filters [type=bool] - │ │ │ └── a.y = b.z::INT [type=bool] + │ │ │ └── y = z::INT [type=bool] │ │ └── aggregations │ │ ├── max [type=string] │ │ │ └── variable: b.x [type=string] │ │ └── const-agg [type=int] - │ │ └── variable: a.y [type=int] + │ │ └── variable: y [type=int] │ └── filters [type=bool] │ └── max > 'foo' [type=bool] └── projections diff --git a/pkg/sql/opt/memo/typing_test.go b/pkg/sql/opt/memo/typing_test.go index 4748bf0e5a36..c0c164a16c66 100644 --- a/pkg/sql/opt/memo/typing_test.go +++ b/pkg/sql/opt/memo/typing_test.go @@ -25,7 +25,7 @@ import ( ) func TestTyping(t *testing.T) { - runDataDrivenTest(t, "testdata/typing", opt.ExprFmtHideAll) + runDataDrivenTest(t, "testdata/typing", memo.ExprFmtHideAll) } func TestBinaryOverloadExists(t *testing.T) { diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index a582a64581d5..0c7b448d4799 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -153,11 +153,6 @@ type mdColumn struct { typ types.T } -// NewMetadata constructs a new instance of metadata for the optimizer. -func NewMetadata() *Metadata { - return &Metadata{} -} - // AddColumn assigns a new unique id to a column within the query and records // its label and type. func (md *Metadata) AddColumn(label string, typ types.T) ColumnID { @@ -211,28 +206,70 @@ func (md *Metadata) ColumnType(id ColumnID) types.T { // ColumnOrdinal returns the ordinal position of the column in its base table. // It panics if the column has no base table because it was synthesized. func (md *Metadata) ColumnOrdinal(id ColumnID) int { - tabID := md.cols[id].tabID + tabID := md.cols[id-1].tabID if tabID == 0 { panic("column was synthesized and has no ordinal position") } return int(id - tabID.firstColID()) } +// QualifiedColumnLabel returns the column label, possibly qualified with the +// table name if either of these conditions is true: +// +// 1. fullyQualify is true +// 2. the label might be ambiguous if it's not qualified, because there's +// another column in the metadata with the same name +// +// If the column label is qualified, the table is prefixed to it and separated +// by a "." character. +func (md *Metadata) QualifiedColumnLabel(id ColumnID, fullyQualify bool) string { + col := md.cols[id-1] + if col.tabID == 0 { + // Column doesn't belong to a table, so no need to qualify it further. + return col.label + } + + // If a fully qualified label has not been requested, then only qualify it if + // it would otherwise be ambiguous. + ambiguous := fullyQualify + if !fullyQualify { + for i := range md.cols { + if i == int(id-1) { + continue + } + + // If there are two columns with same name, then column is ambiguous. + otherCol := &md.cols[i] + if otherCol.label == col.label { + ambiguous = true + break + } + } + } + + if !ambiguous { + return col.label + } + + var sb strings.Builder + tabName := md.Table(col.tabID).Name() + if fullyQualify { + sb.WriteString(tabName.FQString()) + } else { + sb.WriteString(string(tabName.TableName)) + } + sb.WriteRune('.') + sb.WriteString(col.label) + return sb.String() +} + // AddTable indexes a new reference to a table within the query. Separate // references to the same table are assigned different table ids (e.g. in a // self-join query). func (md *Metadata) AddTable(tab Table) TableID { - return md.AddTableWithName(tab, string(tab.Name().TableName)) -} - -// AddTableWithName indexes a new reference to a table within the query. -// Separate references to the same table are assigned different table ids -// (e.g. in a self-join query). The given table name is used when creating -// column labels. -func (md *Metadata) AddTableWithName(tab Table, tabName string) TableID { tabID := makeTableID(len(md.tables), ColumnID(len(md.cols)+1)) if md.tables == nil { - md.tables = make([]mdTable, 0, 1) + md.tables = make([]mdTable, 0, 4) } md.tables = append(md.tables, mdTable{tab: tab}) @@ -243,18 +280,9 @@ func (md *Metadata) AddTableWithName(tab Table, tabName string) TableID { for i := 0; i < colCount; i++ { col := tab.Column(i) - - // Format column name. - colName := string(col.ColName()) - var sb strings.Builder - sb.Grow(len(tabName) + len(colName) + 1) - sb.WriteString(tabName) - sb.WriteRune('.') - sb.WriteString(colName) - md.cols = append(md.cols, mdColumn{ tabID: tabID, - label: sb.String(), + label: string(col.ColName()), typ: col.DatumType(), }) } diff --git a/pkg/sql/opt/metadata_test.go b/pkg/sql/opt/metadata_test.go index d5794539d630..1b5987ed6025 100644 --- a/pkg/sql/opt/metadata_test.go +++ b/pkg/sql/opt/metadata_test.go @@ -25,7 +25,7 @@ import ( ) func TestMetadataColumns(t *testing.T) { - md := opt.NewMetadata() + var md opt.Metadata // Add standalone column. colID := md.AddColumn("alias", types.Int) @@ -69,7 +69,7 @@ func TestMetadataColumns(t *testing.T) { } func TestMetadataTables(t *testing.T) { - md := opt.NewMetadata() + var md opt.Metadata // Add a table reference to the metadata. a := &testcat.Table{} @@ -94,7 +94,7 @@ func TestMetadataTables(t *testing.T) { } label := md.ColumnLabel(colID) - if label != "a.x" { + if label != "x" { t.Fatalf("unexpected column label: %s", label) } @@ -109,7 +109,7 @@ func TestMetadataTables(t *testing.T) { } label = md.ColumnLabel(otherTabID.ColumnID(0)) - if label != "b.x" { + if label != "x" { t.Fatalf("unexpected column label: %s", label) } } @@ -129,7 +129,7 @@ func TestIndexColumns(t *testing.T) { t.Fatal(err) } - md := opt.NewMetadata() + var md opt.Metadata a := md.AddTable(cat.Table(tree.NewUnqualifiedTableName("a"))) k := int(a.ColumnID(0)) diff --git a/pkg/sql/opt/norm/custom_funcs.go b/pkg/sql/opt/norm/custom_funcs.go index bf9bfea8c449..3ea968d54765 100644 --- a/pkg/sql/opt/norm/custom_funcs.go +++ b/pkg/sql/opt/norm/custom_funcs.go @@ -870,7 +870,7 @@ func (c *CustomFuncs) ExtractRedundantSubclause( panic(fmt.Errorf( "ExtractRedundantSubclause called with non-redundant subclause:\n%s", - memo.MakeNormExprView(c.mem, subclause).FormatString(opt.ExprFmtHideScalars), + memo.MakeNormExprView(c.mem, subclause).FormatString(memo.ExprFmtHideScalars), )) } } diff --git a/pkg/sql/opt/norm/join.go b/pkg/sql/opt/norm/join.go index 6cc133b0f1cc..a23a74a24d1d 100644 --- a/pkg/sql/opt/norm/join.go +++ b/pkg/sql/opt/norm/join.go @@ -154,8 +154,8 @@ func (c *CustomFuncs) Map(filters, src, dst memo.GroupID) memo.GroupID { if !ok { panic(fmt.Errorf( "Map called on src that cannot be mapped to dst. src:\n%s\ndst:\n%s", - memo.MakeNormExprView(c.f.mem, src).FormatString(opt.ExprFmtHideScalars), - memo.MakeNormExprView(c.f.mem, dst).FormatString(opt.ExprFmtHideScalars), + memo.MakeNormExprView(c.f.mem, src).FormatString(memo.ExprFmtHideScalars), + memo.MakeNormExprView(c.f.mem, dst).FormatString(memo.ExprFmtHideScalars), )) } colMap.Set(srcCol, dstCol) diff --git a/pkg/sql/opt/norm/norm_test.go b/pkg/sql/opt/norm/norm_test.go index 0a612655b806..053145689583 100644 --- a/pkg/sql/opt/norm/norm_test.go +++ b/pkg/sql/opt/norm/norm_test.go @@ -42,8 +42,8 @@ import ( // make test PKG=./pkg/sql/opt/norm TESTS="TestNormRules/comp" // ... func TestNormRules(t *testing.T) { - const fmtFlags = opt.ExprFmtHideStats | opt.ExprFmtHideCost | opt.ExprFmtHideRuleProps | - opt.ExprFmtHideQualifications | opt.ExprFmtHideScalars + const fmtFlags = memo.ExprFmtHideStats | memo.ExprFmtHideCost | memo.ExprFmtHideRuleProps | + memo.ExprFmtHideQualifications | memo.ExprFmtHideScalars datadriven.Walk(t, "testdata/rules", func(t *testing.T, path string) { catalog := testcat.New() datadriven.RunTest(t, path, func(d *datadriven.TestData) string { diff --git a/pkg/sql/opt/norm/testdata/rules/agg b/pkg/sql/opt/norm/testdata/rules/agg index aec8eec48c66..8cea0997098c 100644 --- a/pkg/sql/opt/norm/testdata/rules/agg +++ b/pkg/sql/opt/norm/testdata/rules/agg @@ -28,12 +28,12 @@ scalar-group-by │ ├── scan a │ │ └── columns: i:2(int) f:3(float) │ └── projections [outer=(2,3)] - │ └── a.i > a.f [type=bool, outer=(2,3)] + │ └── i > f [type=bool, outer=(2,3)] └── aggregations [outer=(2,9)] ├── min [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── max [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── bool-and [type=bool, outer=(9)] │ └── variable: column9 [type=bool, outer=(9)] └── bool-or [type=bool, outer=(9)] @@ -63,32 +63,32 @@ scalar-group-by │ ├── scan a │ │ └── columns: i:2(int) f:3(float) s:4(string) j:5(jsonb) │ └── projections [outer=(2-5)] - │ └── a.s::BYTES [type=bytes, outer=(4)] + │ └── s::BYTES [type=bytes, outer=(4)] └── aggregations [outer=(2,3,5,13)] ├── count [type=int, outer=(2)] │ └── agg-distinct [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── sum [type=decimal, outer=(2)] │ └── agg-distinct [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── sum-int [type=int, outer=(2)] │ └── agg-distinct [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── avg [type=decimal, outer=(2)] │ └── agg-distinct [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── std-dev [type=float, outer=(3)] │ └── agg-distinct [type=float, outer=(3)] - │ └── variable: a.f [type=float, outer=(3)] + │ └── variable: f [type=float, outer=(3)] ├── variance [type=float, outer=(3)] │ └── agg-distinct [type=float, outer=(3)] - │ └── variable: a.f [type=float, outer=(3)] + │ └── variable: f [type=float, outer=(3)] ├── xor-agg [type=bytes, outer=(13)] │ └── agg-distinct [type=bytes, outer=(13)] │ └── variable: column13 [type=bytes, outer=(13)] ├── array-agg [type=int[], outer=(2)] │ └── agg-distinct [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] └── json-agg [type=jsonb, outer=(5)] └── agg-distinct [type=jsonb, outer=(5)] - └── variable: a.j [type=jsonb, outer=(5)] + └── variable: j [type=jsonb, outer=(5)] diff --git a/pkg/sql/opt/norm/testdata/rules/bool b/pkg/sql/opt/norm/testdata/rules/bool index cc7f63dd8188..036f252b9bbe 100644 --- a/pkg/sql/opt/norm/testdata/rules/bool +++ b/pkg/sql/opt/norm/testdata/rules/bool @@ -73,7 +73,7 @@ project ├── scan a │ └── columns: i:2(int) s:4(string) └── projections [outer=(2,4)] - └── (a.i = 5) AND (a.s < 'foo') [type=bool, outer=(2,4)] + └── (i = 5) AND (s < 'foo') [type=bool, outer=(2,4)] # -------------------------------------------------- # SimplifyAnd @@ -110,7 +110,7 @@ project │ ├── columns: k:1(int!null) │ └── key: (1) └── projections [outer=(1)] - └── a.k = 1 [type=bool, outer=(1)] + └── k = 1 [type=bool, outer=(1)] opt SELECT k=1 AND i=2 AND true AS r FROM a @@ -122,7 +122,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── projections [outer=(1,2)] - └── (a.k = 1) AND (a.i = 2) [type=bool, outer=(1,2)] + └── (k = 1) AND (i = 2) [type=bool, outer=(1,2)] # No conditions left after rule. opt @@ -144,7 +144,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(3,4) └── projections [outer=(1,3,4)] - └── (((a.k > 1) AND (a.k < 5)) AND (a.f = 3.5)) AND (a.s = 'foo') [type=bool, outer=(1,3,4)] + └── (((k > 1) AND (k < 5)) AND (f = 3.5)) AND (s = 'foo') [type=bool, outer=(1,3,4)] # -------------------------------------------------- # SimplifyOr @@ -181,7 +181,7 @@ project │ ├── columns: k:1(int!null) │ └── key: (1) └── projections [outer=(1)] - └── a.k = 1 [type=bool, outer=(1)] + └── k = 1 [type=bool, outer=(1)] opt SELECT * FROM a WHERE k=1 OR i=2 OR false @@ -195,7 +195,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1,2)] - └── (a.k = 1) OR (a.i = 2) [type=bool, outer=(1,2)] + └── (k = 1) OR (i = 2) [type=bool, outer=(1,2)] # No conditions left after rule. opt @@ -220,7 +220,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1-4)] - └── (((a.k = 1) OR (a.i = 2)) OR (a.f = 3.5)) OR (a.s = 'foo') [type=bool, outer=(1-4)] + └── (((k = 1) OR (i = 2)) OR (f = 3.5)) OR (s = 'foo') [type=bool, outer=(1-4)] # -------------------------------------------------- # SimplifyAnd + SimplifyOr @@ -234,7 +234,7 @@ project │ ├── columns: k:1(int!null) │ └── key: (1) └── projections [outer=(1)] - └── (a.k = 1) AND (a.k = 2) [type=bool, outer=(1)] + └── (k = 1) AND (k = 2) [type=bool, outer=(1)] # Use parentheses to make and/or tree right-heavy instead of left-heavy. opt @@ -247,7 +247,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2-4) └── projections [outer=(1-4)] - └── ((((a.k = 1) OR (a.i = 2)) OR (a.f = 3.5)) AND (a.s = 'foo')) AND (a.s != 'bar') [type=bool, outer=(1-4)] + └── ((((k = 1) OR (i = 2)) OR (f = 3.5)) AND (s = 'foo')) AND (s != 'bar') [type=bool, outer=(1-4)] # -------------------------------------------------- # SimplifyFilters @@ -331,8 +331,8 @@ project │ ├── columns: k:1(int!null) │ └── key: (1) └── projections [outer=(1)] - ├── NULL OR (a.k = 1) [type=bool, outer=(1)] - └── NULL AND (a.k = 1) [type=bool, outer=(1)] + ├── NULL OR (k = 1) [type=bool, outer=(1)] + └── NULL AND (k = 1) [type=bool, outer=(1)] # -------------------------------------------------- # NegateComparison @@ -351,12 +351,12 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1-3), constraints=(/1: (/NULL - ]; /2: [/2 - ]; /3: [/1.0 - ]), fd=(2)==(3), (3)==(2)] - ├── a.i != 1 [type=bool, outer=(2), constraints=(/2: (/NULL - /0] [/2 - ]; tight)] - ├── a.f = a.i [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] - ├── a.i <= a.k [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - ├── a.i < a.f [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] - ├── a.f >= 1.0 [type=bool, outer=(3), constraints=(/3: [/1.0 - ]; tight)] - └── a.i > 1 [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] + ├── i != 1 [type=bool, outer=(2), constraints=(/2: (/NULL - /0] [/2 - ]; tight)] + ├── f = i [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + ├── i <= k [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] + ├── i < f [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + ├── f >= 1.0 [type=bool, outer=(3), constraints=(/3: [/1.0 - ]; tight)] + └── i > 1 [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] # IN and IS comparisons. opt @@ -373,10 +373,10 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2-4), constraints=(/3: [/3.0 - /3.0] [/4.0 - /4.0]; /4: [/NULL - /NULL]), fd=()-->(4)] - ├── a.i NOT IN (1, 2) [type=bool, outer=(2)] - ├── a.f IN (3.0, 4.0) [type=bool, outer=(3), constraints=(/3: [/3.0 - /3.0] [/4.0 - /4.0]; tight)] - ├── a.f IS NOT NULL [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] - └── a.s IS NULL [type=bool, outer=(4), constraints=(/4: [/NULL - /NULL]; tight)] + ├── i NOT IN (1, 2) [type=bool, outer=(2)] + ├── f IN (3.0, 4.0) [type=bool, outer=(3), constraints=(/3: [/3.0 - /3.0] [/4.0 - /4.0]; tight)] + ├── f IS NOT NULL [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] + └── s IS NULL [type=bool, outer=(4), constraints=(/4: [/NULL - /NULL]; tight)] # Like comparisons. opt @@ -393,10 +393,10 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(4)] - ├── a.s NOT LIKE 'foo' [type=bool, outer=(4)] - ├── a.s LIKE 'foo' [type=bool, outer=(4)] - ├── a.s NOT ILIKE 'foo' [type=bool, outer=(4)] - └── a.s ILIKE 'foo' [type=bool, outer=(4)] + ├── s NOT LIKE 'foo' [type=bool, outer=(4)] + ├── s LIKE 'foo' [type=bool, outer=(4)] + ├── s NOT ILIKE 'foo' [type=bool, outer=(4)] + └── s ILIKE 'foo' [type=bool, outer=(4)] # SimilarTo comparisons. opt @@ -411,8 +411,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(4)] - ├── a.s NOT SIMILAR TO 'foo' [type=bool, outer=(4)] - └── a.s SIMILAR TO 'foo' [type=bool, outer=(4)] + ├── s NOT SIMILAR TO 'foo' [type=bool, outer=(4)] + └── s SIMILAR TO 'foo' [type=bool, outer=(4)] # RegMatch comparisons. opt @@ -427,10 +427,10 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(4)] - ├── a.s !~ 'foo' [type=bool, outer=(4)] - ├── a.s ~ 'foo' [type=bool, outer=(4)] - ├── a.s !~* 'foo' [type=bool, outer=(4)] - └── a.s ~* 'foo' [type=bool, outer=(4)] + ├── s !~ 'foo' [type=bool, outer=(4)] + ├── s ~ 'foo' [type=bool, outer=(4)] + ├── s !~* 'foo' [type=bool, outer=(4)] + └── s ~* 'foo' [type=bool, outer=(4)] opt SELECT * FROM a WHERE @@ -448,11 +448,11 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(5)] - ├── NOT ('[1, 2]' @> a.j) [type=bool, outer=(5)] - ├── NOT ('[3, 4]' @> a.j) [type=bool, outer=(5)] - ├── NOT (a.j ? 'foo') [type=bool, outer=(5)] - ├── NOT (a.j ?| ARRAY['foo']) [type=bool, outer=(5)] - └── NOT (a.j ?& ARRAY['foo']) [type=bool, outer=(5)] + ├── NOT ('[1, 2]' @> j) [type=bool, outer=(5)] + ├── NOT ('[3, 4]' @> j) [type=bool, outer=(5)] + ├── NOT (j ? 'foo') [type=bool, outer=(5)] + ├── NOT (j ?| ARRAY['foo']) [type=bool, outer=(5)] + └── NOT (j ?& ARRAY['foo']) [type=bool, outer=(5)] # -------------------------------------------------- # EliminateNot @@ -469,7 +469,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(5), constraints=(/5: [/'{}' - /'{}']; tight), fd=()-->(5)] - └── a.j = '{}' [type=bool, outer=(5), constraints=(/5: [/'{}' - /'{}']; tight)] + └── j = '{}' [type=bool, outer=(5), constraints=(/5: [/'{}' - /'{}']; tight)] # -------------------------------------------------- # NegateAnd + NegateComparison @@ -486,7 +486,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1-3)] - └── (a.k < a.i) OR (a.i >= a.f) [type=bool, outer=(1-3)] + └── (k < i) OR (i >= f) [type=bool, outer=(1-3)] opt SELECT * FROM a WHERE NOT (k >= i AND i < f AND (i > 5 AND i < 10 AND f > 1)) @@ -500,7 +500,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1-3)] - └── ((((a.k < a.i) OR (a.i >= a.f)) OR (a.i <= 5)) OR (a.i >= 10)) OR (a.f <= 1.0) [type=bool, outer=(1-3)] + └── ((((k < i) OR (i >= f)) OR (i <= 5)) OR (i >= 10)) OR (f <= 1.0) [type=bool, outer=(1-3)] # -------------------------------------------------- @@ -518,9 +518,9 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1-3), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /3: (/NULL - ])] - ├── a.k < a.i [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - ├── a.i >= a.f [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] - └── a.f <= (a.k + a.i) [type=bool, outer=(1-3), constraints=(/3: (/NULL - ])] + ├── k < i [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] + ├── i >= f [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + └── f <= (k + i) [type=bool, outer=(1-3), constraints=(/3: (/NULL - ])] opt SELECT * FROM a WHERE NOT (k >= i OR i < f OR (i > 10 OR i < 5 OR f > 1)) @@ -534,11 +534,11 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1-3), constraints=(/1: (/NULL - ]; /2: [/5 - /10]; /3: (/NULL - /1.0])] - ├── a.k < a.i [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - ├── a.i >= a.f [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] - ├── a.i <= 10 [type=bool, outer=(2), constraints=(/2: (/NULL - /10]; tight)] - ├── a.i >= 5 [type=bool, outer=(2), constraints=(/2: [/5 - ]; tight)] - └── a.f <= 1.0 [type=bool, outer=(3), constraints=(/3: (/NULL - /1.0]; tight)] + ├── k < i [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] + ├── i >= f [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + ├── i <= 10 [type=bool, outer=(2), constraints=(/2: (/NULL - /10]; tight)] + ├── i >= 5 [type=bool, outer=(2), constraints=(/2: [/5 - ]; tight)] + └── f <= 1.0 [type=bool, outer=(3), constraints=(/3: (/NULL - /1.0]; tight)] # -------------------------------------------------- # NegateAnd + NegateOr + NegateComparison @@ -555,7 +555,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1-3)] - └── ((a.k < a.i) AND (a.i >= a.f)) OR ((a.i <= 5) AND (a.f <= 1.0)) [type=bool, outer=(1-3)] + └── ((k < i) AND (i >= f)) OR ((i <= 5) AND (f <= 1.0)) [type=bool, outer=(1-3)] opt SELECT * FROM a WHERE NOT ((k >= i AND i < f) OR (i > 5 AND f > 1)) @@ -569,8 +569,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1-3)] - ├── (a.k < a.i) OR (a.i >= a.f) [type=bool, outer=(1-3)] - └── (a.i <= 5) OR (a.f <= 1.0) [type=bool, outer=(2,3)] + ├── (k < i) OR (i >= f) [type=bool, outer=(1-3)] + └── (i <= 5) OR (f <= 1.0) [type=bool, outer=(2,3)] # -------------------------------------------------- # ExtractRedundantClause + ExtractRedundantSubclause @@ -583,7 +583,7 @@ project ├── scan c │ └── columns: b:2(bool) └── projections [outer=(2)] - └── variable: c.b [type=bool, outer=(2)] + └── variable: b [type=bool, outer=(2)] opt SELECT a OR (a AND b) OR (a AND c) FROM c @@ -593,7 +593,7 @@ project ├── scan c │ └── columns: a:1(bool) └── projections [outer=(1)] - └── variable: c.a [type=bool, outer=(1)] + └── variable: a [type=bool, outer=(1)] opt SELECT (a AND b) OR a OR (a AND c) FROM c @@ -603,7 +603,7 @@ project ├── scan c │ └── columns: a:1(bool) └── projections [outer=(1)] - └── variable: c.a [type=bool, outer=(1)] + └── variable: a [type=bool, outer=(1)] opt SELECT (a AND b) OR (b AND a) FROM c @@ -613,7 +613,7 @@ project ├── scan c │ └── columns: a:1(bool) b:2(bool) └── projections [outer=(1,2)] - └── c.a AND c.b [type=bool, outer=(1,2)] + └── a AND b [type=bool, outer=(1,2)] opt SELECT (a AND b) OR (c AND a) FROM c @@ -623,7 +623,7 @@ project ├── scan c │ └── columns: a:1(bool) b:2(bool) c:3(bool) └── projections [outer=(1-3)] - └── c.a AND (c.b OR c.c) [type=bool, outer=(1-3)] + └── a AND (b OR c) [type=bool, outer=(1-3)] opt SELECT * FROM c WHERE (a AND b) OR (a AND b AND c) OR (b AND a) @@ -634,8 +634,8 @@ select ├── scan c │ └── columns: a:1(bool) b:2(bool) c:3(bool) d:4(bool) e:5(bool) └── filters [type=bool, outer=(1,2), constraints=(/1: [/true - /true]; /2: [/true - /true]; tight), fd=()-->(1,2)] - ├── variable: c.a [type=bool, outer=(1), constraints=(/1: [/true - /true]; tight)] - └── variable: c.b [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight)] + ├── variable: a [type=bool, outer=(1), constraints=(/1: [/true - /true]; tight)] + └── variable: b [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight)] opt SELECT * FROM c WHERE (b AND a AND c) OR (d AND e AND a) @@ -646,8 +646,8 @@ select ├── scan c │ └── columns: a:1(bool) b:2(bool) c:3(bool) d:4(bool) e:5(bool) └── filters [type=bool, outer=(1-5), constraints=(/1: [/true - /true]), fd=()-->(1)] - ├── variable: c.a [type=bool, outer=(1), constraints=(/1: [/true - /true]; tight)] - └── (c.b AND c.c) OR (c.d AND c.e) [type=bool, outer=(2-5)] + ├── variable: a [type=bool, outer=(1), constraints=(/1: [/true - /true]; tight)] + └── (b AND c) OR (d AND e) [type=bool, outer=(2-5)] opt SELECT * FROM a WHERE ((k > 5) AND (i < 2) AND (i > 0)) OR ((k > 5) AND (i < 2) AND (s = 'foo')) @@ -662,8 +662,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2,4), constraints=(/2: (/NULL - /1])] - ├── a.i < 2 [type=bool, outer=(2), constraints=(/2: (/NULL - /1]; tight)] - └── (a.i > 0) OR (a.s = 'foo') [type=bool, outer=(2,4)] + ├── i < 2 [type=bool, outer=(2), constraints=(/2: (/NULL - /1]; tight)] + └── (i > 0) OR (s = 'foo') [type=bool, outer=(2,4)] opt SELECT * FROM a WHERE (k > 5) OR ((k > 5) AND (i < 2) AND (s = 'foo')) @@ -694,7 +694,7 @@ project │ ├── columns: k:1(int!null) │ └── key: (1) └── projections [outer=(1)] - └── NULL AND ((a.k = 2) OR (a.k = 1)) [type=bool, outer=(1)] + └── NULL AND ((k = 2) OR (k = 1)) [type=bool, outer=(1)] # Check that we don't match non-redundant cases. opt @@ -705,7 +705,7 @@ project ├── scan c │ └── columns: a:1(bool) b:2(bool) └── projections [outer=(1,2)] - └── (c.a OR c.b) OR c.b [type=bool, outer=(1,2)] + └── (a OR b) OR b [type=bool, outer=(1,2)] opt SELECT (a AND b) OR (a OR c) FROM c @@ -715,7 +715,7 @@ project ├── scan c │ └── columns: a:1(bool) b:2(bool) c:3(bool) └── projections [outer=(1-3)] - └── ((c.a AND c.b) OR c.a) OR c.c [type=bool, outer=(1-3)] + └── ((a AND b) OR a) OR c [type=bool, outer=(1-3)] opt SELECT (a AND b) OR (NOT a AND c) FROM c @@ -725,4 +725,4 @@ project ├── scan c │ └── columns: a:1(bool) b:2(bool) c:3(bool) └── projections [outer=(1-3)] - └── (c.a AND c.b) OR ((NOT c.a) AND c.c) [type=bool, outer=(1-3)] + └── (a AND b) OR ((NOT a) AND c) [type=bool, outer=(1-3)] diff --git a/pkg/sql/opt/norm/testdata/rules/combo b/pkg/sql/opt/norm/testdata/rules/combo index f197ab304359..7bad14ce0011 100644 --- a/pkg/sql/opt/norm/testdata/rules/combo +++ b/pkg/sql/opt/norm/testdata/rules/combo @@ -56,7 +56,7 @@ Initial expression │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── (a.k = xy.x) AND ((a.i + 1) = 10) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── (k = x) AND ((i + 1) = 10) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ================================================================================ NormalizeCmpPlusConst Cost: 2910.74 @@ -77,9 +77,9 @@ NormalizeCmpPlusConst │ ├── key: (6) │ └── fd: (6)-->(7) - └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - - └── (a.k = xy.x) AND ((a.i + 1) = 10) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + - └── (k = x) AND ((i + 1) = 10) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] - + └── (a.k = xy.x) AND (a.i = (10 - 1)) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] + + └── (k = x) AND (i = (10 - 1)) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] ================================================================================ SimplifyFilters Cost: 2176.67 @@ -101,10 +101,10 @@ SimplifyFilters │ ├── key: (6) │ └── fd: (6)-->(7) - └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] - - └── (a.k = xy.x) AND (a.i = (10 - 1)) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] + - └── (k = x) AND (i = (10 - 1)) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] + └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - + ├── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - + └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + + └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ================================================================================ PushFilterIntoJoinLeft Cost: 2178.33 @@ -127,16 +127,16 @@ PushFilterIntoJoinLeft + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - + │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) │ └── fd: (6)-->(7) - └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - - ├── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - - └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + - ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + - └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - + └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ================================================================================ PruneJoinLeftCols Cost: 2181.67 @@ -160,7 +160,7 @@ PruneJoinLeftCols - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + - │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ ├── fd: (1)-->(2,4) + │ └── select + │ ├── columns: k:1(int!null) i:2(int!null) f:3(float) s:4(string) j:5(jsonb) @@ -171,13 +171,13 @@ PruneJoinLeftCols + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - + │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ================================================================================ PruneSelectCols Cost: 2161.67 @@ -205,13 +205,13 @@ PruneSelectCols - │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ │ └── fd: (1)-->(2,4) │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ================================================================================ EliminateProject Cost: 2158.33 @@ -236,19 +236,19 @@ EliminateProject - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2,4) - │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + - │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ ├── scan a + │ │ ├── columns: k:1(int!null) i:2(int) s:4(string) + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2,4) + │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - + │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ================================================================================ PruneJoinRightCols Cost: 2148.33 @@ -270,7 +270,7 @@ PruneJoinRightCols │ │ ├── key: (1) │ │ └── fd: (1)-->(2,4) │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ├── scan xy - │ ├── columns: x:6(int!null) y:7(int) - │ ├── key: (6) @@ -278,7 +278,7 @@ PruneJoinRightCols + │ ├── columns: x:6(int!null) + │ └── key: (6) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] -------------------------------------------------------------------------------- GenerateIndexScans (higher cost) -------------------------------------------------------------------------------- @@ -303,12 +303,12 @@ GenerateIndexScans (higher cost) + │ │ ├── key: (1) + │ │ └── fd: (1)-->(4) │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ├── scan xy │ ├── columns: x:6(int!null) │ └── key: (6) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] -------------------------------------------------------------------------------- GenerateIndexScans (no changes) -------------------------------------------------------------------------------- @@ -334,12 +334,12 @@ CommuteJoin │ │ ├── key: (1) │ │ └── fd: (1)-->(2,4) │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - ├── scan xy - │ ├── columns: x:6(int!null) - │ └── key: (6) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ================================================================================ GenerateMergeJoins Cost: 2140.00 @@ -366,9 +366,9 @@ GenerateMergeJoins + │ │ ├── fd: (1)-->(2,4) + │ │ └── ordering: +1 │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + - └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── scan xy + │ ├── columns: x:6(int!null) + │ ├── key: (6) @@ -377,7 +377,7 @@ GenerateMergeJoins + ├── left ordering: +1 + ├── right ordering: +6 + └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - + └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] -------------------------------------------------------------------------------- GenerateLookupJoin (higher cost) -------------------------------------------------------------------------------- @@ -401,7 +401,7 @@ GenerateLookupJoin (higher cost) - │ │ └── ordering: +1 + │ │ └── fd: (1)-->(2,4) │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - ├── scan xy - │ ├── columns: x:6(int!null) - │ ├── key: (6) @@ -410,9 +410,9 @@ GenerateLookupJoin (higher cost) - ├── left ordering: +1 - ├── right ordering: +6 - └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + - └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - + └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] -------------------------------------------------------------------------------- CommuteJoin (no changes) -------------------------------------------------------------------------------- @@ -441,7 +441,7 @@ GenerateLookupJoinWithFilter (higher cost) - │ │ ├── fd: (1)-->(2,4) - │ │ └── ordering: +1 - │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + - │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ├── scan xy │ ├── columns: x:6(int!null) - │ ├── key: (6) @@ -450,11 +450,11 @@ GenerateLookupJoinWithFilter (higher cost) - ├── left ordering: +1 - ├── right ordering: +6 - └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + - └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── key: (6) + └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - + ├── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - + └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + + └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ================================================================================ Final best expression Cost: 2140.00 @@ -476,7 +476,7 @@ Final best expression │ │ ├── fd: (1)-->(2,4) │ │ └── ordering: +1 │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ └── a.i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ └── i = (10 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] ├── scan xy │ ├── columns: x:6(int!null) │ ├── key: (6) @@ -485,7 +485,7 @@ Final best expression ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Exploration patterns with varying costs. optsteps @@ -508,7 +508,7 @@ Initial expression │ ├── key: (1) │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] - └── (a.s = 'foo') AND (a.f > 100.0) [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] + └── (s = 'foo') AND (f > 100.0) [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] ================================================================================ SimplifyFilters Cost: 1110.00 @@ -526,9 +526,9 @@ SimplifyFilters │ ├── key: (1) │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] - - └── (a.s = 'foo') AND (a.f > 100.0) [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] - + ├── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] - + └── a.f > 100.0 [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + - └── (s = 'foo') AND (f > 100.0) [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight)] + + ├── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + + └── f > 100.0 [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] ================================================================================ PruneSelectCols Cost: 1090.00 @@ -550,8 +550,8 @@ PruneSelectCols - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── fd: (1)-->(3,4), (3,4)~~>(1) └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] - ├── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] - └── a.f > 100.0 [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + ├── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + └── f > 100.0 [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] ================================================================================ GenerateIndexScans Cost: 1080.00 @@ -570,8 +570,8 @@ GenerateIndexScans │ ├── key: (1) │ └── fd: (1)-->(3,4), (3,4)~~>(1) └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] - ├── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] - └── a.f > 100.0 [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + ├── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + └── f > 100.0 [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] ================================================================================ ConstrainScan Cost: 1.55 @@ -591,8 +591,8 @@ ConstrainScan │ ├── key: (1) - │ └── fd: (1)-->(3,4), (3,4)~~>(1) - └── filters [type=bool, outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] - - ├── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] - - └── a.f > 100.0 [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + - ├── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + - └── f > 100.0 [type=bool, outer=(3), constraints=(/3: [/100.00000000000001 - ]; tight)] + │ └── fd: ()-->(4), (1)-->(3), (3)-->(1) + └── filters [type=bool] + ├── true [type=bool] @@ -662,7 +662,7 @@ Initial expression │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] ================================================================================ HoistSelectExists Cost: 2182.53 @@ -688,7 +688,7 @@ HoistSelectExists - │ ├── key: (6) - │ └── fd: (6)-->(7) - └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - - └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + - └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── scan a + │ │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) @@ -704,7 +704,7 @@ HoistSelectExists + │ │ │ ├── key: (6) + │ │ │ └── fd: (6)-->(7) + │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - + │ │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + + │ │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ └── true [type=bool] + └── filters [type=bool] ================================================================================ @@ -737,11 +737,11 @@ TryDecorrelateSelect - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) - │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - - │ │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + - │ │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - │ └── true [type=bool] + │ │ └── fd: (6)-->(7) + │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - + │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + + │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] └── filters [type=bool] ================================================================================ DecorrelateJoin @@ -767,7 +767,7 @@ DecorrelateJoin │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] └── filters [type=bool] ================================================================================ EliminateEmptyAnd @@ -790,7 +790,7 @@ EliminateEmptyAnd │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - └── filters [type=bool] + └── true [type=bool] ================================================================================ @@ -816,7 +816,7 @@ EliminateSelect - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) - │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - - │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + - │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - └── true [type=bool] + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan xy @@ -824,7 +824,7 @@ EliminateSelect + │ ├── key: (6) + │ └── fd: (6)-->(7) + └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - + └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + + └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] -------------------------------------------------------------------------------- GenerateIndexScans (higher cost) -------------------------------------------------------------------------------- @@ -847,7 +847,7 @@ GenerateIndexScans (higher cost) │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] -------------------------------------------------------------------------------- GenerateIndexScans (no changes) -------------------------------------------------------------------------------- @@ -871,7 +871,7 @@ Final best expression │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Decorrelation pattern using ANY function. optsteps @@ -896,7 +896,7 @@ Initial expression │ ├── columns: k:3(int!null) i:4(int) f:5(float) s:6(string) j:7(jsonb) │ ├── key: (3) │ └── fd: (3)-->(4-7), (5,6)~~>(3,4,7) - └── variable: xy.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] ================================================================================ PruneScanCols Cost: 2130.00 @@ -918,7 +918,7 @@ PruneScanCols - │ └── fd: (3)-->(4-7), (5,6)~~>(3,4,7) + │ ├── columns: k:3(int!null) + │ └── key: (3) - └── variable: xy.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] ================================================================================ EliminateProject Cost: 2120.00 @@ -939,7 +939,7 @@ EliminateProject - │ ├── columns: k:3(int!null) - │ └── key: (3) + │ └── key: (3) - └── variable: xy.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] ================================================================================ PruneScanCols Cost: 2110.00 @@ -957,7 +957,7 @@ PruneScanCols ├── scan a │ ├── columns: k:3(int!null) │ └── key: (3) - └── variable: xy.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] ================================================================================ HoistProjectSubquery Cost: 2152.55 @@ -972,7 +972,7 @@ HoistProjectSubquery - ├── scan a - │ ├── columns: k:3(int!null) - │ └── key: (3) - - └── variable: xy.x [type=int, outer=(1)] + - └── variable: x [type=int, outer=(1)] + ├── inner-join-apply + │ ├── columns: x:1(int!null) case:11(bool) + │ ├── key: (1) @@ -1002,14 +1002,14 @@ HoistProjectSubquery + │ │ │ │ │ ├── scan a + │ │ │ │ │ │ ├── columns: k:3(int!null) + │ │ │ │ │ │ └── key: (3) - + │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + + │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] + │ │ │ │ └── projections [outer=(3)] - + │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + + │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ │ └── aggregations [outer=(9)] + │ │ │ └── bool-or [type=bool, outer=(9)] + │ │ │ └── variable: notnull [type=bool, outer=(9)] + │ │ └── projections [outer=(1,10)] - + │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + + │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + │ └── true [type=bool] + └── projections [outer=(11)] + └── variable: case [type=bool, outer=(11)] @@ -1048,16 +1048,16 @@ EnsureSelectFilters │ │ │ │ │ ├── scan a │ │ │ │ │ │ ├── columns: k:3(int!null) │ │ │ │ │ │ └── key: (3) - - │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + - │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] + │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - + │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + + │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] │ │ │ │ └── projections [outer=(3)] - │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] │ │ │ └── aggregations [outer=(9)] │ │ │ └── bool-or [type=bool, outer=(9)] │ │ │ └── variable: notnull [type=bool, outer=(9)] │ │ └── projections [outer=(1,10)] - │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] │ └── true [type=bool] └── projections [outer=(11)] └── variable: case [type=bool, outer=(11)] @@ -1116,9 +1116,9 @@ TryDecorrelateProject │ │ │ │ │ │ ├── columns: k:3(int!null) - │ │ │ │ │ │ └── key: (3) - │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - - │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + - │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] - │ │ │ │ └── projections [outer=(3)] - - │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + - │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] - │ │ │ └── aggregations [outer=(9)] - │ │ │ └── bool-or [type=bool, outer=(9)] - │ │ │ └── variable: notnull [type=bool, outer=(9)] @@ -1128,15 +1128,15 @@ TryDecorrelateProject + │ │ │ │ │ │ │ ├── columns: k:3(int!null) + │ │ │ │ │ │ │ └── key: (3) + │ │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - + │ │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + + │ │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] + │ │ │ │ │ └── projections [outer=(3)] - + │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + + │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ │ │ └── aggregations [outer=(9)] + │ │ │ │ └── bool-or [type=bool, outer=(9)] + │ │ │ │ └── variable: notnull [type=bool, outer=(9)] + │ │ │ └── true [type=bool] │ │ └── projections [outer=(1,10)] - │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] │ └── true [type=bool] └── projections [outer=(11)] └── variable: case [type=bool, outer=(11)] @@ -1192,25 +1192,25 @@ TryDecorrelateScalarGroupBy │ │ │ │ │ │ │ ├── columns: k:3(int!null) - │ │ │ │ │ │ │ └── key: (3) - │ │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - - │ │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + - │ │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] - │ │ │ │ │ └── projections [outer=(3)] - - │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + - │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ │ │ │ │ │ ├── outer: (1) + │ │ │ │ │ │ │ ├── key: (3) + │ │ │ │ │ │ │ ├── scan a + │ │ │ │ │ │ │ │ ├── columns: k:3(int!null) + │ │ │ │ │ │ │ │ └── key: (3) + │ │ │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - + │ │ │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + + │ │ │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] + │ │ │ │ │ │ └── projections [outer=(3)] - + │ │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + + │ │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ │ │ │ └── true [type=bool] │ │ │ │ └── aggregations [outer=(9)] │ │ │ │ └── bool-or [type=bool, outer=(9)] │ │ │ │ └── variable: notnull [type=bool, outer=(9)] │ │ │ └── true [type=bool] │ │ └── projections [outer=(1,10)] - │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] │ └── true [type=bool] └── projections [outer=(11)] └── variable: case [type=bool, outer=(11)] @@ -1254,9 +1254,9 @@ TryDecorrelateProjectSelect - │ │ │ │ │ │ │ │ ├── columns: k:3(int!null) - │ │ │ │ │ │ │ │ └── key: (3) - │ │ │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - - │ │ │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + - │ │ │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] - │ │ │ │ │ │ └── projections [outer=(3)] - - │ │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + - │ │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] - │ │ │ │ │ └── true [type=bool] + │ │ │ │ │ └── left-join-apply + │ │ │ │ │ ├── columns: x:1(int!null) k:3(int) notnull:9(bool) @@ -1273,15 +1273,15 @@ TryDecorrelateProjectSelect + │ │ │ │ │ │ │ ├── columns: k:3(int!null) + │ │ │ │ │ │ │ └── key: (3) + │ │ │ │ │ │ └── projections [outer=(3)] - + │ │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + + │ │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - + │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + + │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] │ │ │ │ └── aggregations [outer=(9)] │ │ │ │ └── bool-or [type=bool, outer=(9)] │ │ │ │ └── variable: notnull [type=bool, outer=(9)] │ │ │ └── true [type=bool] │ │ └── projections [outer=(1,10)] - │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] │ └── true [type=bool] └── projections [outer=(11)] └── variable: case [type=bool, outer=(11)] @@ -1327,15 +1327,15 @@ DecorrelateJoin │ │ │ │ │ │ │ ├── columns: k:3(int!null) │ │ │ │ │ │ │ └── key: (3) │ │ │ │ │ │ └── projections [outer=(3)] - │ │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + │ │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] │ │ │ │ └── aggregations [outer=(9)] │ │ │ │ └── bool-or [type=bool, outer=(9)] │ │ │ │ └── variable: notnull [type=bool, outer=(9)] │ │ │ └── true [type=bool] │ │ └── projections [outer=(1,10)] - │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] │ └── true [type=bool] └── projections [outer=(11)] └── variable: case [type=bool, outer=(11)] @@ -1379,9 +1379,9 @@ EliminateGroupByProject - │ │ │ │ │ │ │ ├── columns: k:3(int!null) - │ │ │ │ │ │ │ └── key: (3) - │ │ │ │ │ │ └── projections [outer=(3)] - - │ │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + - │ │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] - │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - - │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + - │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] + │ │ │ │ ├── left-join + │ │ │ │ │ ├── columns: x:1(int!null) k:3(int) notnull:9(bool) + │ │ │ │ │ ├── key: (1,3) @@ -1397,15 +1397,15 @@ EliminateGroupByProject + │ │ │ │ │ │ │ ├── columns: k:3(int!null) + │ │ │ │ │ │ │ └── key: (3) + │ │ │ │ │ │ └── projections [outer=(3)] - + │ │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + + │ │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - + │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + + │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] │ │ │ │ └── aggregations [outer=(9)] │ │ │ │ └── bool-or [type=bool, outer=(9)] │ │ │ │ └── variable: notnull [type=bool, outer=(9)] │ │ │ └── true [type=bool] │ │ └── projections [outer=(1,10)] - │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] │ └── true [type=bool] └── projections [outer=(11)] └── variable: case [type=bool, outer=(11)] @@ -1459,9 +1459,9 @@ EliminateSelect - │ │ │ │ │ │ │ ├── columns: k:3(int!null) - │ │ │ │ │ │ │ └── key: (3) - │ │ │ │ │ │ └── projections [outer=(3)] - - │ │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + - │ │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] - │ │ │ │ │ └── filters [type=bool, outer=(1,3)] - - │ │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + - │ │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] - │ │ │ │ └── aggregations [outer=(9)] - │ │ │ │ └── bool-or [type=bool, outer=(9)] - │ │ │ │ └── variable: notnull [type=bool, outer=(9)] @@ -1470,14 +1470,14 @@ EliminateSelect + │ │ │ │ │ │ ├── columns: k:3(int!null) + │ │ │ │ │ │ └── key: (3) + │ │ │ │ │ └── projections [outer=(3)] - + │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + + │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ │ │ └── filters [type=bool, outer=(1,3)] - + │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + + │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] + │ │ │ └── aggregations [outer=(9)] + │ │ │ └── bool-or [type=bool, outer=(9)] + │ │ │ └── variable: notnull [type=bool, outer=(9)] │ │ └── projections [outer=(1,10)] - │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] │ └── true [type=bool] └── projections [outer=(11)] └── variable: case [type=bool, outer=(11)] @@ -1531,27 +1531,27 @@ EliminateSelect - │ │ │ │ │ │ ├── columns: k:3(int!null) - │ │ │ │ │ │ └── key: (3) - │ │ │ │ │ └── projections [outer=(3)] - - │ │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + - │ │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] - │ │ │ │ └── filters [type=bool, outer=(1,3)] - - │ │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + - │ │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] - │ │ │ └── aggregations [outer=(9)] - │ │ │ └── bool-or [type=bool, outer=(9)] - │ │ │ └── variable: notnull [type=bool, outer=(9)] - │ │ └── projections [outer=(1,10)] - - │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + - │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] - │ └── true [type=bool] + │ │ │ │ ├── scan a + │ │ │ │ │ ├── columns: k:3(int!null) + │ │ │ │ │ └── key: (3) + │ │ │ │ └── projections [outer=(3)] - + │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + + │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ │ └── filters [type=bool, outer=(1,3)] - + │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + + │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] + │ │ └── aggregations [outer=(9)] + │ │ └── bool-or [type=bool, outer=(9)] + │ │ └── variable: notnull [type=bool, outer=(9)] + │ └── projections [outer=(1,10)] - + │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + + │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] └── projections [outer=(11)] └── variable: case [type=bool, outer=(11)] ================================================================================ @@ -1585,14 +1585,14 @@ PruneProjectCols │ │ │ │ │ ├── columns: k:3(int!null) │ │ │ │ │ └── key: (3) │ │ │ │ └── projections [outer=(3)] - │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] │ │ │ └── filters [type=bool, outer=(1,3)] - │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] │ │ └── aggregations [outer=(9)] │ │ └── bool-or [type=bool, outer=(9)] │ │ └── variable: notnull [type=bool, outer=(9)] │ └── projections [outer=(1,10)] - │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] └── projections [outer=(11)] └── variable: case [type=bool, outer=(11)] ================================================================================ @@ -1638,28 +1638,28 @@ InlineProjectInProject - │ │ │ │ │ ├── columns: k:3(int!null) - │ │ │ │ │ └── key: (3) - │ │ │ │ └── projections [outer=(3)] - - │ │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + - │ │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] - │ │ │ └── filters [type=bool, outer=(1,3)] - - │ │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + - │ │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] - │ │ └── aggregations [outer=(9)] - │ │ └── bool-or [type=bool, outer=(9)] - │ │ └── variable: notnull [type=bool, outer=(9)] - │ └── projections [outer=(1,10)] - - │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + - │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] - └── projections [outer=(11)] - └── variable: case [type=bool, outer=(11)] + │ │ │ ├── scan a + │ │ │ │ ├── columns: k:3(int!null) + │ │ │ │ └── key: (3) + │ │ │ └── projections [outer=(3)] - + │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + + │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ └── filters [type=bool, outer=(1,3)] - + │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + + │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] + │ └── aggregations [outer=(9)] + │ └── bool-or [type=bool, outer=(9)] + │ └── variable: notnull [type=bool, outer=(9)] + └── projections [outer=(1,10)] - + └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + + └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] -------------------------------------------------------------------------------- GenerateIndexScans (no changes) -------------------------------------------------------------------------------- @@ -1690,14 +1690,14 @@ GenerateIndexScans │ │ │ │ ├── columns: k:3(int!null) │ │ │ │ └── key: (3) │ │ │ └── projections [outer=(3)] - │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] │ │ └── filters [type=bool, outer=(1,3)] - │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] │ └── aggregations [outer=(9)] │ └── bool-or [type=bool, outer=(9)] │ └── variable: notnull [type=bool, outer=(9)] └── projections [outer=(1,10)] - └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] -------------------------------------------------------------------------------- CommuteLeftJoin (higher cost) -------------------------------------------------------------------------------- @@ -1724,17 +1724,17 @@ CommuteLeftJoin (higher cost) │ │ │ │ ├── columns: k:3(int!null) │ │ │ │ └── key: (3) │ │ │ └── projections [outer=(3)] - │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] + │ │ ├── scan xy + │ │ │ ├── columns: x:1(int!null) + │ │ │ └── key: (1) │ │ └── filters [type=bool, outer=(1,3)] - │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] │ └── aggregations [outer=(9)] │ └── bool-or [type=bool, outer=(9)] │ └── variable: notnull [type=bool, outer=(9)] └── projections [outer=(1,10)] - └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] -------------------------------------------------------------------------------- GenerateMergeJoins (no changes) -------------------------------------------------------------------------------- @@ -1770,11 +1770,11 @@ Final best expression │ │ │ │ ├── columns: k:3(int!null) │ │ │ │ └── key: (3) │ │ │ └── projections [outer=(3)] - │ │ │ └── a.k IS NOT NULL [type=bool, outer=(3)] + │ │ │ └── k IS NOT NULL [type=bool, outer=(3)] │ │ └── filters [type=bool, outer=(1,3)] - │ │ └── (xy.x = a.k) IS NOT false [type=bool, outer=(1,3)] + │ │ └── (x = k) IS NOT false [type=bool, outer=(1,3)] │ └── aggregations [outer=(9)] │ └── bool-or [type=bool, outer=(9)] │ └── variable: notnull [type=bool, outer=(9)] └── projections [outer=(1,10)] - └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] + └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(1,10)] diff --git a/pkg/sql/opt/norm/testdata/rules/comp b/pkg/sql/opt/norm/testdata/rules/comp index d4461d36f877..59a8e38953ff 100644 --- a/pkg/sql/opt/norm/testdata/rules/comp +++ b/pkg/sql/opt/norm/testdata/rules/comp @@ -29,10 +29,10 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1,2), side-effects, constraints=(/1: (/NULL - ]; /2: (/NULL - ])] - ├── a.k > (a.i + 1) [type=bool, outer=(1,2), constraints=(/1: (/NULL - ])] - ├── a.i >= (a.k - 1) [type=bool, outer=(1,2), constraints=(/2: (/NULL - ])] - ├── a.k < (a.i * a.i) [type=bool, outer=(1,2), constraints=(/1: (/NULL - ])] - └── a.i <= (a.k / 2) [type=bool, outer=(1,2), side-effects, constraints=(/2: (/NULL - ])] + ├── k > (i + 1) [type=bool, outer=(1,2), constraints=(/1: (/NULL - ])] + ├── i >= (k - 1) [type=bool, outer=(1,2), constraints=(/2: (/NULL - ])] + ├── k < (i * i) [type=bool, outer=(1,2), constraints=(/1: (/NULL - ])] + └── i <= (k / 2) [type=bool, outer=(1,2), side-effects, constraints=(/2: (/NULL - ])] # -------------------------------------------------- # CommuteConstInequality @@ -49,10 +49,10 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1,2,4), constraints=(/2: (/NULL - /4]; /4: (/NULL - /'foo'])] - ├── (a.i + a.k) > (length('foo') + 1) [type=bool, outer=(1,2)] - ├── (a.i * 2) >= length('bar') [type=bool, outer=(2)] - ├── a.i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] - └── a.s <= 'foo' [type=bool, outer=(4), constraints=(/4: (/NULL - /'foo']; tight)] + ├── (i + k) > (length('foo') + 1) [type=bool, outer=(1,2)] + ├── (i * 2) >= length('bar') [type=bool, outer=(2)] + ├── i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] + └── s <= 'foo' [type=bool, outer=(4), constraints=(/4: (/NULL - /'foo']; tight)] # Impure function should not be considered constant. opt @@ -68,7 +68,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2), side-effects] - └── random()::INT > (a.i + a.i) [type=bool, outer=(2), side-effects] + └── random()::INT > (i + i) [type=bool, outer=(2), side-effects] # -------------------------------------------------- # NormalizeCmpPlusConst @@ -92,11 +92,11 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ])] - ├── a.i = (2 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - ├── (a.f + a.f) < (5.0 - 2.0) [type=bool, outer=(3)] - ├── a.i <= (length('foo') - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - ├── a.i > ((10 - 2) - 2) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - └── a.i::INTERVAL >= ('02:00:00' - '01:00:00') [type=bool, outer=(2)] + ├── i = (2 - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + ├── (f + f) < (5.0 - 2.0) [type=bool, outer=(3)] + ├── i <= (length('foo') - 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + ├── i > ((10 - 2) - 2) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + └── i::INTERVAL >= ('02:00:00' - '01:00:00') [type=bool, outer=(2)] # Try case that should not match pattern because Minus overload is not defined. opt @@ -111,7 +111,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(4)] - └── (a.s::DATE + '02:00:00') = '2000-01-01 02:00:00+00:00' [type=bool, outer=(4)] + └── (s::DATE + '02:00:00') = '2000-01-01 02:00:00+00:00' [type=bool, outer=(4)] # -------------------------------------------------- # NormalizeCmpMinusConst @@ -135,11 +135,11 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ])] - ├── a.i = (2 + 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - ├── (a.f + a.f) < (5.0 + 2.0) [type=bool, outer=(3)] - ├── a.i <= (length('foo') + 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - ├── a.i > ((10 + 2) + 2) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - └── (a.f + a.i::FLOAT) >= (100.0 + 10.0) [type=bool, outer=(2,3)] + ├── i = (2 + 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + ├── (f + f) < (5.0 + 2.0) [type=bool, outer=(3)] + ├── i <= (length('foo') + 1) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + ├── i > ((10 + 2) + 2) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + └── (f + i::FLOAT) >= (100.0 + 10.0) [type=bool, outer=(2,3)] # Try case that should not match pattern because Plus overload is not defined. opt @@ -154,7 +154,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(4)] - └── (a.s::JSON - 1) = '[1]' [type=bool, outer=(4)] + └── (s::JSON - 1) = '[1]' [type=bool, outer=(4)] # -------------------------------------------------- # NormalizeCmpConstMinus @@ -178,11 +178,11 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ])] - ├── a.i = (1 - 2) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - ├── (a.f + a.f) > (2.0 - 5.0) [type=bool, outer=(3)] - ├── a.i >= (1 - length('foo')) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - ├── a.i > (2 - (2 - 10)) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - └── (a.f + a.i::FLOAT) <= (10.0 - 100.0) [type=bool, outer=(2,3)] + ├── i = (1 - 2) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + ├── (f + f) > (2.0 - 5.0) [type=bool, outer=(3)] + ├── i >= (1 - length('foo')) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + ├── i > (2 - (2 - 10)) [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + └── (f + i::FLOAT) <= (10.0 - 100.0) [type=bool, outer=(2,3)] # Try case that should not match pattern because Minus overload is not defined. opt @@ -197,7 +197,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2)] - └── ('[1, 2]' - a.i) = '[1]' [type=bool, outer=(2)] + └── ('[1, 2]' - i) = '[1]' [type=bool, outer=(2)] # -------------------------------------------------- # NormalizeTupleEquality @@ -214,9 +214,9 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2-4), constraints=(/2: [/1 - /1]; /3: [/3.5 - /3.5]; /4: [/'foo' - /'foo']; tight), fd=()-->(2-4)] - ├── a.i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] - ├── a.f = 3.5 [type=bool, outer=(3), constraints=(/3: [/3.5 - /3.5]; tight)] - └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + ├── i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + ├── f = 3.5 [type=bool, outer=(3), constraints=(/3: [/3.5 - /3.5]; tight)] + └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] # -------------------------------------------------- # NormalizeTupleEquality, FlattenAnd @@ -238,8 +238,8 @@ select │ ├── key: () │ └── fd: ()-->(1-5) └── filters [type=bool, outer=(2,4), constraints=(/2: [/2 - /2]; /4: [/'foo' - /'foo']; tight), fd=()-->(2,4)] - ├── a.i = 2 [type=bool, outer=(2), constraints=(/2: [/2 - /2]; tight)] - └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + ├── i = 2 [type=bool, outer=(2), constraints=(/2: [/2 - /2]; tight)] + └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] # -------------------------------------------------- # FoldNullComparisonLeft, FoldNullComparisonRight @@ -350,8 +350,8 @@ project │ └── fd: (1)-->(2) └── projections [outer=(1,2)] ├── true [type=bool] - ├── a.k IS NOT NULL [type=bool, outer=(1)] - └── a.i IS NOT NULL [type=bool, outer=(2)] + ├── k IS NOT NULL [type=bool, outer=(1)] + └── i IS NOT NULL [type=bool, outer=(2)] # -------------------------------------------------- # CommuteNullIs diff --git a/pkg/sql/opt/norm/testdata/rules/decorrelate b/pkg/sql/opt/norm/testdata/rules/decorrelate index 21bcf7947a76..965fc540a0d0 100644 --- a/pkg/sql/opt/norm/testdata/rules/decorrelate +++ b/pkg/sql/opt/norm/testdata/rules/decorrelate @@ -61,7 +61,7 @@ semi-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] opt SELECT * FROM a WHERE NOT EXISTS(SELECT * FROM xy WHERE x=k) @@ -84,7 +84,7 @@ anti-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # -------------------------------------------------- # TryDecorrelateSelect @@ -104,10 +104,10 @@ semi-join-apply │ ├── columns: column1:6(int) │ ├── outer: (1,2) │ ├── cardinality: [2 - 2] - │ ├── (a.k,) [type=tuple{int}, outer=(1)] - │ └── (a.i,) [type=tuple{int}, outer=(2)] + │ ├── (k,) [type=tuple{int}, outer=(1)] + │ └── (i,) [type=tuple{int}, outer=(2)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── column1 = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── column1 = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] opt SELECT * FROM a WHERE NOT EXISTS(SELECT * FROM (VALUES (k), (i)) WHERE column1=k) @@ -124,10 +124,10 @@ anti-join-apply │ ├── columns: column1:6(int) │ ├── outer: (1,2) │ ├── cardinality: [2 - 2] - │ ├── (a.k,) [type=tuple{int}, outer=(1)] - │ └── (a.i,) [type=tuple{int}, outer=(2)] + │ ├── (k,) [type=tuple{int}, outer=(1)] + │ └── (i,) [type=tuple{int}, outer=(2)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── column1 = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── column1 = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Attempt to decorrelate query by pulling up outer select. But since limit query # cannot be decorrelated, push the outer select back down again (and make sure @@ -165,10 +165,10 @@ semi-join-apply │ │ │ │ ├── key: (6) │ │ │ │ └── fd: (6)-->(7) │ │ │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ │ │ └── xy.y = a.k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ │ │ └── y = k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] │ │ └── const: 1 [type=int] │ └── filters [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] - │ └── xy.y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] + │ └── y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] └── true [type=bool] # Same as previous, but using anti-join. @@ -205,10 +205,10 @@ anti-join-apply │ │ │ │ ├── key: (6) │ │ │ │ └── fd: (6)-->(7) │ │ │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ │ │ └── xy.y = a.k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ │ │ └── y = k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] │ │ └── const: 1 [type=int] │ └── filters [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] - │ └── xy.y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] + │ └── y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] └── true [type=bool] # Decorrelate Select with reference to outer column and no limit. @@ -236,9 +236,9 @@ project │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(7), constraints=(/7: [/101 - ]; tight)] - │ └── xy.y > 100 [type=bool, outer=(7), constraints=(/7: [/101 - ]; tight)] + │ └── y > 100 [type=bool, outer=(7), constraints=(/7: [/101 - ]; tight)] └── filters [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] - └── xy.x = a.i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + └── x = i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] # Decorrelate Select with LeftJoinApply. opt @@ -266,10 +266,10 @@ project │ │ ├── key: () │ │ └── fd: ()-->(6) │ └── filters [type=bool, outer=(6), constraints=(/6: [/101 - ]; tight)] - │ └── xy.x > 100 [type=bool, outer=(6), constraints=(/6: [/101 - ]; tight)] + │ └── x > 100 [type=bool, outer=(6), constraints=(/6: [/101 - ]; tight)] └── filters [type=bool, outer=(1,6), constraints=(/1: [/101 - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── a.k > 100 [type=bool, outer=(1), constraints=(/1: [/101 - ]; tight)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k > 100 [type=bool, outer=(1), constraints=(/1: [/101 - ]; tight)] # Decorrelate with non-apply operator because of multi-level nesting. opt @@ -301,9 +301,9 @@ semi-join-apply │ │ ├── key: () │ │ └── fd: ()-->(8,9) │ └── filters [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] - │ └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # -------------------------------------------------- # TryDecorrelateProject + @@ -349,14 +349,14 @@ project │ │ │ │ │ ├── fd: (6)-->(7) │ │ │ │ │ └── ordering: +6 │ │ │ │ └── filters [type=bool, outer=(7)] - │ │ │ │ └── (xy.y = 5) IS NOT false [type=bool, outer=(7)] + │ │ │ │ └── (y = 5) IS NOT false [type=bool, outer=(7)] │ │ │ └── projections [outer=(6,7)] - │ │ │ └── xy.y IS NOT NULL [type=bool, outer=(7)] + │ │ │ └── y IS NOT NULL [type=bool, outer=(7)] │ │ └── merge-on │ │ ├── left ordering: +1 │ │ ├── right ordering: +6 │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ └── aggregations [outer=(9)] │ └── bool-or [type=bool, outer=(9)] │ └── variable: notnull [type=bool, outer=(9)] @@ -389,9 +389,9 @@ project │ │ ├── key: () │ │ └── fd: ()-->(6,7) │ └── filters [type=bool, outer=(7)] - │ └── (xy.y + 1) > 10 [type=bool, outer=(7)] + │ └── (y + 1) > 10 [type=bool, outer=(7)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Any clause with constant. opt @@ -428,14 +428,14 @@ project │ │ │ │ │ ├── fd: (6)-->(7) │ │ │ │ │ └── ordering: +6 │ │ │ │ └── filters [type=bool, outer=(7)] - │ │ │ │ └── (xy.y = 5) IS NOT false [type=bool, outer=(7)] + │ │ │ │ └── (y = 5) IS NOT false [type=bool, outer=(7)] │ │ │ └── projections [outer=(6,7)] - │ │ │ └── xy.y IS NOT NULL [type=bool, outer=(7)] + │ │ │ └── y IS NOT NULL [type=bool, outer=(7)] │ │ └── merge-on │ │ ├── left ordering: +1 │ │ ├── right ordering: +6 │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ └── aggregations [outer=(9)] │ └── bool-or [type=bool, outer=(9)] │ └── variable: notnull [type=bool, outer=(9)] @@ -473,20 +473,20 @@ project │ │ │ │ ├── fd: (6)-->(7) │ │ │ │ └── ordering: +6 │ │ │ └── projections [outer=(6,7)] - │ │ │ └── xy.y IS NOT NULL [type=bool, outer=(7)] + │ │ │ └── y IS NOT NULL [type=bool, outer=(7)] │ │ └── merge-on │ │ ├── left ordering: +1 │ │ ├── right ordering: +6 │ │ └── filters [type=bool, outer=(1,2,6,7), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ ├── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - │ │ └── (a.i = xy.y) IS NOT false [type=bool, outer=(2,7)] + │ │ ├── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ └── (i = y) IS NOT false [type=bool, outer=(2,7)] │ └── aggregations [outer=(2,9)] │ ├── bool-or [type=bool, outer=(9)] │ │ └── variable: notnull [type=bool, outer=(9)] │ └── const-agg [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] └── projections [outer=(2,10)] - └── CASE WHEN bool_or AND (a.i IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(2,10)] + └── CASE WHEN bool_or AND (i IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(2,10)] # Any clause with more complex expression that must be cached. opt @@ -518,7 +518,7 @@ project │ │ │ │ ├── fd: (1)-->(2) │ │ │ │ └── ordering: +1 │ │ │ └── projections [outer=(1,2), side-effects] - │ │ │ └── (a.i * a.i) / 5 [type=decimal, outer=(2), side-effects] + │ │ │ └── (i * i) / 5 [type=decimal, outer=(2), side-effects] │ │ ├── project │ │ │ ├── columns: notnull:10(bool) x:6(int!null) y:7(int) │ │ │ ├── key: (6) @@ -530,13 +530,13 @@ project │ │ │ │ ├── fd: (6)-->(7) │ │ │ │ └── ordering: +6 │ │ │ └── projections [outer=(6,7)] - │ │ │ └── xy.y IS NOT NULL [type=bool, outer=(7)] + │ │ │ └── y IS NOT NULL [type=bool, outer=(7)] │ │ └── merge-on │ │ ├── left ordering: +1 │ │ ├── right ordering: +6 │ │ └── filters [type=bool, outer=(1,6,7,9), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ ├── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - │ │ └── (scalar = xy.y) IS NOT false [type=bool, outer=(7,9)] + │ │ ├── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ └── (scalar = y) IS NOT false [type=bool, outer=(7,9)] │ └── aggregations [outer=(9,10)] │ ├── bool-or [type=bool, outer=(10)] │ │ └── variable: notnull [type=bool, outer=(10)] @@ -590,12 +590,12 @@ distinct-on │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] - │ │ │ └── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + │ │ │ └── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] │ │ └── true [type=bool] │ └── projections [outer=(1,6,8), side-effects] - │ └── uv.u / 1.1 [type=decimal, outer=(8), side-effects] + │ └── u / 1.1 [type=decimal, outer=(8), side-effects] └── filters [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] - └── xy.x = div [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] + └── x = div [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] # Don't hoist Project operator in right join case. opt @@ -637,11 +637,11 @@ project │ │ │ │ ├── columns: u:8(int!null) │ │ │ │ └── key: (8) │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] - │ │ │ └── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + │ │ │ └── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] │ │ └── projections [outer=(8), side-effects] - │ │ └── uv.u / 1.1 [type=decimal, outer=(8), side-effects] + │ │ └── u / 1.1 [type=decimal, outer=(8), side-effects] │ └── filters [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] - │ └── xy.x = div [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] + │ └── x = div [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] └── true [type=bool] # -------------------------------------------------- @@ -682,11 +682,11 @@ project │ │ │ │ ├── columns: u:8(int!null) │ │ │ │ └── key: (8) │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] - │ │ │ └── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + │ │ │ └── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] │ │ └── projections [outer=(8)] - │ │ └── uv.u + 1 [type=int, outer=(8)] + │ │ └── u + 1 [type=int, outer=(8)] │ └── filters [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] - │ └── xy.x = plus [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] + │ └── x = plus [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] └── true [type=bool] # Don't decorrelate FULL JOIN case. @@ -725,11 +725,11 @@ project │ │ │ │ ├── columns: u:8(int!null) │ │ │ │ └── key: (8) │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] - │ │ │ └── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + │ │ │ └── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] │ │ └── projections [outer=(8)] - │ │ └── uv.u + 1 [type=int, outer=(8)] + │ │ └── u + 1 [type=int, outer=(8)] │ └── filters [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] - │ └── xy.x = plus [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] + │ └── x = plus [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] └── true [type=bool] # -------------------------------------------------- @@ -777,14 +777,14 @@ project │ │ │ │ ├── left ordering: +6 │ │ │ │ ├── right ordering: +8 │ │ │ │ └── filters [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] - │ │ │ │ └── xy.x = uv.u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + │ │ │ │ └── x = u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] │ │ │ └── projections [outer=(6,7,9)] - │ │ │ └── xy.y + uv.v [type=int, outer=(7,9)] + │ │ │ └── y + v [type=int, outer=(7,9)] │ │ └── merge-on │ │ ├── left ordering: +1 │ │ ├── right ordering: +6 │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ └── aggregations [outer=(10)] │ └── sum [type=decimal, outer=(10)] │ └── variable: column10 [type=int, outer=(10)] @@ -828,12 +828,12 @@ semi-join (merge) │ ├── left ordering: +6 │ ├── right ordering: +8 │ └── filters [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] - │ └── xy.x = uv.u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + │ └── x = u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Anti-join as outer. opt @@ -869,12 +869,12 @@ anti-join (merge) │ ├── left ordering: +6 │ ├── right ordering: +8 │ └── filters [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] - │ └── xy.x = uv.u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + │ └── x = u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Right-join as outer. opt @@ -924,15 +924,15 @@ project │ │ │ ├── left ordering: +6 │ │ │ ├── right ordering: +8 │ │ │ └── filters [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] - │ │ │ └── xy.x = uv.u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + │ │ │ └── x = u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] │ │ └── merge-on │ │ ├── left ordering: +1 │ │ ├── right ordering: +6 │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ └── aggregations [outer=(6)] │ └── count [type=int, outer=(6)] - │ └── variable: xy.x [type=int, outer=(6)] + │ └── variable: x [type=int, outer=(6)] └── filters [type=bool, outer=(10), constraints=(/10: [ - /0] [/2 - ]; tight)] └── count IS DISTINCT FROM 1 [type=bool, outer=(10), constraints=(/10: [ - /0] [/2 - ]; tight)] @@ -969,8 +969,8 @@ semi-join-apply │ ├── left ordering: +6 │ ├── right ordering: +8 │ └── filters [type=bool, outer=(1,6,8), constraints=(/1: (/NULL - ]; /6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(1,8), (8)==(1,6), (1)==(6,8)] - │ ├── xy.x = uv.u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] - │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ ├── x = u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + │ └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── true [type=bool] # Can't decorrelate semi-join as inner. @@ -1016,7 +1016,7 @@ semi-join-apply │ │ │ └── fd: (10)-->(11) │ │ └── true [type=bool] │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ └── uv.u = a.k [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ └── uv.u = k [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] └── true [type=bool] # -------------------------------------------------- @@ -1060,8 +1060,8 @@ semi-join-apply │ │ │ ├── key: (7) │ │ │ └── fd: (7)-->(8) │ │ └── filters [type=bool, outer=(2,3,7,8), constraints=(/2: (/NULL - ]; /3: (/NULL - ]; /7: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(7), (7)==(2), (3)==(8), (8)==(3)] - │ │ ├── xy.x = a.k [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - │ │ └── a.i = xy.y [type=bool, outer=(3,8), constraints=(/3: (/NULL - ]; /8: (/NULL - ])] + │ │ ├── x = k [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ └── i = y [type=bool, outer=(3,8), constraints=(/3: (/NULL - ]; /8: (/NULL - ])] │ ├── project │ │ ├── columns: v1:11(int) │ │ ├── outer: (1) @@ -1074,7 +1074,7 @@ semi-join-apply │ │ └── projections [outer=(1)] │ │ └── variable: column1 [type=int, outer=(1)] │ └── filters [type=bool, outer=(7,11), constraints=(/7: (/NULL - ]; /11: (/NULL - ]), fd=(7)==(11), (11)==(7)] - │ └── xy.x = v1 [type=bool, outer=(7,11), constraints=(/7: (/NULL - ]; /11: (/NULL - ])] + │ └── x = v1 [type=bool, outer=(7,11), constraints=(/7: (/NULL - ]; /11: (/NULL - ])] └── true [type=bool] opt @@ -1111,9 +1111,9 @@ project │ │ ├── key: (5) │ │ └── fd: (5)-->(6) │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + │ └── k = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] └── filters [type=bool, outer=(3,6)] - └── uv.u IS DISTINCT FROM a.i [type=bool, outer=(3,6)] + └── u IS DISTINCT FROM i [type=bool, outer=(3,6)] opt SELECT generate_series(1, (SELECT u FROM uv WHERE u=x)) @@ -1140,12 +1140,12 @@ project │ ├── left ordering: +1 │ ├── right ordering: +3 │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ └── uv.u = xy.x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ └── u = x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] ├── zip │ ├── columns: column5:5(int) │ ├── outer: (3) │ ├── side-effects - │ └── generate_series(1, uv.u) [type=int, outer=(3), side-effects] + │ └── generate_series(1, u) [type=int, outer=(3), side-effects] └── true [type=bool] # -------------------------------------------------- @@ -1193,29 +1193,29 @@ group-by │ │ │ │ │ ├── key: (1) │ │ │ │ │ └── fd: (1)-->(2-5) │ │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] - │ │ │ │ └── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + │ │ │ │ └── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] │ │ │ └── true [type=bool] │ │ └── aggregations [outer=(2-5)] │ │ ├── count-rows [type=int] │ │ ├── const-agg [type=int, outer=(2)] - │ │ │ └── variable: a.i [type=int, outer=(2)] + │ │ │ └── variable: i [type=int, outer=(2)] │ │ ├── const-agg [type=float, outer=(3)] - │ │ │ └── variable: a.f [type=float, outer=(3)] + │ │ │ └── variable: f [type=float, outer=(3)] │ │ ├── const-agg [type=string, outer=(4)] - │ │ │ └── variable: a.s [type=string, outer=(4)] + │ │ │ └── variable: s [type=string, outer=(4)] │ │ └── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── filters [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] - │ └── xy.x = cnt [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] + │ └── x = cnt [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] └── aggregations [outer=(2-5)] ├── const-agg [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── const-agg [type=float, outer=(3)] - │ └── variable: a.f [type=float, outer=(3)] + │ └── variable: f [type=float, outer=(3)] ├── const-agg [type=string, outer=(4)] - │ └── variable: a.s [type=string, outer=(4)] + │ └── variable: s [type=string, outer=(4)] └── const-agg [type=jsonb, outer=(5)] - └── variable: a.j [type=jsonb, outer=(5)] + └── variable: j [type=jsonb, outer=(5)] opt SELECT * @@ -1259,31 +1259,31 @@ group-by │ │ │ │ │ ├── key: (1) │ │ │ │ │ └── fd: (1)-->(2-5) │ │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] - │ │ │ │ └── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + │ │ │ │ └── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] │ │ │ └── true [type=bool] │ │ └── aggregations [outer=(2-5,9)] │ │ ├── count [type=int, outer=(9)] │ │ │ └── agg-distinct [type=int, outer=(9)] - │ │ │ └── variable: uv.v [type=int, outer=(9)] + │ │ │ └── variable: v [type=int, outer=(9)] │ │ ├── const-agg [type=int, outer=(2)] - │ │ │ └── variable: a.i [type=int, outer=(2)] + │ │ │ └── variable: i [type=int, outer=(2)] │ │ ├── const-agg [type=float, outer=(3)] - │ │ │ └── variable: a.f [type=float, outer=(3)] + │ │ │ └── variable: f [type=float, outer=(3)] │ │ ├── const-agg [type=string, outer=(4)] - │ │ │ └── variable: a.s [type=string, outer=(4)] + │ │ │ └── variable: s [type=string, outer=(4)] │ │ └── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── filters [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] - │ └── xy.x = cnt [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] + │ └── x = cnt [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] └── aggregations [outer=(2-5)] ├── const-agg [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── const-agg [type=float, outer=(3)] - │ └── variable: a.f [type=float, outer=(3)] + │ └── variable: f [type=float, outer=(3)] ├── const-agg [type=string, outer=(4)] - │ └── variable: a.s [type=string, outer=(4)] + │ └── variable: s [type=string, outer=(4)] └── const-agg [type=jsonb, outer=(5)] - └── variable: a.j [type=jsonb, outer=(5)] + └── variable: j [type=jsonb, outer=(5)] # Indirectly decorrelate GROUP BY after decorrelating scalar GROUP BY. opt @@ -1321,7 +1321,7 @@ project │ │ │ │ ├── key: (3) │ │ │ │ └── fd: (3)-->(4) │ │ │ └── filters [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ │ └── xy.x = uv.v [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] + │ │ │ └── x = v [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] │ │ ├── select │ │ │ ├── columns: k:5(int!null) i:6(int!null) │ │ │ ├── key: (5) @@ -1331,20 +1331,20 @@ project │ │ │ │ ├── key: (5) │ │ │ │ └── fd: (5)-->(6) │ │ │ └── filters [type=bool, outer=(6), constraints=(/6: (/NULL - ]; tight)] - │ │ │ └── a.i IS NOT NULL [type=bool, outer=(6), constraints=(/6: (/NULL - ]; tight)] + │ │ │ └── i IS NOT NULL [type=bool, outer=(6), constraints=(/6: (/NULL - ]; tight)] │ │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ │ └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + │ │ └── k = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] │ └── aggregations [outer=(1,2,4,6)] │ ├── max [type=int, outer=(6)] - │ │ └── variable: a.i [type=int, outer=(6)] + │ │ └── variable: i [type=int, outer=(6)] │ ├── const-agg [type=int, outer=(4)] - │ │ └── variable: uv.v [type=int, outer=(4)] + │ │ └── variable: v [type=int, outer=(4)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: xy.y [type=int, outer=(2)] + │ │ └── variable: y [type=int, outer=(2)] │ └── const-agg [type=int, outer=(1)] - │ └── variable: xy.x [type=int, outer=(1)] + │ └── variable: x [type=int, outer=(1)] └── filters [type=bool, outer=(3,10), constraints=(/3: (/NULL - ]; /10: (/NULL - ]), fd=(3)==(10), (10)==(3)] - └── uv.u = max [type=bool, outer=(3,10), constraints=(/3: (/NULL - ]; /10: (/NULL - ])] + └── u = max [type=bool, outer=(3,10), constraints=(/3: (/NULL - ]; /10: (/NULL - ])] # Indirectly decorrelate GROUP BY after decorrelating scalar GROUP BY. Use # IS DISTINCT FROM to retain left join. @@ -1383,24 +1383,24 @@ project │ │ │ │ ├── key: (3) │ │ │ │ └── fd: (3)-->(4) │ │ │ └── filters [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ │ └── xy.x = uv.v [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] + │ │ │ └── x = v [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] │ │ ├── scan a │ │ │ ├── columns: k:5(int!null) i:6(int) │ │ │ ├── key: (5) │ │ │ └── fd: (5)-->(6) │ │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ │ └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + │ │ └── k = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] │ └── aggregations [outer=(1,2,4,6)] │ ├── max [type=int, outer=(6)] - │ │ └── variable: a.i [type=int, outer=(6)] + │ │ └── variable: i [type=int, outer=(6)] │ ├── const-agg [type=int, outer=(4)] - │ │ └── variable: uv.v [type=int, outer=(4)] + │ │ └── variable: v [type=int, outer=(4)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: xy.y [type=int, outer=(2)] + │ │ └── variable: y [type=int, outer=(2)] │ └── const-agg [type=int, outer=(1)] - │ └── variable: xy.x [type=int, outer=(1)] + │ └── variable: x [type=int, outer=(1)] └── filters [type=bool, outer=(3,10)] - └── uv.u IS DISTINCT FROM max [type=bool, outer=(3,10)] + └── u IS DISTINCT FROM max [type=bool, outer=(3,10)] # Synthesize key when one is not present. opt @@ -1439,12 +1439,12 @@ project │ │ │ │ ├── key: () │ │ │ │ └── fd: ()-->(4,7) │ │ │ └── filters [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] - │ │ │ └── a.k = xy.y [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ])] + │ │ │ └── k = y [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ])] │ │ └── aggregations [outer=(2,7)] │ │ ├── max [type=string, outer=(7)] - │ │ │ └── variable: a.s [type=string, outer=(7)] + │ │ │ └── variable: s [type=string, outer=(7)] │ │ └── const-agg [type=int, outer=(2)] - │ │ └── variable: xy.y [type=int, outer=(2)] + │ │ └── variable: y [type=int, outer=(2)] │ └── filters [type=bool, outer=(9), constraints=(/9: [/'bar' - /'bar']; tight), fd=()-->(9)] │ └── max = 'bar' [type=bool, outer=(9), constraints=(/9: [/'bar' - /'bar']; tight)] └── projections [outer=(2)] @@ -1500,30 +1500,30 @@ group-by │ │ │ │ │ ├── key: (1) │ │ │ │ │ └── fd: (1)-->(2-5) │ │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] - │ │ │ │ └── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + │ │ │ │ └── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] │ │ │ └── true [type=bool] │ │ └── aggregations [outer=(2-5,8)] │ │ ├── first-agg [type=int, outer=(8)] - │ │ │ └── variable: uv.u [type=int, outer=(8)] + │ │ │ └── variable: u [type=int, outer=(8)] │ │ ├── const-agg [type=int, outer=(2)] - │ │ │ └── variable: a.i [type=int, outer=(2)] + │ │ │ └── variable: i [type=int, outer=(2)] │ │ ├── const-agg [type=float, outer=(3)] - │ │ │ └── variable: a.f [type=float, outer=(3)] + │ │ │ └── variable: f [type=float, outer=(3)] │ │ ├── const-agg [type=string, outer=(4)] - │ │ │ └── variable: a.s [type=string, outer=(4)] + │ │ │ └── variable: s [type=string, outer=(4)] │ │ └── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── filters [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] - │ └── xy.x = uv.u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + │ └── x = u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] └── aggregations [outer=(2-5)] ├── const-agg [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── const-agg [type=float, outer=(3)] - │ └── variable: a.f [type=float, outer=(3)] + │ └── variable: f [type=float, outer=(3)] ├── const-agg [type=string, outer=(4)] - │ └── variable: a.s [type=string, outer=(4)] + │ └── variable: s [type=string, outer=(4)] └── const-agg [type=jsonb, outer=(5)] - └── variable: a.j [type=jsonb, outer=(5)] + └── variable: j [type=jsonb, outer=(5)] # -------------------------------------------------- # TryDecorrelateScalarGroupBy @@ -1572,29 +1572,29 @@ group-by │ │ │ │ └── projections │ │ │ │ └── true [type=bool] │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] - │ │ │ └── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + │ │ │ └── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] │ │ └── aggregations [outer=(2-5,12)] │ │ ├── count [type=int, outer=(12)] │ │ │ └── variable: canary [type=bool, outer=(12)] │ │ ├── const-agg [type=int, outer=(2)] - │ │ │ └── variable: a.i [type=int, outer=(2)] + │ │ │ └── variable: i [type=int, outer=(2)] │ │ ├── const-agg [type=float, outer=(3)] - │ │ │ └── variable: a.f [type=float, outer=(3)] + │ │ │ └── variable: f [type=float, outer=(3)] │ │ ├── const-agg [type=string, outer=(4)] - │ │ │ └── variable: a.s [type=string, outer=(4)] + │ │ │ └── variable: s [type=string, outer=(4)] │ │ └── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── filters [type=bool, outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] - │ └── xy.x = cnt [type=bool, outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ])] + │ └── x = cnt [type=bool, outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ])] └── aggregations [outer=(2-5)] ├── const-agg [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── const-agg [type=float, outer=(3)] - │ └── variable: a.f [type=float, outer=(3)] + │ └── variable: f [type=float, outer=(3)] ├── const-agg [type=string, outer=(4)] - │ └── variable: a.s [type=string, outer=(4)] + │ └── variable: s [type=string, outer=(4)] └── const-agg [type=jsonb, outer=(5)] - └── variable: a.j [type=jsonb, outer=(5)] + └── variable: j [type=jsonb, outer=(5)] # Synthesize key when one is not present. opt @@ -1631,14 +1631,14 @@ project │ │ │ │ │ ├── key: (7) │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ └── filters [type=bool, outer=(8), constraints=(/8: (/NULL - ]; tight)] - │ │ │ │ └── xy.y IS NOT NULL [type=bool, outer=(8), constraints=(/8: (/NULL - ]; tight)] + │ │ │ │ └── y IS NOT NULL [type=bool, outer=(8), constraints=(/8: (/NULL - ]; tight)] │ │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ │ └── xy.x = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ │ └── x = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] │ │ └── aggregations [outer=(2,8)] │ │ ├── max [type=int, outer=(8)] - │ │ │ └── variable: xy.y [type=int, outer=(8)] + │ │ │ └── variable: y [type=int, outer=(8)] │ │ └── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ └── filters [type=bool, outer=(9), constraints=(/9: [/5 - /5]; tight), fd=()-->(9)] │ └── max = 5 [type=bool, outer=(9), constraints=(/9: [/5 - /5]; tight)] └── projections [outer=(2)] @@ -1666,14 +1666,14 @@ project │ │ ├── scan xy │ │ │ └── columns: y:7(int) │ │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ │ └── xy.y = a.k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ │ └── y = k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(7)] │ ├── array-agg [type=int[], outer=(7)] - │ │ └── variable: xy.y [type=int, outer=(7)] + │ │ └── variable: y [type=int, outer=(7)] │ └── any-not-null-agg [type=int, outer=(7)] - │ └── variable: xy.y [type=int, outer=(7)] + │ └── variable: y [type=int, outer=(7)] └── projections [outer=(1,7,10)] - └── CASE WHEN xy.y IS NOT NULL THEN array_agg END [type=int[], outer=(7,10)] + └── CASE WHEN y IS NOT NULL THEN array_agg END [type=int[], outer=(7,10)] # With multiple columns. Without LATERAL these tests are a bit verbose. norm @@ -1696,16 +1696,16 @@ project │ │ ├── scan xy │ │ │ └── columns: y:7(int) │ │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ │ └── xy.y = a.k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ │ └── y = k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(7)] │ ├── array-agg [type=int[], outer=(7)] - │ │ └── variable: xy.y [type=int, outer=(7)] + │ │ └── variable: y [type=int, outer=(7)] │ ├── max [type=int, outer=(7)] - │ │ └── variable: xy.y [type=int, outer=(7)] + │ │ └── variable: y [type=int, outer=(7)] │ └── any-not-null-agg [type=int, outer=(7)] - │ └── variable: xy.y [type=int, outer=(7)] + │ └── variable: y [type=int, outer=(7)] └── projections [outer=(1,7,9,12)] - └── (CASE WHEN xy.y IS NOT NULL THEN r END, q) [type=tuple{int[], int}, outer=(7,9,12)] + └── (CASE WHEN y IS NOT NULL THEN r END, q) [type=tuple{int[], int}, outer=(7,9,12)] # With an aggregate that can't ignore nulls and when a non-nullable column must be synthesized. @@ -1736,10 +1736,10 @@ project │ │ │ └── projections [outer=(7)] │ │ │ └── true [type=bool] │ │ └── filters [type=bool, outer=(2,7)] - │ │ └── (xy.y = a.i) OR (xy.y IS NULL) [type=bool, outer=(2,7)] + │ │ └── (y = i) OR (y IS NULL) [type=bool, outer=(2,7)] │ └── aggregations [outer=(7,10)] │ ├── array-agg [type=int[], outer=(7)] - │ │ └── variable: xy.y [type=int, outer=(7)] + │ │ └── variable: y [type=int, outer=(7)] │ └── any-not-null-agg [type=bool, outer=(10)] │ └── variable: canary [type=bool, outer=(10)] └── projections [outer=(1,10,11)] @@ -1776,12 +1776,12 @@ project │ │ │ └── projections [outer=(7)] │ │ │ └── true [type=bool] │ │ └── filters [type=bool, outer=(1,7)] - │ │ └── (xy.y = a.k) OR NULL [type=bool, outer=(1,7)] + │ │ └── (y = k) OR NULL [type=bool, outer=(1,7)] │ └── aggregations [outer=(2,7,10)] │ ├── array-agg [type=int[], outer=(7)] - │ │ └── variable: xy.y [type=int, outer=(7)] + │ │ └── variable: y [type=int, outer=(7)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ └── any-not-null-agg [type=bool, outer=(10)] │ └── variable: canary [type=bool, outer=(10)] └── projections [outer=(2,10,11)] @@ -1848,26 +1848,26 @@ project │ │ │ │ │ │ ├── key: (8) │ │ │ │ │ │ └── fd: (8)-->(9) │ │ │ │ │ └── filters [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] - │ │ │ │ │ └── uv.u = a.k [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] + │ │ │ │ │ └── u = k [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] │ │ │ │ └── aggregations [outer=(7,9)] │ │ │ │ ├── max [type=int, outer=(9)] - │ │ │ │ │ └── variable: uv.v [type=int, outer=(9)] + │ │ │ │ │ └── variable: v [type=int, outer=(9)] │ │ │ │ └── const-agg [type=int, outer=(7)] - │ │ │ │ └── variable: xy.y [type=int, outer=(7)] + │ │ │ │ └── variable: y [type=int, outer=(7)] │ │ │ └── filters [type=bool, outer=(10), constraints=(/10: [/NULL - /NULL]; tight), fd=()-->(10)] │ │ │ └── max IS NULL [type=bool, outer=(10), constraints=(/10: [/NULL - /NULL]; tight)] │ │ └── true [type=bool] │ └── aggregations [outer=(2-5,7)] │ ├── max [type=int, outer=(7)] - │ │ └── variable: xy.y [type=int, outer=(7)] + │ │ └── variable: y [type=int, outer=(7)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ └── const-agg [type=jsonb, outer=(5)] - │ └── variable: a.j [type=jsonb, outer=(5)] + │ └── variable: j [type=jsonb, outer=(5)] └── filters [type=bool, outer=(11), constraints=(/11: [/NULL - /NULL]; tight), fd=()-->(11)] └── max IS NULL [type=bool, outer=(11), constraints=(/11: [/NULL - /NULL]; tight)] @@ -1912,16 +1912,16 @@ project │ │ │ │ ├── key: (3) │ │ │ │ └── fd: (3)-->(4) │ │ │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ │ │ └── cd.c = xy.x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ │ │ └── c = x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] │ │ └── aggregations [outer=(2-4)] │ │ ├── array-agg [type=int[], outer=(4)] - │ │ │ └── variable: xy.y [type=int, outer=(4)] + │ │ │ └── variable: y [type=int, outer=(4)] │ │ ├── const-agg [type=int, outer=(2)] - │ │ │ └── variable: cd.d [type=int, outer=(2)] + │ │ │ └── variable: d [type=int, outer=(2)] │ │ └── any-not-null-agg [type=int, outer=(3)] - │ │ └── variable: xy.x [type=int, outer=(3)] + │ │ └── variable: x [type=int, outer=(3)] │ └── projections [outer=(1-3,6)] - │ └── CASE WHEN xy.x IS NOT NULL THEN array_agg END [type=int[], outer=(3,6)] + │ └── CASE WHEN x IS NOT NULL THEN array_agg END [type=int[], outer=(3,6)] └── filters [type=bool, outer=(5), constraints=(/5: (/NULL - ])] └── array_agg = ARRAY[] [type=bool, outer=(5), constraints=(/5: (/NULL - ])] @@ -1963,20 +1963,20 @@ project │ │ │ │ │ └── fd: (6)-->(7) │ │ │ │ └── projections [outer=(6,7)] │ │ │ │ ├── true [type=bool] - │ │ │ │ └── xy.y::STRING [type=string, outer=(7)] + │ │ │ │ └── y::STRING [type=string, outer=(7)] │ │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ │ └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ │ └── aggregations [outer=(2-5,8,10)] │ │ ├── concat-agg [type=string, outer=(8)] │ │ │ └── variable: column8 [type=string, outer=(8)] │ │ ├── const-agg [type=int, outer=(2)] - │ │ │ └── variable: a.i [type=int, outer=(2)] + │ │ │ └── variable: i [type=int, outer=(2)] │ │ ├── const-agg [type=float, outer=(3)] - │ │ │ └── variable: a.f [type=float, outer=(3)] + │ │ │ └── variable: f [type=float, outer=(3)] │ │ ├── const-agg [type=string, outer=(4)] - │ │ │ └── variable: a.s [type=string, outer=(4)] + │ │ │ └── variable: s [type=string, outer=(4)] │ │ ├── const-agg [type=jsonb, outer=(5)] - │ │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ │ └── variable: j [type=jsonb, outer=(5)] │ │ └── any-not-null-agg [type=bool, outer=(10)] │ │ └── variable: canary [type=bool, outer=(10)] │ └── projections [outer=(1-5,10,11)] @@ -2017,22 +2017,22 @@ project │ │ │ │ ├── scan xy │ │ │ │ │ └── columns: y:7(int) │ │ │ │ └── projections [outer=(7)] - │ │ │ │ └── xy.y::STRING [type=string, outer=(7)] + │ │ │ │ └── y::STRING [type=string, outer=(7)] │ │ │ └── filters [type=bool, outer=(9), constraints=(/9: (/NULL - ]; tight)] │ │ │ └── column9 IS NOT NULL [type=bool, outer=(9), constraints=(/9: (/NULL - ]; tight)] │ │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ │ └── xy.y = a.k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ │ └── y = k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(2-5,9)] │ ├── max [type=string, outer=(9)] │ │ └── variable: column9 [type=string, outer=(9)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ └── const-agg [type=jsonb, outer=(5)] - │ └── variable: a.j [type=jsonb, outer=(5)] + │ └── variable: j [type=jsonb, outer=(5)] └── filters [type=bool, outer=(10), constraints=(/10: [/'foo' - /'foo']; tight), fd=()-->(10)] └── max = 'foo' [type=bool, outer=(10), constraints=(/10: [/'foo' - /'foo']; tight)] @@ -2085,19 +2085,19 @@ group-by │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ └── filters [type=bool, outer=(2,10), constraints=(/10: (/NULL - ])] - │ │ │ └── a.f = xy.y::FLOAT [type=bool, outer=(2,10), constraints=(/10: (/NULL - ])] + │ │ │ └── a.f = y::FLOAT [type=bool, outer=(2,10), constraints=(/10: (/NULL - ])] │ │ └── aggregations [outer=(2,4,9)] │ │ ├── max [type=int, outer=(9)] │ │ │ └── variable: a.i [type=int, outer=(9)] │ │ ├── const-agg [type=int, outer=(4)] │ │ │ └── variable: a.i [type=int, outer=(4)] │ │ └── const-agg [type=int, outer=(2)] - │ │ └── variable: xy.y [type=int, outer=(2)] + │ │ └── variable: y [type=int, outer=(2)] │ └── filters [type=bool, outer=(4,13), constraints=(/4: (/NULL - ]; /13: (/NULL - ]), fd=(4)==(13), (13)==(4)] │ └── a.i = max [type=bool, outer=(4,13), constraints=(/4: (/NULL - ]; /13: (/NULL - ])] └── aggregations [outer=(2)] └── const-agg [type=int, outer=(2)] - └── variable: xy.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # Right input of SemiJoin is DistinctOn. opt @@ -2132,17 +2132,17 @@ group-by │ │ │ ├── scan a │ │ │ │ └── columns: i:4(int) f:5(float) │ │ │ └── filters [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] - │ │ │ └── xy.y > a.f [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] + │ │ │ └── y > f [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] │ │ └── aggregations [outer=(2,4)] │ │ ├── first-agg [type=int, outer=(4)] - │ │ │ └── variable: a.i [type=int, outer=(4)] + │ │ │ └── variable: i [type=int, outer=(4)] │ │ └── const-agg [type=int, outer=(2)] - │ │ └── variable: xy.y [type=int, outer=(2)] + │ │ └── variable: y [type=int, outer=(2)] │ └── filters [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ └── xy.x = a.i [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] + │ └── x = i [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] └── aggregations [outer=(2)] └── const-agg [type=int, outer=(2)] - └── variable: xy.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # Right input of SemiJoin is Project. opt @@ -2182,11 +2182,11 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ └── filters [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] - │ │ └── uv.u = a.i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] + │ │ └── u = i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] │ └── projections [outer=(1,6,8)] - │ └── COALESCE(uv.u, 10) [type=int, outer=(8)] + │ └── COALESCE(u, 10) [type=int, outer=(8)] └── filters [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] - └── xy.x = computed [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] + └── x = computed [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] # -------------------------------------------------- # TryDecorrelateLimitOne @@ -2229,9 +2229,9 @@ semi-join │ ├── left ordering: +6 │ ├── right ordering: +8 │ └── filters [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] - │ └── xy.x = uv.u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + │ └── x = u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] └── filters [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] - └── uv.v = a.i [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ])] + └── v = i [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ])] # With left join. opt @@ -2259,7 +2259,7 @@ project │ │ │ ├── key: (6) │ │ │ └── fd: (6)-->(7) │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(6)] │ └── first-agg [type=int, outer=(6)] │ └── variable: xy.x [type=int, outer=(6)] @@ -2313,35 +2313,35 @@ project │ │ │ │ │ │ ├── key: (6) │ │ │ │ │ │ └── fd: (6)-->(7) │ │ │ │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ │ │ │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ │ │ │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] │ │ │ │ └── aggregations [outer=(2-6)] │ │ │ │ ├── const-agg [type=int, outer=(2)] - │ │ │ │ │ └── variable: a.i [type=int, outer=(2)] + │ │ │ │ │ └── variable: i [type=int, outer=(2)] │ │ │ │ ├── const-agg [type=float, outer=(3)] - │ │ │ │ │ └── variable: a.f [type=float, outer=(3)] + │ │ │ │ │ └── variable: f [type=float, outer=(3)] │ │ │ │ ├── const-agg [type=string, outer=(4)] - │ │ │ │ │ └── variable: a.s [type=string, outer=(4)] + │ │ │ │ │ └── variable: s [type=string, outer=(4)] │ │ │ │ ├── const-agg [type=jsonb, outer=(5)] - │ │ │ │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ │ │ │ └── variable: j [type=jsonb, outer=(5)] │ │ │ │ └── first-agg [type=int, outer=(6)] - │ │ │ │ └── variable: xy.x [type=int, outer=(6)] + │ │ │ │ └── variable: x [type=int, outer=(6)] │ │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ │ └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ │ └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ │ └── filters [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] - │ │ └── uv.v = a.i [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ])] + │ │ └── v = i [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ])] │ └── aggregations [outer=(2-5,8)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ ├── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── first-agg [type=int, outer=(8)] - │ └── variable: uv.u [type=int, outer=(8)] + │ └── variable: u [type=int, outer=(8)] └── filters [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] - └── a.k = uv.u [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] + └── k = u [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] # With nested limited queries. opt @@ -2401,29 +2401,29 @@ project │ │ │ │ │ │ ├── key: (8) │ │ │ │ │ │ └── fd: (8)-->(9) │ │ │ │ │ └── filters [type=bool, outer=(7,9), constraints=(/7: (/NULL - ]; /9: (/NULL - ]), fd=(7)==(9), (9)==(7)] - │ │ │ │ │ └── uv.v = xy.y [type=bool, outer=(7,9), constraints=(/7: (/NULL - ]; /9: (/NULL - ])] + │ │ │ │ │ └── v = y [type=bool, outer=(7,9), constraints=(/7: (/NULL - ]; /9: (/NULL - ])] │ │ │ │ └── aggregations [outer=(7,8)] │ │ │ │ ├── const-agg [type=int, outer=(7)] - │ │ │ │ │ └── variable: xy.y [type=int, outer=(7)] + │ │ │ │ │ └── variable: y [type=int, outer=(7)] │ │ │ │ └── first-agg [type=int, outer=(8)] - │ │ │ │ └── variable: uv.u [type=int, outer=(8)] + │ │ │ │ └── variable: u [type=int, outer=(8)] │ │ │ └── filters [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] - │ │ │ └── xy.x = uv.u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + │ │ │ └── x = u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(2-6)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ ├── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── first-agg [type=int, outer=(6)] - │ └── variable: xy.x [type=int, outer=(6)] + │ └── variable: x [type=int, outer=(6)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # With inner join + ORDER BY. opt @@ -2469,9 +2469,9 @@ project │ │ │ ├── left ordering: +3 │ │ │ ├── right ordering: +5 │ │ │ └── filters [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] - │ │ │ └── uv.u = a.k [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] + │ │ │ └── u = k [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── a.i = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ └── i = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ └── aggregations [outer=(4)] │ └── first-agg [type=int, outer=(4)] │ └── variable: uv.v [type=int, outer=(4)] @@ -2514,14 +2514,14 @@ project │ │ │ ├── key: (3) │ │ │ └── fd: (3)-->(4-6) │ │ └── filters [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] - │ │ └── a.i = xy.y [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ])] + │ │ └── i = y [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ])] │ └── aggregations [outer=(2,3)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: xy.y [type=int, outer=(2)] + │ │ └── variable: y [type=int, outer=(2)] │ └── first-agg [type=int, outer=(3)] - │ └── variable: a.k [type=int, outer=(3)] + │ └── variable: k [type=int, outer=(3)] └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - └── xy.x = a.k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + └── x = k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] # -------------------------------------------------- # HoistSelectExists @@ -2547,7 +2547,7 @@ semi-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Ensure that EXISTS is hoisted even when it is one of several conjuncts. opt @@ -2568,8 +2568,8 @@ semi-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 opt(4) │ └── filters [type=bool, outer=(2,4), constraints=(/2: [/2 - ]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] - │ ├── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] - │ └── a.i > 1 [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] + │ ├── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ └── i > 1 [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -2579,7 +2579,7 @@ semi-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Multiple Exists operators in same Select list. opt @@ -2602,13 +2602,13 @@ semi-join │ │ ├── key: (8) │ │ └── fd: (8)-->(9) │ └── filters [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] - │ └── xy.x = a.i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] + │ └── xy.x = i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] ├── scan xy │ ├── columns: xy.x:6(int!null) xy.y:7(int) │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── xy.x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Don't hoist uncorrelated subqueries. opt @@ -2661,12 +2661,12 @@ semi-join (merge) │ ├── left ordering: +6 │ ├── right ordering: +8 │ └── filters [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] - │ └── xy.x = uv.u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + │ └── x = u [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # -------------------------------------------------- # HoistSelectNotExists @@ -2692,7 +2692,7 @@ anti-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Ensure that NOT EXISTS is hoisted even when one of several conjuncts. opt @@ -2713,8 +2713,8 @@ anti-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 opt(4) │ └── filters [type=bool, outer=(2,4), constraints=(/2: [/2 - ]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] - │ ├── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] - │ └── a.i > 1 [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] + │ ├── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ └── i > 1 [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -2724,7 +2724,7 @@ anti-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Multiple Not Exists operators in same Select list. opt @@ -2749,13 +2749,13 @@ anti-join │ │ ├── key: (8) │ │ └── fd: (8)-->(9) │ └── filters [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] - │ └── xy.x = a.i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] + │ └── xy.x = i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] ├── scan xy │ ├── columns: xy.x:6(int!null) xy.y:7(int) │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── xy.x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Don't hoist uncorrelated subqueries. opt @@ -2800,13 +2800,13 @@ semi-join │ │ ├── key: (8) │ │ └── fd: (8)-->(9) │ └── filters [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] - │ └── xy.x = a.i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] + │ └── xy.x = i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] ├── scan xy │ ├── columns: xy.x:6(int!null) xy.y:7(int) │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── xy.x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # -------------------------------------------------- # HoistSelectSubquery @@ -2837,20 +2837,20 @@ project │ │ ├── scan xy │ │ │ └── columns: y:7(int) │ │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ │ └── xy.y = a.k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ │ └── y = k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(2-5,7)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ ├── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── first-agg [type=int, outer=(7)] - │ └── variable: xy.y [type=int, outer=(7)] + │ └── variable: y [type=int, outer=(7)] └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── a.i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── i = y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Multiple other conjuncts, including uncorrelated subquery (don't hoist). opt @@ -2915,22 +2915,22 @@ project │ │ ├── left ordering: +1 │ │ ├── right ordering: +7 │ │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ │ └── xy.y = a.k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ │ └── xy.y = k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(1-5,7)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ ├── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ ├── first-agg [type=int, outer=(7)] │ │ └── variable: xy.y [type=int, outer=(7)] │ └── const-agg [type=int, outer=(1)] - │ └── variable: a.k [type=int, outer=(1)] + │ └── variable: k [type=int, outer=(1)] └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── a.i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Multiple correlated subqueries. opt @@ -2974,35 +2974,35 @@ project │ │ │ │ │ ├── scan xy │ │ │ │ │ │ └── columns: xy.y:7(int) │ │ │ │ │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ │ │ │ │ └── xy.y = a.k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ │ │ │ │ └── xy.y = k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] │ │ │ │ └── aggregations [outer=(2-5,7)] │ │ │ │ ├── count [type=int, outer=(7)] │ │ │ │ │ └── variable: xy.y [type=int, outer=(7)] │ │ │ │ ├── const-agg [type=int, outer=(2)] - │ │ │ │ │ └── variable: a.i [type=int, outer=(2)] + │ │ │ │ │ └── variable: i [type=int, outer=(2)] │ │ │ │ ├── const-agg [type=float, outer=(3)] - │ │ │ │ │ └── variable: a.f [type=float, outer=(3)] + │ │ │ │ │ └── variable: f [type=float, outer=(3)] │ │ │ │ ├── const-agg [type=string, outer=(4)] - │ │ │ │ │ └── variable: a.s [type=string, outer=(4)] + │ │ │ │ │ └── variable: s [type=string, outer=(4)] │ │ │ │ └── const-agg [type=jsonb, outer=(5)] - │ │ │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ │ │ └── variable: j [type=jsonb, outer=(5)] │ │ │ └── filters [type=bool, outer=(8), constraints=(/8: [/1 - ]; tight)] │ │ │ └── count > 0 [type=bool, outer=(8), constraints=(/8: [/1 - ]; tight)] │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ └── xy.y = a.k [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ └── xy.y = k [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ └── aggregations [outer=(2-5,10)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ ├── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── first-agg [type=int, outer=(10)] │ └── variable: xy.y [type=int, outer=(10)] └── filters [type=bool, outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] - └── a.i = xy.y [type=bool, outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ])] + └── i = xy.y [type=bool, outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ])] # Subquery nested below interesting scalar operators like cast, function, tuple, # or, etc). @@ -3039,20 +3039,20 @@ project │ │ ├── left ordering: +1 │ │ ├── right ordering: +6 │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── a.k = uv.u [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ └── k = u [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ └── aggregations [outer=(2-6)] │ ├── count [type=int, outer=(6)] - │ │ └── variable: uv.u [type=int, outer=(6)] + │ │ └── variable: u [type=int, outer=(6)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ └── const-agg [type=jsonb, outer=(5)] - │ └── variable: a.j [type=jsonb, outer=(5)] + │ └── variable: j [type=jsonb, outer=(5)] └── filters [type=bool, outer=(2,8)] - └── ((0, length(count::STRING)) > (0, 1)) OR (a.i = 1) [type=bool, outer=(2,8)] + └── ((0, length(count::STRING)) > (0, 1)) OR (i = 1) [type=bool, outer=(2,8)] # Exists within a disjunction. opt @@ -3086,20 +3086,20 @@ project │ │ │ └── projections [outer=(7)] │ │ │ └── true [type=bool] │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(2-5,8)] │ ├── const-not-null-agg [type=bool, outer=(8)] │ │ └── variable: true [type=bool, outer=(8)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ └── const-agg [type=jsonb, outer=(5)] - │ └── variable: a.j [type=jsonb, outer=(5)] + │ └── variable: j [type=jsonb, outer=(5)] └── filters [type=bool, outer=(2,9)] - └── (a.i = 1) OR (true_agg IS NOT NULL) [type=bool, outer=(2,9)] + └── (i = 1) OR (true_agg IS NOT NULL) [type=bool, outer=(2,9)] # Any with IS NULL. opt @@ -3135,22 +3135,22 @@ project │ │ │ │ ├── scan xy │ │ │ │ │ └── columns: y:7(int) │ │ │ │ └── projections [outer=(7)] - │ │ │ │ └── xy.y IS NOT NULL [type=bool, outer=(7)] + │ │ │ │ └── y IS NOT NULL [type=bool, outer=(7)] │ │ │ └── filters [type=bool, outer=(2,7)] - │ │ │ └── (a.i = xy.y) IS NOT false [type=bool, outer=(2,7)] + │ │ │ └── (i = y) IS NOT false [type=bool, outer=(2,7)] │ │ └── aggregations [outer=(2-5,8)] │ │ ├── bool-or [type=bool, outer=(8)] │ │ │ └── variable: notnull [type=bool, outer=(8)] │ │ ├── const-agg [type=int, outer=(2)] - │ │ │ └── variable: a.i [type=int, outer=(2)] + │ │ │ └── variable: i [type=int, outer=(2)] │ │ ├── const-agg [type=float, outer=(3)] - │ │ │ └── variable: a.f [type=float, outer=(3)] + │ │ │ └── variable: f [type=float, outer=(3)] │ │ ├── const-agg [type=string, outer=(4)] - │ │ │ └── variable: a.s [type=string, outer=(4)] + │ │ │ └── variable: s [type=string, outer=(4)] │ │ └── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── projections [outer=(1-5,9)] - │ └── CASE WHEN bool_or AND (a.i IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(2,9)] + │ └── CASE WHEN bool_or AND (i IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(2,9)] └── filters [type=bool, outer=(10), constraints=(/10: [/NULL - /NULL]; tight), fd=()-->(10)] └── case IS NULL [type=bool, outer=(10), constraints=(/10: [/NULL - /NULL]; tight)] @@ -3182,21 +3182,21 @@ project │ │ │ ├── scan a │ │ │ │ └── columns: i:2(int) s:4(string) │ │ │ └── projections [outer=(2,4), side-effects] - │ │ │ └── (a.i * a.i) / 100 [type=decimal, outer=(2), side-effects] + │ │ │ └── (i * i) / 100 [type=decimal, outer=(2), side-effects] │ │ ├── project │ │ │ ├── columns: notnull:10(bool) y:7(int) │ │ │ ├── fd: (7)-->(10) │ │ │ ├── scan xy │ │ │ │ └── columns: y:7(int) │ │ │ └── projections [outer=(7)] - │ │ │ └── xy.y IS NOT NULL [type=bool, outer=(7)] + │ │ │ └── y IS NOT NULL [type=bool, outer=(7)] │ │ └── filters [type=bool, outer=(7,9)] - │ │ └── (scalar >= xy.y) IS NOT false [type=bool, outer=(7,9)] + │ │ └── (scalar >= y) IS NOT false [type=bool, outer=(7,9)] │ └── aggregations [outer=(4,9,10)] │ ├── bool-or [type=bool, outer=(10)] │ │ └── variable: notnull [type=bool, outer=(10)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ └── const-agg [type=decimal, outer=(9)] │ └── variable: scalar [type=decimal, outer=(9)] └── projections [outer=(4,9,11)] @@ -3225,7 +3225,7 @@ project │ ├── left ordering: +1 │ ├── right ordering: +6 │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── xy.x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── projections [outer=(6)] └── variable: xy.x [type=int, outer=(6)] @@ -3265,11 +3265,11 @@ project │ │ │ ├── left ordering: +1 │ │ │ ├── right ordering: +7 │ │ │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ │ │ └── xy.x = a.k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ │ │ └── xy.x = k [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] │ │ ├── scan xy │ │ │ └── columns: xy.y:20(int) │ │ └── filters [type=bool, outer=(1,20), constraints=(/1: (/NULL - ]; /20: (/NULL - ]), fd=(1)==(20), (20)==(1)] - │ │ └── xy.y = a.k [type=bool, outer=(1,20), constraints=(/1: (/NULL - ]; /20: (/NULL - ])] + │ │ └── xy.y = k [type=bool, outer=(1,20), constraints=(/1: (/NULL - ]; /20: (/NULL - ])] │ └── aggregations [outer=(20)] │ └── count [type=int, outer=(20)] │ └── variable: xy.y [type=int, outer=(20)] @@ -3321,10 +3321,10 @@ scalar-group-by │ │ │ ├── scan xy │ │ │ │ └── columns: y:7(int) │ │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] │ │ └── true [type=bool] │ └── projections [outer=(7)] - │ └── variable: xy.y [type=int, outer=(7)] + │ └── variable: y [type=int, outer=(7)] └── aggregations [outer=(8)] └── max [type=int, outer=(8)] └── variable: column8 [type=int, outer=(8)] @@ -3357,7 +3357,7 @@ project │ │ │ └── projections [outer=(7)] │ │ │ └── true [type=bool] │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(9)] │ └── const-not-null-agg [type=bool, outer=(9)] │ └── variable: true [type=bool, outer=(9)] @@ -3392,11 +3392,11 @@ project │ │ │ │ ├── scan xy │ │ │ │ │ └── columns: y:7(int) │ │ │ │ └── filters [type=bool, outer=(7)] - │ │ │ │ └── (xy.y > 5) IS NOT false [type=bool, outer=(7)] + │ │ │ │ └── (y > 5) IS NOT false [type=bool, outer=(7)] │ │ │ └── projections [outer=(7)] - │ │ │ └── xy.y IS NOT NULL [type=bool, outer=(7)] + │ │ │ └── y IS NOT NULL [type=bool, outer=(7)] │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] │ └── aggregations [outer=(9)] │ └── bool-or [type=bool, outer=(9)] │ └── variable: notnull [type=bool, outer=(9)] @@ -3435,7 +3435,7 @@ project │ └── projections [outer=(7)] │ └── true [type=bool] └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── xy.y = a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # -------------------------------------------------- # HoistJoinSubquery @@ -3459,7 +3459,7 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── filters [type=bool, outer=(1)] - │ └── (a.k + 1) = 0 [type=bool, outer=(1)] + │ └── (k + 1) = 0 [type=bool, outer=(1)] └── true [type=bool] # Right join + multiple subqueries. @@ -3482,8 +3482,8 @@ project │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── projections [outer=(1,6,7)] - │ ├── xy.x + 1 [type=int, outer=(6)] - │ └── a.k + 1 [type=int, outer=(1)] + │ ├── x + 1 [type=int, outer=(6)] + │ └── k + 1 [type=int, outer=(1)] └── filters [type=bool, outer=(8,9), constraints=(/8: (/NULL - ]; /9: (/NULL - ]), fd=(8)==(9), (9)==(8)] └── ?column? = ?column? [type=bool, outer=(8,9), constraints=(/8: (/NULL - ]; /9: (/NULL - ])] @@ -3528,14 +3528,14 @@ project │ │ │ │ └── projections [outer=(8)] │ │ │ │ └── true [type=bool] │ │ │ └── filters [type=bool, outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ]), fd=(7)==(8), (8)==(7)] - │ │ │ └── uv.u = xy.y [type=bool, outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ])] + │ │ │ └── u = y [type=bool, outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ])] │ │ └── aggregations [outer=(10)] │ │ └── const-not-null-agg [type=bool, outer=(10)] │ │ └── variable: true [type=bool, outer=(10)] │ └── projections [outer=(6,11)] │ └── true_agg IS NOT NULL [type=bool, outer=(11)] └── filters [type=bool, outer=(1,6,12)] - └── exists OR (a.k = xy.x) [type=bool, outer=(1,6,12)] + └── exists OR (k = x) [type=bool, outer=(1,6,12)] # Any in Join filter disjunction. opt @@ -3578,19 +3578,19 @@ project │ │ │ │ │ │ ├── key: (8) │ │ │ │ │ │ └── fd: (8)-->(9) │ │ │ │ │ └── projections [outer=(8,9)] - │ │ │ │ │ └── uv.v IS NOT NULL [type=bool, outer=(9)] + │ │ │ │ │ └── v IS NOT NULL [type=bool, outer=(9)] │ │ │ │ └── filters [type=bool, outer=(6-9), constraints=(/7: (/NULL - ]; /8: (/NULL - ]), fd=(7)==(8), (8)==(7)] - │ │ │ │ ├── uv.u = xy.y [type=bool, outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ])] - │ │ │ │ └── (xy.x = uv.v) IS NOT false [type=bool, outer=(6,9)] + │ │ │ │ ├── u = y [type=bool, outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ])] + │ │ │ │ └── (x = v) IS NOT false [type=bool, outer=(6,9)] │ │ │ └── aggregations [outer=(7,10)] │ │ │ ├── bool-or [type=bool, outer=(10)] │ │ │ │ └── variable: notnull [type=bool, outer=(10)] │ │ │ └── const-agg [type=int, outer=(7)] - │ │ │ └── variable: xy.y [type=int, outer=(7)] + │ │ │ └── variable: y [type=int, outer=(7)] │ │ └── projections [outer=(6,7,11)] - │ │ └── CASE WHEN bool_or AND (xy.x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(6,11)] + │ │ └── CASE WHEN bool_or AND (x IS NOT NULL) THEN true WHEN bool_or IS NULL THEN false END [type=bool, outer=(6,11)] │ └── filters [type=bool, outer=(6,12)] - │ └── case OR (xy.x IS NULL) [type=bool, outer=(6,12)] + │ └── case OR (x IS NULL) [type=bool, outer=(6,12)] └── true [type=bool] # -------------------------------------------------- @@ -3635,8 +3635,8 @@ project │ │ │ │ ├── key: () │ │ │ │ └── tuple [type=tuple] │ │ │ └── projections [outer=(1,2)] - │ │ │ ├── a.k + 1 [type=int, outer=(1)] - │ │ │ └── a.i + 1 [type=int, outer=(2)] + │ │ │ ├── k + 1 [type=int, outer=(1)] + │ │ │ └── i + 1 [type=int, outer=(2)] │ │ ├── values │ │ │ ├── columns: column1:8(int) │ │ │ ├── outer: (6,7) @@ -3700,7 +3700,7 @@ project │ │ │ │ │ │ ├── key: () │ │ │ │ │ │ └── tuple [type=tuple] │ │ │ │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ │ │ │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ │ │ │ └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ │ │ │ └── aggregations [outer=(9)] │ │ │ │ └── const-not-null-agg [type=bool, outer=(9)] │ │ │ │ └── variable: true [type=bool, outer=(9)] @@ -3768,15 +3768,15 @@ project │ │ │ │ │ │ │ │ ├── key: (6) │ │ │ │ │ │ │ │ └── fd: (6)-->(7) │ │ │ │ │ │ │ └── filters [type=bool, outer=(7)] - │ │ │ │ │ │ │ └── (xy.y = 5) IS NOT false [type=bool, outer=(7)] + │ │ │ │ │ │ │ └── (y = 5) IS NOT false [type=bool, outer=(7)] │ │ │ │ │ │ └── projections [outer=(6,7)] - │ │ │ │ │ │ └── xy.y IS NOT NULL [type=bool, outer=(7)] + │ │ │ │ │ │ └── y IS NOT NULL [type=bool, outer=(7)] │ │ │ │ │ ├── values │ │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ │ ├── key: () │ │ │ │ │ │ └── tuple [type=tuple] │ │ │ │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ │ │ │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ │ │ │ └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ │ │ │ └── aggregations [outer=(9)] │ │ │ │ └── bool-or [type=bool, outer=(9)] │ │ │ │ └── variable: notnull [type=bool, outer=(9)] @@ -3825,12 +3825,12 @@ project │ ├── left ordering: +1 │ ├── right ordering: +3 │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ └── uv.u = xy.x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ └── u = x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] ├── zip │ ├── columns: column5:5(int) │ ├── outer: (4) │ ├── side-effects - │ └── generate_series(1, uv.v) [type=int, outer=(4), side-effects] + │ └── generate_series(1, v) [type=int, outer=(4), side-effects] └── true [type=bool] # Zip correlation within EXISTS. @@ -3869,12 +3869,12 @@ semi-join-apply │ │ │ ├── key: () │ │ │ └── tuple [type=tuple] │ │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ │ └── uv.u = xy.x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ │ └── u = x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] │ ├── zip │ │ ├── columns: generate_series:5(int) │ │ ├── outer: (4) │ │ ├── side-effects - │ │ └── generate_series(1, uv.v) [type=int, outer=(4), side-effects] + │ │ └── generate_series(1, v) [type=int, outer=(4), side-effects] │ └── true [type=bool] └── true [type=bool] @@ -3914,7 +3914,7 @@ project │ │ ├── left ordering: +1 │ │ ├── right ordering: +6 │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ ├── scan uv │ │ ├── columns: u:8(int!null) v:9(int) │ │ ├── key: (8) @@ -3924,12 +3924,12 @@ project │ ├── left ordering: +1 │ ├── right ordering: +8 │ └── filters [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] - │ └── uv.u = a.k [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] + │ └── u = k [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] ├── zip │ ├── columns: column10:10(int) │ ├── outer: (7,9) │ ├── side-effects - │ └── generate_series(xy.y, uv.v) [type=int, outer=(7,9), side-effects] + │ └── generate_series(y, v) [type=int, outer=(7,9), side-effects] └── true [type=bool] # Multiple functions. @@ -3940,7 +3940,7 @@ SELECT FROM a ---- project - ├── columns: generate_series:8(int) _pg_expandarray:13(tuple{int AS x, int AS n}) + ├── columns: generate_series:8(int) information_schema._pg_expandarray:13(tuple{int AS x, int AS n}) ├── side-effects ├── inner-join-apply │ ├── columns: k:1(int!null) v:7(int) column8:8(int) xy.x:9(int) x:11(int) n:12(int) @@ -3968,7 +3968,7 @@ project │ │ │ ├── left ordering: +1 │ │ │ ├── right ordering: +6 │ │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ │ └── uv.u = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ │ └── u = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ │ ├── scan xy │ │ │ ├── columns: xy.x:9(int!null) │ │ │ ├── key: (9) @@ -3977,12 +3977,12 @@ project │ │ ├── left ordering: +1 │ │ ├── right ordering: +9 │ │ └── filters [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] - │ │ └── xy.x = a.k [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] + │ │ └── xy.x = k [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] │ ├── zip │ │ ├── columns: column8:8(int) x:11(int) n:12(int) │ │ ├── outer: (7,9) │ │ ├── side-effects - │ │ ├── generate_series(1, uv.v) [type=int, outer=(7), side-effects] + │ │ ├── generate_series(1, v) [type=int, outer=(7), side-effects] │ │ └── information_schema._pg_expandarray(ARRAY[xy.x]) [type=tuple{int AS x, int AS n}, outer=(9), side-effects] │ └── true [type=bool] └── projections [outer=(8,11,12)] @@ -4005,7 +4005,7 @@ semi-join ├── scan xy │ └── columns: y:7(int) └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── a.i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── i = y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Any is one of several conjuncts. opt @@ -4028,11 +4028,11 @@ semi-join │ │ ├── key: () │ │ └── fd: ()-->(1-5) │ └── filters [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] - │ └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] ├── scan xy │ └── columns: y:7(int) └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - └── a.i < xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── i < y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Multiple ANY conjuncts. opt @@ -4057,11 +4057,11 @@ semi-join │ │ └── projections [outer=(9)] │ │ └── xy.y::STRING [type=string, outer=(9)] │ └── filters [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] - │ └── a.s = y [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ])] + │ └── s = y [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ])] ├── scan xy │ └── columns: xy.y:7(int) └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - └── a.i < xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── i < xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Don't hoist uncorrelated ANY (but rewrite it to EXISTS). opt @@ -4083,7 +4083,7 @@ select ├── scan xy │ └── columns: y:7(int) └── filters [type=bool, outer=(7), constraints=(/7: [/5 - /5]; tight), fd=()-->(7)] - └── xy.y = 5 [type=bool, outer=(7), constraints=(/7: [/5 - /5]; tight)] + └── y = 5 [type=bool, outer=(7), constraints=(/7: [/5 - /5]; tight)] # ANY in Join On condition. opt @@ -4104,13 +4104,13 @@ inner-join │ ├── scan uv │ │ └── columns: v:9(int) │ └── filters [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] - │ └── a.i = uv.v [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ])] + │ └── i = v [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ])] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # -------------------------------------------------- # NormalizeNotAnyFilter @@ -4129,7 +4129,7 @@ anti-join ├── scan xy │ └── columns: y:7(int) └── filters [type=bool, outer=(2,7)] - └── (a.i = xy.y) IS NOT false [type=bool, outer=(2,7)] + └── (i = y) IS NOT false [type=bool, outer=(2,7)] # NOT ANY is one of several conjuncts. Note that i > ALL(...) gets mapped to # NOT i <= ANY(...) by optbuilder. @@ -4148,7 +4148,7 @@ anti-join ├── scan xy │ └── columns: y:7(int) └── filters [type=bool, outer=(2,7)] - └── (a.i <= xy.y) IS NOT false [type=bool, outer=(2,7)] + └── (i <= y) IS NOT false [type=bool, outer=(2,7)] # Multiple NOT ANY conjuncts. opt @@ -4173,11 +4173,11 @@ anti-join │ │ └── projections [outer=(9)] │ │ └── xy.y::STRING [type=string, outer=(9)] │ └── filters [type=bool, outer=(4,10)] - │ └── (a.s = y) IS NOT false [type=bool, outer=(4,10)] + │ └── (s = y) IS NOT false [type=bool, outer=(4,10)] ├── scan xy │ └── columns: xy.y:7(int) └── filters [type=bool, outer=(2,7)] - └── (a.i >= xy.y) IS NOT false [type=bool, outer=(2,7)] + └── (i >= xy.y) IS NOT false [type=bool, outer=(2,7)] # Don't hoist uncorrelated NOT ANY (but rewrite it to NOT EXISTS). opt @@ -4199,7 +4199,7 @@ select ├── scan xy │ └── columns: y:7(int) └── filters [type=bool, outer=(7)] - └── (xy.y = 5) IS NOT false [type=bool, outer=(7)] + └── (y = 5) IS NOT false [type=bool, outer=(7)] # NOT ANY in Join On condition. opt @@ -4220,13 +4220,13 @@ inner-join │ ├── scan uv │ │ └── columns: v:9(int) │ └── filters [type=bool, outer=(2,9)] - │ └── (a.i = uv.v) IS NOT false [type=bool, outer=(2,9)] + │ └── (i = v) IS NOT false [type=bool, outer=(2,9)] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # -------------------------------------------------- # NormalizeAnyFilter + NormalizeNotAnyFilter @@ -4253,8 +4253,8 @@ semi-join │ │ └── projections [outer=(9)] │ │ └── xy.y::STRING [type=string, outer=(9)] │ └── filters [type=bool, outer=(4,10)] - │ └── (a.s = y) IS NOT false [type=bool, outer=(4,10)] + │ └── (s = y) IS NOT false [type=bool, outer=(4,10)] ├── scan xy │ └── columns: xy.y:7(int) └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── a.i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] diff --git a/pkg/sql/opt/norm/testdata/rules/groupby b/pkg/sql/opt/norm/testdata/rules/groupby index 88539e6b5017..9c6afb574f0d 100644 --- a/pkg/sql/opt/norm/testdata/rules/groupby +++ b/pkg/sql/opt/norm/testdata/rules/groupby @@ -115,7 +115,7 @@ group-by │ └── columns: f:3(float) s:4(string!null) └── aggregations [outer=(3)] └── sum [type=float, outer=(3)] - └── variable: a.f [type=float, outer=(3)] + └── variable: f [type=float, outer=(3)] # -------------------------------------------------- @@ -273,10 +273,10 @@ project │ │ ├── columns: i:2(int!null) s:4(string!null) │ │ └── key: (2,4) │ └── projections [outer=(2,4)] - │ └── a.i + 1 [type=int, outer=(2)] + │ └── i + 1 [type=int, outer=(2)] └── aggregations [outer=(4)] └── min [type=string, outer=(4)] - └── variable: a.s [type=string, outer=(4)] + └── variable: s [type=string, outer=(4)] # -------------------------------------------------- # ReduceGroupingCols @@ -295,11 +295,11 @@ group-by │ └── fd: (1)-->(2-4), (2,4)-->(1,3), (2,3)~~>(1,4) └── aggregations [outer=(2-4)] ├── min [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── const-agg [type=float, outer=(3)] - │ └── variable: a.f [type=float, outer=(3)] + │ └── variable: f [type=float, outer=(3)] └── const-agg [type=string, outer=(4)] - └── variable: a.s [type=string, outer=(4)] + └── variable: s [type=string, outer=(4)] opt SELECT k, sum(DISTINCT i), f, s FROM a, xy GROUP BY s, f, k @@ -321,11 +321,11 @@ group-by └── aggregations [outer=(2-4)] ├── sum [type=decimal, outer=(2)] │ └── agg-distinct [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── const-agg [type=float, outer=(3)] - │ └── variable: a.f [type=float, outer=(3)] + │ └── variable: f [type=float, outer=(3)] └── const-agg [type=string, outer=(4)] - └── variable: a.s [type=string, outer=(4)] + └── variable: s [type=string, outer=(4)] # Eliminated columns are not part of projection. opt @@ -344,7 +344,7 @@ project │ └── fd: (2,4)-->(3), (2,3)~~>(4) └── aggregations [outer=(3)] └── min [type=float, outer=(3)] - └── variable: a.f [type=float, outer=(3)] + └── variable: f [type=float, outer=(3)] # All grouping columns eliminated. opt @@ -363,9 +363,9 @@ group-by │ └── fd: ()-->(1-3) └── aggregations [outer=(2,3)] ├── sum [type=float, outer=(3)] - │ └── variable: a.f [type=float, outer=(3)] + │ └── variable: f [type=float, outer=(3)] └── const-agg [type=int, outer=(2)] - └── variable: a.i [type=int, outer=(2)] + └── variable: i [type=int, outer=(2)] opt SELECT DISTINCT ON (k, f, s) i, f, x FROM a JOIN xy ON i=y @@ -388,14 +388,14 @@ distinct-on │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ └── a.i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ └── i = y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] └── aggregations [outer=(2,3,6)] ├── first-agg [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── first-agg [type=int, outer=(6)] - │ └── variable: xy.x [type=int, outer=(6)] + │ └── variable: x [type=int, outer=(6)] └── const-agg [type=float, outer=(3)] - └── variable: a.f [type=float, outer=(3)] + └── variable: f [type=float, outer=(3)] # -------------------------------------------------- # EliminateAggDistinctForKeys @@ -417,10 +417,10 @@ scalar-group-by │ └── fd: (1)-->(2) └── aggregations [outer=(1,2)] ├── sum [type=decimal, outer=(1)] - │ └── variable: a.k [type=int, outer=(1)] + │ └── variable: k [type=int, outer=(1)] └── sum [type=decimal, outer=(2)] └── agg-distinct [type=int, outer=(2)] - └── variable: a.i [type=int, outer=(2)] + └── variable: i [type=int, outer=(2)] # GroupBy with key argument. opt @@ -439,7 +439,7 @@ project │ └── fd: (1)-->(2) └── aggregations [outer=(1)] └── sum [type=decimal, outer=(1)] - └── variable: a.k [type=int, outer=(1)] + └── variable: k [type=int, outer=(1)] # GroupBy with no key. opt @@ -457,7 +457,7 @@ project └── aggregations [outer=(1)] └── sum [type=decimal, outer=(1)] └── agg-distinct [type=int, outer=(1)] - └── variable: abc.a [type=int, outer=(1)] + └── variable: a [type=int, outer=(1)] # GroupBy with composite key formed by argument plus grouping columns. opt @@ -475,7 +475,7 @@ project │ └── key: (1-3) └── aggregations [outer=(1)] └── sum [type=decimal, outer=(1)] - └── variable: abc.a [type=int, outer=(1)] + └── variable: a [type=int, outer=(1)] # GroupBy with multiple aggregations simplified. opt @@ -494,9 +494,9 @@ project │ └── fd: (1)-->(2,3), (2,3)~~>(1) └── aggregations [outer=(2,3)] ├── sum [type=decimal, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] └── avg [type=float, outer=(3)] - └── variable: a.f [type=float, outer=(3)] + └── variable: f [type=float, outer=(3)] # GroupBy where only some aggregations are simplified (the table has # keys u,v and v,w). @@ -516,9 +516,9 @@ project │ └── fd: (1,2)-->(3,4), (2,3)-->(1,4) └── aggregations [outer=(1,3,4)] ├── sum [type=decimal, outer=(1)] - │ └── variable: uvwz.u [type=int, outer=(1)] + │ └── variable: u [type=int, outer=(1)] ├── std-dev [type=decimal, outer=(3)] - │ └── variable: uvwz.w [type=int, outer=(3)] + │ └── variable: w [type=int, outer=(3)] └── avg [type=decimal, outer=(4)] └── agg-distinct [type=int, outer=(4)] - └── variable: uvwz.z [type=int, outer=(4)] + └── variable: z [type=int, outer=(4)] diff --git a/pkg/sql/opt/norm/testdata/rules/inline b/pkg/sql/opt/norm/testdata/rules/inline index 13fbe84005d9..1dd5d8b094dc 100644 --- a/pkg/sql/opt/norm/testdata/rules/inline +++ b/pkg/sql/opt/norm/testdata/rules/inline @@ -36,9 +36,9 @@ project │ │ ├── columns: k:1(int!null) │ │ └── key: (1) │ └── filters [type=bool, outer=(1)] - │ └── (a.k = 1) IS NULL [type=bool, outer=(1)] + │ └── (k = 1) IS NULL [type=bool, outer=(1)] └── projections [outer=(1)] - └── a.k = 1 [type=bool, outer=(1)] + └── k = 1 [type=bool, outer=(1)] # Inline arithmetic. opt @@ -53,9 +53,9 @@ project │ │ ├── columns: k:1(int!null) │ │ └── key: (1) │ └── filters [type=bool, outer=(1)] - │ └── ((a.k * 2) + 1) > 10 [type=bool, outer=(1)] + │ └── ((k * 2) + 1) > 10 [type=bool, outer=(1)] └── projections [outer=(1)] - └── (a.k * 2) + 1 [type=int, outer=(1)] + └── (k * 2) + 1 [type=int, outer=(1)] # Inline boolean logic. opt @@ -68,7 +68,7 @@ project │ ├── constraint: /1: [ - /1] [/6 - ] │ └── key: (1) └── projections [outer=(1)] - └── (a.k <= 1) OR (a.k > 5) [type=bool, outer=(1)] + └── (k <= 1) OR (k > 5) [type=bool, outer=(1)] # Inline constants. opt @@ -81,9 +81,9 @@ project │ ├── scan a │ │ └── columns: f:3(float) │ └── filters [type=bool, outer=(3)] - │ └── (a.f IS NULL) OR (a.f != 10.5) [type=bool, outer=(3)] + │ └── (f IS NULL) OR (f != 10.5) [type=bool, outer=(3)] └── projections [outer=(3)] - └── (a.f IS NULL) OR (a.f != 10.5) [type=bool, outer=(3)] + └── (f IS NULL) OR (f != 10.5) [type=bool, outer=(3)] # Reference the expression to inline multiple times. opt @@ -96,9 +96,9 @@ project │ ├── scan a │ │ └── columns: f:3(float) │ └── filters [type=bool, outer=(3)] - │ └── (a.f + 1.0) = (a.f + 1.0) [type=bool, outer=(3)] + │ └── (f + 1.0) = (f + 1.0) [type=bool, outer=(3)] └── projections [outer=(3)] - └── a.f + 1.0 [type=float, outer=(3)] + └── f + 1.0 [type=float, outer=(3)] # Use outer references in both inlined expression and in referencing expression. opt @@ -116,7 +116,7 @@ semi-join │ ├── columns: x:6(int!null) │ └── key: (6) └── filters [type=bool, outer=(2,6)] - └── (xy.x - a.i) > (a.i * a.i) [type=bool, outer=(2,6)] + └── (x - i) > (i * i) [type=bool, outer=(2,6)] # -------------------------------------------------- # InlineProjectInProject @@ -131,8 +131,8 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── projections [outer=(1,2)] - ├── a.i + 1 [type=int, outer=(2)] - └── a.k = 1 [type=bool, outer=(1)] + ├── i + 1 [type=int, outer=(2)] + └── k = 1 [type=bool, outer=(1)] # Inline multiple expressions. opt @@ -145,7 +145,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2,4) └── projections [outer=(1,2,4)] - ├── (a.k + 1) + 1 [type=int, outer=(1)] + ├── (k + 1) + 1 [type=int, outer=(1)] └── (a.s || 'foo') || 'bar' [type=string, outer=(4)] # Don't inline when there are multiple references. @@ -161,7 +161,7 @@ project │ │ ├── columns: k:1(int!null) │ │ └── key: (1) │ └── projections [outer=(1)] - │ └── a.k + 1 [type=int, outer=(1)] + │ └── k + 1 [type=int, outer=(1)] └── projections [outer=(6)] └── expr * 2 [type=int, outer=(6)] @@ -182,7 +182,7 @@ project │ ├── constraint: /7: [/1 - /2] │ ├── key: (7) │ └── fd: (7)-->(8) - └── (a.k + 1) * 2 [type=int, outer=(1)] + └── (k + 1) * 2 [type=int, outer=(1)] # Correlated subquery should be hoisted as usual. opt @@ -208,7 +208,7 @@ project │ │ │ │ ├── columns: k:1(int!null) │ │ │ │ └── key: (1) │ │ │ └── projections [outer=(1)] - │ │ │ └── a.k + 1 [type=int, outer=(1)] + │ │ │ └── k + 1 [type=int, outer=(1)] │ │ ├── project │ │ │ ├── columns: true:10(bool!null) │ │ │ ├── fd: ()-->(10) diff --git a/pkg/sql/opt/norm/testdata/rules/join b/pkg/sql/opt/norm/testdata/rules/join index ac84be070c86..072fe0629144 100644 --- a/pkg/sql/opt/norm/testdata/rules/join +++ b/pkg/sql/opt/norm/testdata/rules/join @@ -43,8 +43,8 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,2,6,7), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── b.y < a.i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── y < i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # -------------------------------------------------- # EnsureJoinFilters @@ -70,7 +70,7 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] opt SELECT * FROM a INNER JOIN b ON a.s='foo' OR b.y<10 @@ -88,7 +88,7 @@ inner-join │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(4,7)] - └── (a.s = 'foo') OR (b.y < 10) [type=bool, outer=(4,7)] + └── (s = 'foo') OR (y < 10) [type=bool, outer=(4,7)] # -------------------------------------------------- # PushFilterIntoJoinLeft @@ -110,9 +110,9 @@ inner-join (lookup b) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ └── filters [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] - │ └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] opt SELECT * FROM a RIGHT JOIN b ON (a.i<0 OR a.i>10) AND b.y=1 AND a.s='foo' AND a.k=b.x @@ -132,8 +132,8 @@ right-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 opt(4) │ └── filters [type=bool, outer=(2,4), constraints=(/4: [/'foo' - /'foo']), fd=()-->(4)] - │ ├── (a.i < 0) OR (a.i > 10) [type=bool, outer=(2)] - │ └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ ├── (i < 0) OR (i > 10) [type=bool, outer=(2)] + │ └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] ├── scan b │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -143,8 +143,8 @@ right-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6,7), constraints=(/1: (/NULL - ]; /6: (/NULL - ]; /7: [/1 - /1]), fd=()-->(7), (1)==(6), (6)==(1)] - ├── b.y = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── y = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # LEFT JOIN should not push down conditions to left side of join. opt @@ -168,8 +168,8 @@ left-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: [/1 - /1]; /6: (/NULL - ]), fd=()-->(2), (1)==(6), (6)==(1)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── a.i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] # Semi-join case. opt @@ -190,7 +190,7 @@ semi-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 opt(4) │ └── filters [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] - │ └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + │ └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] ├── scan b │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -200,7 +200,7 @@ semi-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── b.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Do not push anti-join conditions into left input. opt @@ -224,8 +224,8 @@ anti-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,4,6), constraints=(/1: (/NULL - ]; /4: [/'foo' - /'foo']; /6: (/NULL - ]), fd=()-->(4), (1)==(6), (6)==(1)] - ├── b.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + ├── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] # -------------------------------------------------- # PushFilterIntoJoinRight @@ -247,9 +247,9 @@ inner-join (lookup b) │ │ ├── key: (3) │ │ └── fd: (3)-->(4-7) │ └── filters [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] - │ └── a.s = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight)] + │ └── s = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight)] └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - └── b.x = a.k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + └── x = k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] opt SELECT * FROM b LEFT JOIN a ON (a.i<0 OR a.i>10) AND b.y=1 AND a.s='foo' AND b.x=a.k @@ -274,14 +274,14 @@ left-join (merge) │ │ ├── fd: (3)-->(4-7) │ │ └── ordering: +3 opt(6) │ └── filters [type=bool, outer=(4,6), constraints=(/6: [/'foo' - /'foo']), fd=()-->(6)] - │ ├── (a.i < 0) OR (a.i > 10) [type=bool, outer=(4)] - │ └── a.s = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight)] + │ ├── (i < 0) OR (i > 10) [type=bool, outer=(4)] + │ └── s = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight)] └── merge-on ├── left ordering: +1 ├── right ordering: +3 └── filters [type=bool, outer=(1-3), constraints=(/1: (/NULL - ]; /2: [/1 - /1]; /3: (/NULL - ]), fd=()-->(2), (1)==(3), (3)==(1)] - ├── b.y = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] - └── b.x = a.k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + ├── y = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + └── x = k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] # RIGHT JOIN should not push down conditions to right side of join. opt @@ -305,8 +305,8 @@ right-join (merge) ├── left ordering: +1 ├── right ordering: +3 └── filters [type=bool, outer=(1,3,4), constraints=(/1: (/NULL - ]; /3: (/NULL - ]; /4: [/1 - /1]), fd=()-->(4), (1)==(3), (3)==(1)] - ├── b.x = a.k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] - └── a.i = 1 [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight)] + ├── x = k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + └── i = 1 [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight)] # Semi-join case. opt @@ -332,12 +332,12 @@ semi-join (merge) │ │ ├── fd: (6)-->(7) │ │ └── ordering: +6 │ └── filters [type=bool, outer=(7), constraints=(/7: [/11 - ]; tight)] - │ └── b.y > 10 [type=bool, outer=(7), constraints=(/7: [/11 - ]; tight)] + │ └── y > 10 [type=bool, outer=(7), constraints=(/7: [/11 - ]; tight)] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── b.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Anti-join case. opt @@ -363,12 +363,12 @@ anti-join (merge) │ │ ├── fd: (6)-->(7) │ │ └── ordering: +6 │ └── filters [type=bool, outer=(7), constraints=(/7: [/11 - ]; tight)] - │ └── b.y > 10 [type=bool, outer=(7), constraints=(/7: [/11 - ]; tight)] + │ └── y > 10 [type=bool, outer=(7), constraints=(/7: [/11 - ]; tight)] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── b.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # ------------------------------------------------------------------------------- # PushFilterIntoJoinLeftAndRight + MapFilterIntoJoinLeft + MapFilterIntoJoinRight @@ -393,7 +393,7 @@ inner-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 │ └── filters [type=bool, outer=(1,2)] - │ └── (a.k * a.i) = 3 [type=bool, outer=(1,2)] + │ └── (k * i) = 3 [type=bool, outer=(1,2)] ├── select │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -405,12 +405,12 @@ inner-join (merge) │ │ ├── fd: (6)-->(7) │ │ └── ordering: +6 │ └── filters [type=bool, outer=(6,7)] - │ └── (b.x + b.y) > 5 [type=bool, outer=(6,7)] + │ └── (x + y) > 5 [type=bool, outer=(6,7)] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Multiple equivalent columns. # TODO(rytaft): We should also infer the equality predicates a.k=a.i and b.x=b.y. @@ -431,12 +431,12 @@ inner-join (lookup b) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ └── filters [type=bool, outer=(1,3,4)] - │ ├── (a.f + a.k::FLOAT) > 5.0 [type=bool, outer=(1,3)] - │ └── (a.s || a.k::STRING) = 'foo1' [type=bool, outer=(1,4)] + │ ├── (f + k::FLOAT) > 5.0 [type=bool, outer=(1,3)] + │ └── (s || k::STRING) = 'foo1' [type=bool, outer=(1,4)] └── filters [type=bool, outer=(1,2,6,7), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(2,6,7), (6)==(1,2,7), (2)==(1,6,7), (7)==(1,2,6)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - ├── a.i = b.x [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] - └── a.i = b.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── i = x [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + └── i = y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Can push to both sides with semi-join. opt @@ -459,7 +459,7 @@ semi-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 │ └── filters [type=bool, outer=(1,2)] - │ └── (a.k * a.i) = 3 [type=bool, outer=(1,2)] + │ └── (k * i) = 3 [type=bool, outer=(1,2)] ├── select │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -471,12 +471,12 @@ semi-join (merge) │ │ ├── fd: (6)-->(7) │ │ └── ordering: +6 │ └── filters [type=bool, outer=(6,7)] - │ └── (b.x + b.y) > 5 [type=bool, outer=(6,7)] + │ └── (x + y) > 5 [type=bool, outer=(6,7)] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] opt SELECT * FROM a WHERE EXISTS( @@ -503,7 +503,7 @@ semi-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Can only push to right side with left join. opt @@ -529,13 +529,13 @@ left-join (merge) │ │ ├── fd: (6)-->(7) │ │ └── ordering: +6 │ └── filters [type=bool, outer=(6,7)] - │ └── (b.x + b.y) > 5 [type=bool, outer=(6,7)] + │ └── (x + y) > 5 [type=bool, outer=(6,7)] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── (b.x * a.i) = 3 [type=bool, outer=(2,6)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── (x * i) = 3 [type=bool, outer=(2,6)] opt SELECT * FROM a LEFT JOIN b ON a.k=b.x AND a.k > 5 AND b.x IN (3, 7, 10) @@ -559,7 +559,7 @@ left-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Can only push to left side with right join. opt @@ -580,7 +580,7 @@ right-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 │ └── filters [type=bool, outer=(1,2)] - │ └── (a.k * a.i) = 3 [type=bool, outer=(1,2)] + │ └── (k * i) = 3 [type=bool, outer=(1,2)] ├── scan b │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -590,8 +590,8 @@ right-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6,7), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── (a.k + b.y) > 5 [type=bool, outer=(1,7)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── (k + y) > 5 [type=bool, outer=(1,7)] opt SELECT * FROM a RIGHT JOIN b ON a.k=b.x AND a.k > 5 AND b.x IN (3, 7, 10) @@ -615,7 +615,7 @@ right-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Cannot push with full join. opt @@ -639,9 +639,9 @@ full-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,2,6,7), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - ├── (a.k + b.y) > 5 [type=bool, outer=(1,7)] - └── (b.x * a.i) = 3 [type=bool, outer=(2,6)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── (k + y) > 5 [type=bool, outer=(1,7)] + └── (x * i) = 3 [type=bool, outer=(2,6)] opt SELECT * FROM a FULL JOIN b ON a.k=b.x AND a.k > 5 AND b.x IN (3, 7, 10) @@ -664,9 +664,9 @@ full-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: [/6 - ]; /6: [/3 - /3] [/7 - /7] [/10 - /10]), fd=(1)==(6), (6)==(1)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - ├── a.k > 5 [type=bool, outer=(1), constraints=(/1: [/6 - ]; tight)] - └── b.x IN (3, 7, 10) [type=bool, outer=(6), constraints=(/6: [/3 - /3] [/7 - /7] [/10 - /10]; tight)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── k > 5 [type=bool, outer=(1), constraints=(/1: [/6 - ]; tight)] + └── x IN (3, 7, 10) [type=bool, outer=(6), constraints=(/6: [/3 - /3] [/7 - /7] [/10 - /10]; tight)] # Can only push to right side with anti-join. opt @@ -694,13 +694,13 @@ anti-join (merge) │ │ ├── fd: (6)-->(7) │ │ └── ordering: +6 │ └── filters [type=bool, outer=(6,7)] - │ └── (b.x + b.y) > 5 [type=bool, outer=(6,7)] + │ └── (x + y) > 5 [type=bool, outer=(6,7)] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── (b.x * a.i) = 3 [type=bool, outer=(2,6)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── (x * i) = 3 [type=bool, outer=(2,6)] opt SELECT * FROM a WHERE NOT EXISTS( @@ -726,7 +726,7 @@ anti-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Works with a non-correlated subquery. opt @@ -748,7 +748,7 @@ inner-join (merge) │ │ └── ordering: +1 │ └── filters [type=bool, outer=(1,2)] │ └── eq [type=bool, outer=(1,2)] - │ ├── a.k * a.i [type=int, outer=(1,2)] + │ ├── k * i [type=int, outer=(1,2)] │ └── subquery [type=int] │ └── scalar-group-by │ ├── columns: min:10(int) @@ -772,7 +772,7 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Optimization does not apply with correlated suqueries. opt @@ -827,17 +827,17 @@ project │ ├── const-agg [type=int, outer=(7)] │ │ └── variable: b.y [type=int, outer=(7)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ ├── const-agg [type=jsonb, outer=(5)] - │ │ └── variable: a.j [type=jsonb, outer=(5)] + │ │ └── variable: j [type=jsonb, outer=(5)] │ └── const-agg [type=int, outer=(1)] │ └── variable: a.k [type=int, outer=(1)] └── filters [type=bool, outer=(2,6,11)] - └── (b.x * a.i) = min [type=bool, outer=(2,6,11)] + └── (b.x * i) = min [type=bool, outer=(2,6,11)] # Ensure that we do not map filters for types with composite key encoding. opt @@ -887,7 +887,7 @@ inner-join │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ └── filters [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] - │ └── a.k = a.i [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] + │ └── k = i [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] ├── select │ ├── columns: x:6(int!null) y:7(int!null) │ ├── key: (6) @@ -897,10 +897,10 @@ inner-join │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] - │ └── b.y = b.x [type=bool, outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ])] + │ └── y = x [type=bool, outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ])] └── filters [type=bool, outer=(1,2,6,7)] - ├── (a.k + b.y) > 5 [type=bool, outer=(1,7)] - └── (b.x * a.i) = 3 [type=bool, outer=(2,6)] + ├── (k + y) > 5 [type=bool, outer=(1,7)] + └── (x * i) = 3 [type=bool, outer=(2,6)] exec-ddl CREATE TABLE t1 (a DATE) @@ -935,9 +935,9 @@ inner-join │ ├── scan t2 │ │ └── columns: b:3(timestamptz) │ └── filters [type=bool, outer=(3)] - │ └── age(t2.b, '2017-01-01 00:00:00+00:00') > '1d' [type=bool, outer=(3)] + │ └── age(b, '2017-01-01 00:00:00+00:00') > '1d' [type=bool, outer=(3)] └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - └── t1.a = t2.b [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + └── a = b [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] # -------------------------------------------------- # PushFilterIntoJoinLeft + PushFilterIntoJoinRight @@ -960,10 +960,10 @@ inner-join (lookup a) │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] - │ └── b.y = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] + │ └── y = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] └── filters [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: [/1 - /1]; /6: (/NULL - ]), fd=()-->(2), (1)==(6), (6)==(1)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── a.i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] # FULL JOIN should not push down conditions to either side of join. opt @@ -987,9 +987,9 @@ full-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,2,6,7), constraints=(/1: (/NULL - ]; /2: [/1 - /1]; /6: (/NULL - ]; /7: [/1 - /1]), fd=()-->(2,7), (1)==(6), (6)==(1)] - ├── a.k = b.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - ├── a.i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] - └── b.y = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + └── y = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] # Nested semi/anti-join case. opt @@ -1033,9 +1033,9 @@ semi-join-apply │ │ └── filters [type=bool, outer=(9), constraints=(/9: [/10 - /10]; tight), fd=()-->(9)] │ │ └── a.i = 10 [type=bool, outer=(9), constraints=(/9: [/10 - /10]; tight)] │ └── filters [type=bool, outer=(2), constraints=(/2: [/101 - ]; tight)] - │ └── b.y > 100 [type=bool, outer=(2), constraints=(/2: [/101 - ]; tight)] + │ └── y > 100 [type=bool, outer=(2), constraints=(/2: [/101 - ]; tight)] └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - └── a.k = b.x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + └── a.k = x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] # -------------------------------------------------- # SimplifyLeftJoinWithoutFilters @@ -1112,20 +1112,20 @@ project │ │ │ ├── columns: column1:6(int) │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [2 - 2] - │ │ │ ├── (a.k,) [type=tuple{int}, outer=(1)] + │ │ │ ├── (k,) [type=tuple{int}, outer=(1)] │ │ │ └── (1,) [type=tuple{int}] │ │ └── true [type=bool] │ └── aggregations [outer=(2-6)] │ ├── sum [type=decimal, outer=(6)] │ │ └── variable: column1 [type=int, outer=(6)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ └── const-agg [type=jsonb, outer=(5)] - │ └── variable: a.j [type=jsonb, outer=(5)] + │ └── variable: j [type=jsonb, outer=(5)] └── filters [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] └── sum = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] @@ -1528,7 +1528,7 @@ full-join │ └── fd: (8)-->(9-12) └── filters [type=bool, outer=(1,6,8), constraints=(/1: (/NULL - ]; /6: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(6,8), (8)==(1,6), (6)==(1,8)] ├── a.k = a.k [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] - └── b.x = a.k [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] + └── x = a.k [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] # Can't simplify: The a2.x column is not part of unfilteredCols. opt @@ -1556,7 +1556,7 @@ right-join │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] - └── a.k = b.x [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ])] + └── a.k = x [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ])] # -------------------------------------------------- # EliminateSemiJoin @@ -1592,7 +1592,7 @@ select ├── scan a │ └── columns: s:6(string) └── filters [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] - └── a.s = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight)] + └── s = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight)] # -------------------------------------------------- # EliminateJoinNoColsRight @@ -1606,7 +1606,7 @@ select ├── scan a │ └── columns: s:4(string) └── filters [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] - └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] # -------------------------------------------------- # HoistJoinProject @@ -1635,9 +1635,9 @@ project │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] - │ └── b.y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] + │ └── y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── b.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Left-join case. opt @@ -1667,9 +1667,9 @@ project │ │ ├── fd: (6)-->(7) │ │ └── ordering: +6 opt(7) │ └── filters [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] - │ └── b.y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] + │ └── y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── b.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] diff --git a/pkg/sql/opt/norm/testdata/rules/limit b/pkg/sql/opt/norm/testdata/rules/limit index 484f07407d25..6dcf1e15195d 100644 --- a/pkg/sql/opt/norm/testdata/rules/limit +++ b/pkg/sql/opt/norm/testdata/rules/limit @@ -109,7 +109,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(3) └── projections [outer=(1,3)] - └── a.f * 2.0 [type=float, outer=(3)] + └── f * 2.0 [type=float, outer=(3)] opt SELECT k, f*2.0 AS r FROM a ORDER BY k LIMIT 5 @@ -127,7 +127,7 @@ project │ ├── fd: (1)-->(3) │ └── ordering: +1 └── projections [outer=(1,3)] - └── a.f * 2.0 [type=float, outer=(3)] + └── f * 2.0 [type=float, outer=(3)] # Don't push the limit through project when the ordering is on a # synthesized column. @@ -155,7 +155,7 @@ limit │ │ ├── key: (1) │ │ └── fd: (1)-->(3) │ └── projections [outer=(1,3)] - │ └── a.f * 2.0 [type=float, outer=(3)] + │ └── f * 2.0 [type=float, outer=(3)] └── const: 5 [type=int] @@ -185,7 +185,7 @@ project │ │ └── columns: i:2(int) f:3(float) │ └── const: 5 [type=int] └── projections [outer=(3)] - └── a.f + 1.1 [type=float, outer=(3)] + └── f + 1.1 [type=float, outer=(3)] # Don't push zero limit into Scan. opt @@ -237,7 +237,7 @@ project │ │ └── fd: (1)-->(3) │ └── const: 5 [type=int] └── projections [outer=(1,3)] - └── a.f * 2.0 [type=float, outer=(3)] + └── f * 2.0 [type=float, outer=(3)] opt SELECT k, f*2.0 AS r FROM a ORDER BY k OFFSET 5 @@ -260,7 +260,7 @@ project │ │ └── ordering: +1 │ └── const: 5 [type=int] └── projections [outer=(1,3)] - └── a.f * 2.0 [type=float, outer=(3)] + └── f * 2.0 [type=float, outer=(3)] # Don't push the offset through project when the ordering is on a # synthesized column. @@ -287,7 +287,7 @@ offset │ │ ├── key: (1) │ │ └── fd: (1)-->(3) │ └── projections [outer=(1,3)] - │ └── a.f * 2.0 [type=float, outer=(3)] + │ └── f * 2.0 [type=float, outer=(3)] └── const: 5 [type=int] # Detect PushOffsetIntoProject and FilterUnusedOffsetCols dependency cycle. @@ -314,7 +314,7 @@ project │ │ └── columns: i:2(int) f:3(float) │ └── const: 5 [type=int] └── projections [outer=(3)] - └── a.f + 1.1 [type=float, outer=(3)] + └── f + 1.1 [type=float, outer=(3)] # -------------------------------------------------- # PushLimitIntoProject + PushOffsetIntoProject @@ -343,7 +343,7 @@ project │ │ └── const: 5 [type=int] │ └── const: 10 [type=int] └── projections [outer=(1,3)] - └── a.f * 2.0 [type=float, outer=(3)] + └── f * 2.0 [type=float, outer=(3)] opt SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i) a ORDER BY f OFFSET 5 LIMIT 10 @@ -376,4 +376,4 @@ project │ │ └── const: 5 [type=int] │ └── const: 10 [type=int] └── projections [outer=(3)] - └── a.f + 1.1 [type=float, outer=(3)] + └── f + 1.1 [type=float, outer=(3)] diff --git a/pkg/sql/opt/norm/testdata/rules/numeric b/pkg/sql/opt/norm/testdata/rules/numeric index f159dd069e54..e080c573f9a4 100644 --- a/pkg/sql/opt/norm/testdata/rules/numeric +++ b/pkg/sql/opt/norm/testdata/rules/numeric @@ -27,12 +27,12 @@ project ├── scan a │ └── columns: i:2(int) f:3(float) d:4(decimal) └── projections [outer=(2-4)] - ├── a.i + a.i [type=int, outer=(2)] - ├── a.i + a.i [type=int, outer=(2)] - ├── a.f + a.f [type=float, outer=(3)] - ├── a.f + a.f [type=float, outer=(3)] - ├── a.d + a.d [type=decimal, outer=(4)] - └── a.d + a.d [type=decimal, outer=(4)] + ├── i + i [type=int, outer=(2)] + ├── i + i [type=int, outer=(2)] + ├── f + f [type=float, outer=(3)] + ├── f + f [type=float, outer=(3)] + ├── d + d [type=decimal, outer=(4)] + └── d + d [type=decimal, outer=(4)] # -------------------------------------------------- # FoldMinusZero @@ -51,9 +51,9 @@ project ├── scan a │ └── columns: i:2(int) f:3(float) d:4(decimal) └── projections [outer=(2-4)] - ├── a.i + a.i [type=int, outer=(2)] - ├── a.f + a.f [type=float, outer=(3)] - └── a.d + a.d [type=decimal, outer=(4)] + ├── i + i [type=int, outer=(2)] + ├── f + f [type=float, outer=(3)] + └── d + d [type=decimal, outer=(4)] # -------------------------------------------------- # FoldMultOne, FoldOneMult @@ -72,12 +72,12 @@ project ├── scan a │ └── columns: i:2(int) f:3(float) d:4(decimal) └── projections [outer=(2-4)] - ├── a.i + a.i [type=int, outer=(2)] - ├── a.i + a.i [type=int, outer=(2)] - ├── a.f + a.f [type=float, outer=(3)] - ├── a.f + a.f [type=float, outer=(3)] - ├── a.d + a.d [type=decimal, outer=(4)] - └── a.d + a.d [type=decimal, outer=(4)] + ├── i + i [type=int, outer=(2)] + ├── i + i [type=int, outer=(2)] + ├── f + f [type=float, outer=(3)] + ├── f + f [type=float, outer=(3)] + ├── d + d [type=decimal, outer=(4)] + └── d + d [type=decimal, outer=(4)] # -------------------------------------------------- # FoldDivOne @@ -95,9 +95,9 @@ project ├── scan a │ └── columns: i:2(int) f:3(float) d:4(decimal) └── projections [outer=(2-4)] - ├── variable: a.i [type=int, outer=(2)] - ├── variable: a.f [type=float, outer=(3)] - └── variable: a.d [type=decimal, outer=(4)] + ├── variable: i [type=int, outer=(2)] + ├── variable: f [type=float, outer=(3)] + └── variable: d [type=decimal, outer=(4)] # -------------------------------------------------- # InvertMinus @@ -114,8 +114,8 @@ project ├── scan a │ └── columns: i:2(int) f:3(float) d:4(decimal) a.t:5(time) └── projections [outer=(2-5)] - ├── a.f - a.f [type=float, outer=(3)] - ├── a.i - a.d [type=decimal, outer=(2,4)] + ├── f - f [type=float, outer=(3)] + ├── i - d [type=decimal, outer=(2,4)] └── a.t - a.t [type=interval, outer=(5)] # -------------------------------------------------- @@ -129,7 +129,7 @@ project ├── scan a │ └── columns: i:2(int) └── projections [outer=(2)] - └── variable: a.i [type=int, outer=(2)] + └── variable: i [type=int, outer=(2)] # -------------------------------------------------- # FoldUnaryMinus diff --git a/pkg/sql/opt/norm/testdata/rules/ordering b/pkg/sql/opt/norm/testdata/rules/ordering index 69d189927629..f13f02fe04b0 100644 --- a/pkg/sql/opt/norm/testdata/rules/ordering +++ b/pkg/sql/opt/norm/testdata/rules/ordering @@ -65,8 +65,8 @@ limit │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5), (2,3)~~>(1,4,5) │ └── filters [type=bool, outer=(4,5), constraints=(/4: [/1 - /1]; /5: [/2 - /2]; tight), fd=()-->(4,5)] - │ ├── abcde.d = 1 [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight)] - │ └── abcde.e = 2 [type=bool, outer=(5), constraints=(/5: [/2 - /2]; tight)] + │ ├── d = 1 [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight)] + │ └── e = 2 [type=bool, outer=(5), constraints=(/5: [/2 - /2]; tight)] └── const: 10 [type=int] # Remove functionally dependent column that's only used in ordering. @@ -116,9 +116,9 @@ sort │ └── fd: (1)-->(2,3), (2,3)~~>(1) └── aggregations [outer=(2,3)] ├── array-agg [type=int[], outer=(2)] - │ └── variable: abcde.b [type=int, outer=(2)] + │ └── variable: b [type=int, outer=(2)] └── const-agg [type=int, outer=(3)] - └── variable: abcde.c [type=int, outer=(3)] + └── variable: c [type=int, outer=(3)] # ScalarGroupBy case. opt @@ -137,7 +137,7 @@ scalar-group-by │ └── ordering: +1 └── aggregations [outer=(2)] └── array-agg [type=int[], outer=(2)] - └── variable: abcde.b [type=int, outer=(2)] + └── variable: b [type=int, outer=(2)] # DistinctOn case. opt @@ -157,7 +157,7 @@ distinct-on │ └── ordering: +2,+3,+1 └── aggregations [outer=(1)] └── first-agg [type=int, outer=(1)] - └── variable: abcde.a [type=int, outer=(1)] + └── variable: a [type=int, outer=(1)] # -------------------------------------------------- # SimplifyRowNumberOrdering @@ -184,8 +184,8 @@ sort │ ├── key: (1) │ └── fd: (1)-->(2-5), (2,3)~~>(1,4,5) └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]; tight)] - ├── abcde.b IS NOT NULL [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] - └── abcde.c IS NOT NULL [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] + ├── b IS NOT NULL [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] + └── c IS NOT NULL [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] # -------------------------------------------------- # SimplifyExplainOrdering @@ -204,7 +204,7 @@ explain │ ├── columns: b:2(int) c:3(int) │ └── ordering: +2,+3 └── projections [outer=(2,3)] - └── abcde.b + 1 [type=int, outer=(2)] + └── b + 1 [type=int, outer=(2)] # Regression: Explain a statement having constant column, but with no ordering. opt diff --git a/pkg/sql/opt/norm/testdata/rules/project b/pkg/sql/opt/norm/testdata/rules/project index b5d79574a869..249e0564c1df 100644 --- a/pkg/sql/opt/norm/testdata/rules/project +++ b/pkg/sql/opt/norm/testdata/rules/project @@ -92,4 +92,4 @@ project │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ └── a.x = b.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] └── projections [outer=(2)] - └── a.y + 1 [type=int, outer=(2)] + └── y + 1 [type=int, outer=(2)] diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index db71475b7b19..3eadb39a5c6f 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -75,7 +75,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── projections [outer=(1,2)] - └── a.k + a.i [type=int, outer=(1,2)] + └── k + i [type=int, outer=(1,2)] # Discard non-computed columns and keep computed column. opt @@ -90,7 +90,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(4) └── projections [outer=(1,4)] - └── length(a.s) [type=int, outer=(4)] + └── length(s) [type=int, outer=(4)] # Compute column based on another computed column. opt @@ -109,7 +109,7 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(4) │ └── projections [outer=(1,4)] - │ └── length(a.s) [type=int, outer=(4)] + │ └── length(s) [type=int, outer=(4)] └── projections [outer=(1,5)] └── l * l [type=int, outer=(5)] @@ -138,8 +138,8 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── projections [outer=(1,2)] - ├── a.k + 1 [type=int, outer=(1)] - └── a.i + 1 [type=int, outer=(2)] + ├── k + 1 [type=int, outer=(1)] + └── i + 1 [type=int, outer=(2)] # Use columns only in computed columns. opt @@ -152,7 +152,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── projections [outer=(1,2)] - └── a.k + a.i [type=int, outer=(1,2)] + └── k + i [type=int, outer=(1,2)] # Use no scan columns. opt @@ -185,7 +185,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] - └── a.i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] + └── i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] # Columns used by both projection and filter. opt @@ -203,7 +203,7 @@ select │ ├── key: () │ └── fd: ()-->(1,2) └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] - └── a.i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] + └── i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] # No needed select columns. opt @@ -234,11 +234,11 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2,4) │ └── filters [type=bool, outer=(1,4), constraints=(/1: (/NULL - ])] - │ ├── a.k < (5 - 1) [type=bool, outer=(1), constraints=(/1: (/NULL - ])] - │ └── (a.s || 'o') = 'foo' [type=bool, outer=(4)] + │ ├── k < (5 - 1) [type=bool, outer=(1), constraints=(/1: (/NULL - ])] + │ └── (s || 'o') = 'foo' [type=bool, outer=(4)] └── projections [outer=(1,2)] - ├── a.i - 1 [type=int, outer=(2)] - └── a.k * a.k [type=int, outer=(1)] + ├── i - 1 [type=int, outer=(2)] + └── k * k [type=int, outer=(1)] # Select nested in select. opt @@ -271,7 +271,7 @@ project │ └── projections [outer=(2,3), side-effects] │ └── a.f / 2.0 [type=float, outer=(3), side-effects] └── filters [type=bool, outer=(2,5), constraints=(/5: (/NULL - ])] - └── f = a.i::FLOAT [type=bool, outer=(2,5), constraints=(/5: (/NULL - ])] + └── f = i::FLOAT [type=bool, outer=(2,5), constraints=(/5: (/NULL - ])] # Detect PruneSelectCols and PushSelectIntoProject dependency cycle. opt @@ -294,13 +294,13 @@ project │ │ │ └── fd: (1)-->(3) │ │ └── aggregations [outer=(1,3)] │ │ ├── sum [type=decimal, outer=(1)] - │ │ │ └── variable: a.k [type=int, outer=(1)] + │ │ │ └── variable: k [type=int, outer=(1)] │ │ └── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ └── filters [type=bool, outer=(5), constraints=(/5: [/100 - /100]; tight), fd=()-->(5)] │ └── column5 = 100 [type=bool, outer=(5), constraints=(/5: [/100 - /100]; tight)] └── projections [outer=(3)] - └── a.f + 1.1 [type=float, outer=(3)] + └── f + 1.1 [type=float, outer=(3)] # -------------------------------------------------- # PruneLimitCols @@ -410,7 +410,7 @@ project │ │ └── columns: f:3(float) s:4(string) │ └── const: 5 [type=int] └── projections [outer=(3)] - └── a.f * 2.0 [type=float, outer=(3)] + └── f * 2.0 [type=float, outer=(3)] # -------------------------------------------------- # PruneOffsetCols @@ -643,7 +643,7 @@ project │ │ └── const: 5 [type=int] │ └── const: 5 [type=int] └── projections [outer=(3)] - └── a.f * 2.0 [type=float, outer=(3)] + └── f * 2.0 [type=float, outer=(3)] # -------------------------------------------------- # PruneJoinLeftCols @@ -675,7 +675,7 @@ project ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── k = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] # Columns used by both projection and on condition, left join. opt @@ -699,8 +699,8 @@ left-join (merge) ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,2,5), constraints=(/1: (/NULL - ]; /2: (/NULL - /4]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - ├── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── a.i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] + ├── k = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] # Columns only used by on condition, right join opt @@ -726,7 +726,7 @@ project ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── k = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] # Columns needed only by projection, full join. opt @@ -748,7 +748,7 @@ project │ │ └── fd: (5)-->(6) │ └── true [type=bool] └── projections [outer=(1,5,6)] - └── a.k + 1 [type=int, outer=(1)] + └── k + 1 [type=int, outer=(1)] # No columns needed from left side of join. opt @@ -791,10 +791,10 @@ project │ │ └── filters [type=bool, outer=(4)] │ │ └── (a.s || 'o') = 'foo' [type=bool, outer=(4)] │ └── filters [type=bool, outer=(1,5), constraints=(/5: (/NULL - ])] - │ └── xy.x = (a.k * a.k) [type=bool, outer=(1,5), constraints=(/5: (/NULL - ])] + │ └── x = (k * k) [type=bool, outer=(1,5), constraints=(/5: (/NULL - ])] └── projections [outer=(1,2,5,6), side-effects] - ├── a.k + 1 [type=int, outer=(1)] - └── a.i / 2 [type=decimal, outer=(2), side-effects] + ├── k + 1 [type=int, outer=(1)] + └── i / 2 [type=decimal, outer=(2), side-effects] # Join that is nested in another join. opt @@ -831,13 +831,13 @@ project │ ├── left ordering: +1 │ ├── right ordering: +5 │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + │ └── k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] ├── scan xy │ ├── columns: xy.x:7(int!null) xy.y:8(int) │ ├── key: (7) │ └── fd: (7)-->(8) └── filters [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] - └── a.i < xy.y [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] + └── i < xy.y [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] # ApplyJoin operator. opt @@ -880,9 +880,9 @@ project │ │ │ ├── scan xy │ │ │ │ └── columns: y:6(int) │ │ │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ │ └── xy.y = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ │ │ └── y = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] │ │ └── projections [outer=(1)] - │ │ └── a.k + 1 [type=int, outer=(1)] + │ │ └── k + 1 [type=int, outer=(1)] │ └── filters [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] │ └── r = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] └── true [type=bool] @@ -920,7 +920,7 @@ project │ ├── left ordering: +1 │ ├── right ordering: +7 │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ └── k = xy.x [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] ├── scan xy │ ├── columns: xy.x:5(int!null) │ ├── key: (5) @@ -929,7 +929,7 @@ project ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] # AntiJoin operator. opt @@ -964,7 +964,7 @@ project │ ├── left ordering: +1 │ ├── right ordering: +7 │ └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + │ └── k = xy.x [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] ├── scan xy │ ├── columns: xy.x:5(int!null) │ ├── key: (5) @@ -973,7 +973,7 @@ project ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] # -------------------------------------------------- # PruneJoinRightCols @@ -1005,7 +1005,7 @@ project ├── left ordering: +1 ├── right ordering: +3 └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - └── xy.x = a.k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + └── x = k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] # Columns used by both projection and on condition, left join. opt @@ -1031,12 +1031,12 @@ left-join (merge) │ │ ├── fd: (3)-->(4) │ │ └── ordering: +3 │ └── filters [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ])] - │ └── a.i < a.k [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ])] + │ └── i < k [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ])] └── merge-on ├── left ordering: +1 ├── right ordering: +3 └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - └── xy.x = a.k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + └── x = k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] # Columns only used by on condition, right join opt @@ -1062,7 +1062,7 @@ project ├── left ordering: +1 ├── right ordering: +3 └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - └── xy.x = a.k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + └── x = k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] # Columns needed only by projection, full join. opt @@ -1084,7 +1084,7 @@ project │ │ └── key: (3) │ └── true [type=bool] └── projections [outer=(1-3)] - └── a.k + 1 [type=int, outer=(3)] + └── k + 1 [type=int, outer=(3)] # No columns needed from right side of join. opt @@ -1127,10 +1127,10 @@ project │ │ └── filters [type=bool, outer=(6)] │ │ └── (a.s || 'o') = 'foo' [type=bool, outer=(6)] │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ])] - │ └── xy.x = (a.k * a.k) [type=bool, outer=(1,3), constraints=(/1: (/NULL - ])] + │ └── x = (k * k) [type=bool, outer=(1,3), constraints=(/1: (/NULL - ])] └── projections [outer=(1-4), side-effects] - ├── a.k + 1 [type=int, outer=(3)] - └── a.i / 2 [type=decimal, outer=(4), side-effects] + ├── k + 1 [type=int, outer=(3)] + └── i / 2 [type=decimal, outer=(4), side-effects] # Join that is nested in another join. opt @@ -1171,7 +1171,7 @@ project │ ├── left ordering: +3 │ ├── right ordering: +7 │ └── filters [type=bool, outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] - │ └── a.k = xy.x [type=bool, outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ])] + │ └── k = xy.x [type=bool, outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ])] └── filters [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] └── xy.y < xy.y [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] @@ -1216,9 +1216,9 @@ project │ ├── left ordering: +1 │ ├── right ordering: +5 │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + │ └── k = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] └── projections [outer=(1,5)] - └── a.k + xy.x [type=int, outer=(1,5)] + └── k + x [type=int, outer=(1,5)] # -------------------------------------------------- # PruneAggCols @@ -1248,7 +1248,7 @@ group-by │ └── columns: i:2(int) s:4(string) └── aggregations [outer=(2)] └── sum [type=decimal, outer=(2)] - └── variable: a.i [type=int, outer=(2)] + └── variable: i [type=int, outer=(2)] # No aggregates to discard. opt @@ -1280,10 +1280,10 @@ scalar-group-by │ ├── scan a │ │ └── columns: i:2(int) s:4(string) │ └── projections [outer=(2,4)] - │ └── a.s || 'foo' [type=string, outer=(4)] + │ └── s || 'foo' [type=string, outer=(4)] └── aggregations [outer=(2)] └── sum [type=decimal, outer=(2)] - └── variable: a.i [type=int, outer=(2)] + └── variable: i [type=int, outer=(2)] opt SELECT f FROM (SELECT DISTINCT ON (i) f, s FROM a) @@ -1299,7 +1299,7 @@ project │ └── columns: i:2(int) f:3(float) └── aggregations [outer=(3)] └── first-agg [type=float, outer=(3)] - └── variable: a.f [type=float, outer=(3)] + └── variable: f [type=float, outer=(3)] # -------------------------------------------------- # PruneGroupByCols @@ -1318,7 +1318,7 @@ group-by │ └── columns: i:2(int) s:4(string) └── aggregations [outer=(2)] └── sum [type=decimal, outer=(2)] - └── variable: a.i [type=int, outer=(2)] + └── variable: i [type=int, outer=(2)] # Columns used by both grouping and aggregation. opt @@ -1335,7 +1335,7 @@ group-by │ ├── scan a │ │ └── columns: i:2(int) s:4(string) │ └── projections [outer=(2,4)] - │ └── a.i + a.s::INT [type=int, outer=(2,4)] + │ └── i + s::INT [type=int, outer=(2,4)] └── aggregations [outer=(5)] └── avg [type=decimal, outer=(5)] └── variable: column5 [type=int, outer=(5)] @@ -1355,9 +1355,9 @@ scalar-group-by │ └── fd: (1)-->(2) └── aggregations [outer=(1,2)] ├── min [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] └── max [type=int, outer=(1)] - └── variable: a.k [type=int, outer=(1)] + └── variable: k [type=int, outer=(1)] # Columns used only by groupings, no aggregation columns. opt @@ -1372,7 +1372,7 @@ project │ └── scan a │ └── columns: i:2(int) s:4(string) └── projections [outer=(2,4)] - └── a.i + 1 [type=int, outer=(2)] + └── i + 1 [type=int, outer=(2)] # Groupby a groupby. opt @@ -1394,7 +1394,7 @@ group-by │ │ └── fd: (1)-->(2,4) │ └── aggregations [outer=(1)] │ └── sum [type=decimal, outer=(1)] - │ └── variable: a.k [type=int, outer=(1)] + │ └── variable: k [type=int, outer=(1)] └── aggregations [outer=(5)] └── min [type=decimal, outer=(5)] └── variable: sm [type=decimal, outer=(5)] @@ -1485,7 +1485,7 @@ project │ │ ├── fd: (1)-->(2) │ │ └── ordering: +1 │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] - │ └── a.i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] + │ └── i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] ├── scan xy │ ├── columns: x:5(int!null) y:6(int) │ ├── key: (5) @@ -1495,7 +1495,7 @@ project ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── a.k = xy.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── k = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] opt SELECT k FROM (SELECT k, min(s) FROM a GROUP BY k HAVING sum(i) > 5) @@ -1518,7 +1518,7 @@ project │ │ └── fd: (1)-->(2) │ └── aggregations [outer=(2)] │ └── sum [type=decimal, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] └── filters [type=bool, outer=(5), constraints=(/5: (/5 - ]; tight)] └── column5 > 5 [type=bool, outer=(5), constraints=(/5: (/5 - ]; tight)] diff --git a/pkg/sql/opt/norm/testdata/rules/reject_nulls b/pkg/sql/opt/norm/testdata/rules/reject_nulls index 447e3fbffc30..d084183f5233 100644 --- a/pkg/sql/opt/norm/testdata/rules/reject_nulls +++ b/pkg/sql/opt/norm/testdata/rules/reject_nulls @@ -135,7 +135,7 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── xy.x = a.k [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── x = k [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] # Right-join operator. opt @@ -154,7 +154,7 @@ inner-join │ ├── key: (5) │ └── fd: (5)-->(6) └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── a.k > xy.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k > y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Full-join operator. opt @@ -221,9 +221,9 @@ right-join │ │ │ ├── key: (7) │ │ │ └── fd: (7)-->(8) │ │ └── filters [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] - │ │ └── uv.v = a.k [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] + │ │ └── v = k [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ └── a.i = uv.u [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ └── i = u [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] └── true [type=bool] # ---------------------------------------------------------- @@ -263,7 +263,7 @@ project │ │ └── true [type=bool] │ └── aggregations [outer=(5)] │ └── max [type=int, outer=(5)] - │ └── variable: xy.x [type=int, outer=(5)] + │ └── variable: x [type=int, outer=(5)] └── filters [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] └── column7 = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] @@ -300,9 +300,9 @@ project │ │ └── true [type=bool] │ └── aggregations [outer=(5)] │ ├── min [type=int, outer=(5)] - │ │ └── variable: xy.x [type=int, outer=(5)] + │ │ └── variable: x [type=int, outer=(5)] │ └── max [type=int, outer=(5)] - │ └── variable: xy.x [type=int, outer=(5)] + │ └── variable: x [type=int, outer=(5)] └── filters [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] └── column7 = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] @@ -340,18 +340,18 @@ project │ │ ├── left ordering: +1 │ │ ├── right ordering: +5 │ │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ │ └── xy.x = a.k [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + │ │ └── x = k [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] │ └── aggregations [outer=(2-5)] │ ├── sum [type=decimal, outer=(5)] - │ │ └── variable: xy.x [type=int, outer=(5)] + │ │ └── variable: x [type=int, outer=(5)] │ ├── const-agg [type=int, outer=(2)] - │ │ └── variable: a.i [type=int, outer=(2)] + │ │ └── variable: i [type=int, outer=(2)] │ ├── const-agg [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ └── const-agg [type=string, outer=(4)] - │ └── variable: a.s [type=string, outer=(4)] + │ └── variable: s [type=string, outer=(4)] └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - └── a.i < sum [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── i < sum [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Don't reject nulls when multiple columns are used. opt @@ -388,9 +388,9 @@ project │ │ └── true [type=bool] │ └── aggregations [outer=(5,6)] │ ├── min [type=int, outer=(5)] - │ │ └── variable: xy.x [type=int, outer=(5)] + │ │ └── variable: x [type=int, outer=(5)] │ └── max [type=int, outer=(6)] - │ └── variable: xy.y [type=int, outer=(6)] + │ └── variable: y [type=int, outer=(6)] └── filters [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] └── column7 = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] @@ -428,6 +428,6 @@ project │ │ └── true [type=bool] │ └── aggregations [outer=(5)] │ └── count [type=int, outer=(5)] - │ └── variable: xy.x [type=int, outer=(5)] + │ └── variable: x [type=int, outer=(5)] └── filters [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] └── column7 = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] diff --git a/pkg/sql/opt/norm/testdata/rules/scalar b/pkg/sql/opt/norm/testdata/rules/scalar index a872891ee363..a1f504a193ed 100644 --- a/pkg/sql/opt/norm/testdata/rules/scalar +++ b/pkg/sql/opt/norm/testdata/rules/scalar @@ -38,15 +38,15 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── projections [outer=(1,2)] - ├── a.k = (a.i + 1) [type=bool, outer=(1,2)] - ├── a.i != (2 - a.k) [type=bool, outer=(1,2)] - ├── a.k IS NOT DISTINCT FROM (a.i + 1) [type=bool, outer=(1,2)] - ├── a.k IS DISTINCT FROM (a.i - 1) [type=bool, outer=(1,2)] - ├── a.k + (a.i * 2) [type=int, outer=(1,2)] - ├── a.k * (a.i + 2) [type=int, outer=(1,2)] - ├── a.k & (a.i ^ 2) [type=int, outer=(1,2)] - ├── a.k | (a.i ^ 2) [type=int, outer=(1,2)] - └── a.k # (a.i * a.i) [type=int, outer=(1,2)] + ├── k = (i + 1) [type=bool, outer=(1,2)] + ├── i != (2 - k) [type=bool, outer=(1,2)] + ├── k IS NOT DISTINCT FROM (i + 1) [type=bool, outer=(1,2)] + ├── k IS DISTINCT FROM (i - 1) [type=bool, outer=(1,2)] + ├── k + (i * 2) [type=int, outer=(1,2)] + ├── k * (i + 2) [type=int, outer=(1,2)] + ├── k & (i ^ 2) [type=int, outer=(1,2)] + ├── k | (i ^ 2) [type=int, outer=(1,2)] + └── k # (i * i) [type=int, outer=(1,2)] # -------------------------------------------------- # CommuteConst @@ -72,15 +72,15 @@ project │ ├── key: (1) │ └── fd: (1)-->(2,3) └── projections [outer=(1-3)] - ├── (a.i + a.k) = (length('foo') + 1) [type=bool, outer=(1,2)] - ├── (a.i * 2) != length('bar') [type=bool, outer=(2)] - ├── (1 - a.k) IS NOT DISTINCT FROM 5 [type=bool, outer=(1)] - ├── a.k IS DISTINCT FROM (10 + 1) [type=bool, outer=(1)] - ├── a.f + 1.0 [type=float, outer=(3)] - ├── (a.i * a.i) * (5 * length('foo')) [type=int, outer=(2)] - ├── (a.i + a.i) & (100 ^ 2) [type=int, outer=(2)] - ├── (a.i + a.i) | (length('foo') + 1) [type=int, outer=(2)] - └── (a.k ^ 2) # (1 - length('foo')) [type=int, outer=(1)] + ├── (i + k) = (length('foo') + 1) [type=bool, outer=(1,2)] + ├── (i * 2) != length('bar') [type=bool, outer=(2)] + ├── (1 - k) IS NOT DISTINCT FROM 5 [type=bool, outer=(1)] + ├── k IS DISTINCT FROM (10 + 1) [type=bool, outer=(1)] + ├── f + 1.0 [type=float, outer=(3)] + ├── (i * i) * (5 * length('foo')) [type=int, outer=(2)] + ├── (i + i) & (100 ^ 2) [type=int, outer=(2)] + ├── (i + i) | (length('foo') + 1) [type=int, outer=(2)] + └── (k ^ 2) # (1 - length('foo')) [type=int, outer=(1)] # -------------------------------------------------- # EliminateCoalesce @@ -93,7 +93,7 @@ project ├── scan a │ └── columns: i:2(int) └── projections [outer=(2)] - └── variable: a.i [type=int, outer=(2)] + └── variable: i [type=int, outer=(2)] # -------------------------------------------------- # SimplifyCoalesce @@ -126,7 +126,7 @@ project ├── scan a │ └── columns: s:4(string) └── projections [outer=(4)] - └── COALESCE(a.s, a.s || 'foo') [type=string, outer=(4)] + └── COALESCE(s, s || 'foo') [type=string, outer=(4)] # Trailing null can't be removed. opt @@ -137,7 +137,7 @@ project ├── scan a │ └── columns: i:2(int) └── projections [outer=(2)] - └── COALESCE(a.i, NULL, NULL) [type=int, outer=(2)] + └── COALESCE(i, NULL, NULL) [type=int, outer=(2)] # -------------------------------------------------- # EliminateCast @@ -261,10 +261,10 @@ project ├── null [type=float] ├── null [type=int] ├── null [type=int] - ├── a.arr::DECIMAL[] || CAST(NULL AS DECIMAL[]) [type=decimal[], outer=(6)] - ├── CAST(NULL AS STRING[]) || a.arr::STRING[] [type=string[], outer=(6)] - ├── a.i::DECIMAL || CAST(NULL AS DECIMAL[]) [type=decimal[], outer=(2)] - └── CAST(NULL AS FLOAT[]) || a.i::FLOAT [type=float[], outer=(2)] + ├── arr::DECIMAL[] || CAST(NULL AS DECIMAL[]) [type=decimal[], outer=(6)] + ├── CAST(NULL AS STRING[]) || arr::STRING[] [type=string[], outer=(6)] + ├── i::DECIMAL || CAST(NULL AS DECIMAL[]) [type=decimal[], outer=(2)] + └── CAST(NULL AS FLOAT[]) || i::FLOAT [type=float[], outer=(2)] opt SELECT @@ -330,7 +330,7 @@ project ├── scan a │ └── columns: i:2(int) └── projections [outer=(2)] - └── a.i IN (NULL, 1, 2, 3) [type=bool, outer=(2)] + └── i IN (NULL, 1, 2, 3) [type=bool, outer=(2)] opt SELECT s NOT IN ('foo', s || 'foo', 'bar', length(s)::string, NULL) AS r FROM a @@ -340,7 +340,7 @@ project ├── scan a │ └── columns: s:4(string) └── projections [outer=(4)] - └── a.s NOT IN (NULL, 'bar', 'foo', a.s || 'foo', length(a.s)::STRING) [type=bool, outer=(4)] + └── s NOT IN (NULL, 'bar', 'foo', s || 'foo', length(s)::STRING) [type=bool, outer=(4)] # -------------------------------------------------- # EliminateExistsProject @@ -461,7 +461,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters [type=bool, outer=(5)] - └── a.j @> '{"a": "b"}' [type=bool, outer=(5)] + └── j @> '{"a": "b"}' [type=bool, outer=(5)] opt SELECT * FROM a WHERE j->'a' @> '{"x": "b"}'::JSON @@ -475,7 +475,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters [type=bool, outer=(5)] - └── a.j @> '{"a": {"x": "b"}}' [type=bool, outer=(5)] + └── j @> '{"a": {"x": "b"}}' [type=bool, outer=(5)] opt SELECT * FROM a WHERE j->'a'->'x' = '"b"'::JSON @@ -489,7 +489,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters [type=bool, outer=(5)] - └── a.j @> '{"a": {"x": "b"}}' [type=bool, outer=(5)] + └── j @> '{"a": {"x": "b"}}' [type=bool, outer=(5)] # The transformation is not valid in this case. opt @@ -504,7 +504,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters [type=bool, outer=(5)] - └── (a.j->2) = '"b"' [type=bool, outer=(5)] + └── (j->2) = '"b"' [type=bool, outer=(5)] # The transformation is not valid in this case, since j->'a' could be an array. opt @@ -519,4 +519,4 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters [type=bool, outer=(5)] - └── (a.j->'a') @> '"b"' [type=bool, outer=(5)] + └── (j->'a') @> '"b"' [type=bool, outer=(5)] diff --git a/pkg/sql/opt/norm/testdata/rules/select b/pkg/sql/opt/norm/testdata/rules/select index ea6a9938bdf8..2e6a079830c4 100644 --- a/pkg/sql/opt/norm/testdata/rules/select +++ b/pkg/sql/opt/norm/testdata/rules/select @@ -43,8 +43,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2,4), constraints=(/2: [/5 - /5]; /4: (/NULL - /'foo'); tight), fd=()-->(2)] - ├── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] - └── a.s < 'foo' [type=bool, outer=(4), constraints=(/4: (/NULL - /'foo'); tight)] + ├── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + └── s < 'foo' [type=bool, outer=(4), constraints=(/4: (/NULL - /'foo'); tight)] # -------------------------------------------------- # EnsureSelectFilters @@ -61,7 +61,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] - └── a.i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] + └── i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] opt SELECT * FROM a WHERE i<5 OR s='foo' @@ -75,7 +75,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2,4)] - └── (a.i < 5) OR (a.s = 'foo') [type=bool, outer=(2,4)] + └── (i < 5) OR (s = 'foo') [type=bool, outer=(2,4)] # Don't use Filters for True or False condition. opt @@ -123,7 +123,7 @@ select │ ├── key: () │ └── fd: ()-->(1-5) └── filters [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] - └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] opt SELECT * FROM (SELECT * FROM a WHERE i=1) WHERE False @@ -155,8 +155,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(2,4), constraints=(/2: (/NULL - /4]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] - ├── a.i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] - └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + ├── i < 5 [type=bool, outer=(2), constraints=(/2: (/NULL - /4]; tight)] + └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] opt SELECT * FROM (SELECT * FROM a WHERE i>1 AND i<10) WHERE s='foo' OR k=5 @@ -170,9 +170,9 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1,2,4), constraints=(/2: [/2 - /9])] - ├── a.i > 1 [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] - ├── a.i < 10 [type=bool, outer=(2), constraints=(/2: (/NULL - /9]; tight)] - └── (a.s = 'foo') OR (a.k = 5) [type=bool, outer=(1,4)] + ├── i > 1 [type=bool, outer=(2), constraints=(/2: [/2 - ]; tight)] + ├── i < 10 [type=bool, outer=(2), constraints=(/2: (/NULL - /9]; tight)] + └── (s = 'foo') OR (k = 5) [type=bool, outer=(1,4)] # -------------------------------------------------- # PushSelectIntoProject @@ -189,9 +189,9 @@ project │ ├── scan a │ │ └── columns: i:2(int) f:3(float) │ └── filters [type=bool, outer=(3), constraints=(/3: [/10.0 - /10.0]; tight), fd=()-->(3)] - │ └── a.f = 10.0 [type=bool, outer=(3), constraints=(/3: [/10.0 - /10.0]; tight)] + │ └── f = 10.0 [type=bool, outer=(3), constraints=(/3: [/10.0 - /10.0]; tight)] └── projections [outer=(2,3)] - └── a.i + 1 [type=int, outer=(2)] + └── i + 1 [type=int, outer=(2)] # Don't push down select if it depends on computed column that can't be inlined. opt @@ -207,7 +207,7 @@ select │ ├── scan a │ │ └── columns: i:2(int) f:3(float) │ └── projections [outer=(2,3), side-effects] - │ └── a.i / 2 [type=decimal, outer=(2), side-effects] + │ └── i / 2 [type=decimal, outer=(2), side-effects] └── filters [type=bool, outer=(6), constraints=(/6: [/2 - /2]; tight), fd=()-->(6)] └── div = 2 [type=bool, outer=(6), constraints=(/6: [/2 - /2]; tight)] @@ -229,10 +229,10 @@ select │ │ ├── scan a │ │ │ └── columns: i:2(int) f:3(float) │ │ └── filters [type=bool, outer=(2,3), constraints=(/2: [/1 - /1]; /3: [/10.0 - /10.0]; tight), fd=()-->(2,3)] - │ │ ├── a.f = 10.0 [type=bool, outer=(3), constraints=(/3: [/10.0 - /10.0]; tight)] - │ │ └── a.i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + │ │ ├── f = 10.0 [type=bool, outer=(3), constraints=(/3: [/10.0 - /10.0]; tight)] + │ │ └── i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] │ └── projections [outer=(2,3), side-effects] - │ └── a.i / 2 [type=decimal, outer=(2), side-effects] + │ └── i / 2 [type=decimal, outer=(2), side-effects] └── filters [type=bool, outer=(6), constraints=(/6: [/2 - /2]; tight), fd=()-->(6)] └── div = 2 [type=bool, outer=(6), constraints=(/6: [/2 - /2]; tight)] @@ -255,11 +255,11 @@ project │ │ │ └── columns: i:2(int) f:3(float) │ │ └── aggregations [outer=(3)] │ │ └── sum [type=float, outer=(3)] - │ │ └── variable: a.f [type=float, outer=(3)] + │ │ └── variable: f [type=float, outer=(3)] │ └── filters [type=bool, outer=(6), constraints=(/6: [/10.0 - /10.0]; tight), fd=()-->(6)] │ └── column6 = 10.0 [type=bool, outer=(6), constraints=(/6: [/10.0 - /10.0]; tight)] └── projections [outer=(3)] - └── a.f + 1.1 [type=float, outer=(3)] + └── f + 1.1 [type=float, outer=(3)] # -------------------------------------------------- # PushSelectCondLeftIntoJoinLeftAndRight @@ -294,9 +294,9 @@ select │ ├── left ordering: +1 │ ├── right ordering: +6 │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── filters [type=bool, outer=(6)] - └── (xy.x = 6) OR (xy.x IS NULL) [type=bool, outer=(6)] + └── (x = 6) OR (x IS NULL) [type=bool, outer=(6)] opt SELECT * FROM a WHERE EXISTS (SELECT * FROM xy WHERE a.k=xy.x) AND a.k > 5 @@ -321,7 +321,7 @@ semi-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] opt SELECT * FROM a WHERE NOT EXISTS (SELECT * FROM xy WHERE a.k=xy.x) AND a.k > 5 @@ -346,7 +346,7 @@ anti-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Only the filters bound by the right side are mapped and pushed down. opt @@ -372,16 +372,16 @@ select │ │ │ ├── key: (6) │ │ │ └── fd: (6)-->(7) │ │ └── filters [type=bool, outer=(6,7)] - │ │ ├── (xy.x + xy.y) > 5 [type=bool, outer=(6,7)] - │ │ └── (xy.y % 2) = 0 [type=bool, outer=(7)] + │ │ ├── (x + y) > 5 [type=bool, outer=(6,7)] + │ │ └── (y % 2) = 0 [type=bool, outer=(7)] │ └── filters [type=bool, outer=(1,2,6,7), constraints=(/1: [/10 - ]; /2: (/NULL - ]; /6: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(6), (6)==(1), (2)==(7), (7)==(2)] - │ ├── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - │ ├── a.i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - │ ├── (a.k + a.i) > 5 [type=bool, outer=(1,2)] - │ ├── (a.i % 2) = 0 [type=bool, outer=(2)] - │ └── a.k >= 10 [type=bool, outer=(1), constraints=(/1: [/10 - ]; tight)] + │ ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ ├── i = y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ ├── (k + i) > 5 [type=bool, outer=(1,2)] + │ ├── (i % 2) = 0 [type=bool, outer=(2)] + │ └── k >= 10 [type=bool, outer=(1), constraints=(/1: [/10 - ]; tight)] └── filters [type=bool, outer=(2,6)] - └── ((xy.x + a.i) = 6) OR (xy.x IS NULL) [type=bool, outer=(2,6)] + └── ((x + i) = 6) OR (x IS NULL) [type=bool, outer=(2,6)] # -------------------------------------------------- # PushSelectIntoJoinLeft @@ -403,9 +403,9 @@ inner-join (lookup xy) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ └── filters [type=bool, outer=(3), constraints=(/3: [/1.1 - /1.1]; tight), fd=()-->(3)] - │ └── a.f = 1.1 [type=bool, outer=(3), constraints=(/3: [/1.1 - /1.1]; tight)] + │ └── f = 1.1 [type=bool, outer=(3), constraints=(/3: [/1.1 - /1.1]; tight)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] opt SELECT * FROM a LEFT JOIN xy ON a.k=xy.x @@ -429,12 +429,12 @@ select │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-5) │ │ └── filters [type=bool, outer=(3,4), constraints=(/3: [/1.1 - /1.1]), fd=()-->(3)] - │ │ ├── a.f = 1.1 [type=bool, outer=(3), constraints=(/3: [/1.1 - /1.1]; tight)] - │ │ └── (a.s = 'foo') OR (a.s = 'bar') [type=bool, outer=(4)] + │ │ ├── f = 1.1 [type=bool, outer=(3), constraints=(/3: [/1.1 - /1.1]; tight)] + │ │ └── (s = 'foo') OR (s = 'bar') [type=bool, outer=(4)] │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── filters [type=bool, outer=(2,7)] - └── (a.i < xy.y) OR (xy.y IS NULL) [type=bool, outer=(2,7)] + └── (i < y) OR (y IS NULL) [type=bool, outer=(2,7)] # Pushdown constant condition. opt @@ -464,7 +464,7 @@ inner-join │ │ └── fd: (1)-->(2-5) │ └── filters [type=bool, outer=(2), constraints=(/2: [/100 - /100]), fd=()-->(2)] │ ├── $1 > '2000-01-01T1:00:00' [type=bool] - │ └── a.i = 100 [type=bool, outer=(2), constraints=(/2: [/100 - /100]; tight)] + │ └── i = 100 [type=bool, outer=(2), constraints=(/2: [/100 - /100]; tight)] └── true [type=bool] # Don't push down conditions in case of RIGHT JOIN. @@ -493,9 +493,9 @@ select │ ├── left ordering: +1 │ ├── right ordering: +6 │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── filters [type=bool, outer=(2)] - └── (a.i = 100) OR (a.i IS NULL) [type=bool, outer=(2)] + └── (i = 100) OR (i IS NULL) [type=bool, outer=(2)] # Don't push down conditions in case of FULL JOIN. opt @@ -523,9 +523,9 @@ select │ ├── left ordering: +1 │ ├── right ordering: +6 │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── filters [type=bool, outer=(2)] - └── (a.i = 100) OR (a.i IS NULL) [type=bool, outer=(2)] + └── (i = 100) OR (i IS NULL) [type=bool, outer=(2)] # Push into semi-join. opt @@ -546,7 +546,7 @@ semi-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 opt(2) │ └── filters [type=bool, outer=(2), constraints=(/2: [/0 - /0]; tight), fd=()-->(2)] - │ └── a.i = 0 [type=bool, outer=(2), constraints=(/2: [/0 - /0]; tight)] + │ └── i = 0 [type=bool, outer=(2), constraints=(/2: [/0 - /0]; tight)] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -556,7 +556,7 @@ semi-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Push into anti-join. opt @@ -577,7 +577,7 @@ anti-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 opt(2) │ └── filters [type=bool, outer=(2), constraints=(/2: [/0 - /0]; tight), fd=()-->(2)] - │ └── a.i = 0 [type=bool, outer=(2), constraints=(/2: [/0 - /0]; tight)] + │ └── i = 0 [type=bool, outer=(2), constraints=(/2: [/0 - /0]; tight)] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -587,7 +587,7 @@ anti-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # -------------------------------------------------- # PushSelectIntoJoinRight @@ -609,9 +609,9 @@ inner-join (lookup xy) │ │ ├── key: (3) │ │ └── fd: (3)-->(4-7) │ └── filters [type=bool, outer=(5), constraints=(/5: [/1.1 - /1.1]; tight), fd=()-->(5)] - │ └── a.f = 1.1 [type=bool, outer=(5), constraints=(/5: [/1.1 - /1.1]; tight)] + │ └── f = 1.1 [type=bool, outer=(5), constraints=(/5: [/1.1 - /1.1]; tight)] └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - └── xy.x = a.k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + └── x = k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] opt SELECT * FROM xy RIGHT JOIN a ON xy.x=a.k @@ -635,12 +635,12 @@ select │ │ │ ├── key: (3) │ │ │ └── fd: (3)-->(4-7) │ │ └── filters [type=bool, outer=(5,6), constraints=(/5: [/1.1 - /1.1]), fd=()-->(5)] - │ │ ├── a.f = 1.1 [type=bool, outer=(5), constraints=(/5: [/1.1 - /1.1]; tight)] - │ │ └── (a.s = 'foo') OR (a.s = 'bar') [type=bool, outer=(6)] + │ │ ├── f = 1.1 [type=bool, outer=(5), constraints=(/5: [/1.1 - /1.1]; tight)] + │ │ └── (s = 'foo') OR (s = 'bar') [type=bool, outer=(6)] │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ └── xy.x = a.k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ └── x = k [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] └── filters [type=bool, outer=(2,4)] - └── (a.i < xy.y) OR (xy.y IS NULL) [type=bool, outer=(2,4)] + └── (i < y) OR (y IS NULL) [type=bool, outer=(2,4)] # Don't push down conditions in case of LEFT JOIN. opt @@ -668,9 +668,9 @@ select │ ├── left ordering: +1 │ ├── right ordering: +3 │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ └── k = x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] └── filters [type=bool, outer=(4)] - └── (a.i = 100) OR (a.i IS NULL) [type=bool, outer=(4)] + └── (i = 100) OR (i IS NULL) [type=bool, outer=(4)] # Don't push down conditions in case of FULL JOIN. opt @@ -698,9 +698,9 @@ select │ ├── left ordering: +1 │ ├── right ordering: +3 │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ └── a.k = xy.x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ └── k = x [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] └── filters [type=bool, outer=(4)] - └── (a.i = 100) OR (a.i IS NULL) [type=bool, outer=(4)] + └── (i = 100) OR (i IS NULL) [type=bool, outer=(4)] # -------------------------------------------------- # MergeSelectInnerJoin @@ -726,8 +726,8 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,4,6,7), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── (a.s = 'foo') OR (xy.y < 100) [type=bool, outer=(4,7)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── (s = 'foo') OR (y < 100) [type=bool, outer=(4,7)] opt SELECT * FROM a INNER JOIN xy ON a.k=xy.x WHERE (a.s='foo' OR xy.y<100) @@ -750,8 +750,8 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,4,6,7), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── (a.s = 'foo') OR (xy.y < 100) [type=bool, outer=(4,7)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── (s = 'foo') OR (y < 100) [type=bool, outer=(4,7)] opt SELECT * FROM a INNER JOIN xy ON a.k=xy.x WHERE False @@ -793,7 +793,7 @@ select │ │ └── fd: (6)-->(7) │ └── true [type=bool] └── filters [type=bool, outer=(1,6)] - └── (a.k = xy.x) OR (xy.x IS NULL) [type=bool, outer=(1,6)] + └── (k = x) OR (x IS NULL) [type=bool, outer=(1,6)] # Don't merge with RIGHT JOIN. opt @@ -817,7 +817,7 @@ select │ │ └── fd: (6)-->(7) │ └── true [type=bool] └── filters [type=bool, outer=(1,6)] - └── (a.k = xy.x) OR (a.k IS NULL) [type=bool, outer=(1,6)] + └── (k = x) OR (k IS NULL) [type=bool, outer=(1,6)] # Don't merge with FULL JOIN. opt @@ -841,7 +841,7 @@ select │ │ └── fd: (6)-->(7) │ └── true [type=bool] └── filters [type=bool, outer=(1,6)] - └── ((a.k = xy.x) OR (a.k IS NULL)) OR (xy.x IS NULL) [type=bool, outer=(1,6)] + └── ((k = x) OR (k IS NULL)) OR (x IS NULL) [type=bool, outer=(1,6)] # -------------------------------------------------- # PushSelectIntoJoinLeft + PushSelectIntoJoinRight + MergeSelectInnerJoin @@ -863,12 +863,12 @@ inner-join (lookup a) │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] - │ └── xy.y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] + │ └── y = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight)] └── filters [type=bool, outer=(1-4,6,7), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /3: [/1.1 - /1.1]; /4: [/'foo' - /'foo']; /6: (/NULL - ]; /7: (/NULL - ]), fd=()-->(3,4), (1)==(6), (6)==(1)] - ├── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - ├── a.i < xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] - ├── a.f = 1.1 [type=bool, outer=(3), constraints=(/3: [/1.1 - /1.1]; tight)] - └── a.s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] + ├── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── i < y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + ├── f = 1.1 [type=bool, outer=(3), constraints=(/3: [/1.1 - /1.1]; tight)] + └── s = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight)] opt SELECT * FROM a, xy WHERE a.i=100 AND $1>'2000-01-01T1:00:00' AND xy.x=a.k @@ -888,9 +888,9 @@ inner-join (lookup xy) │ │ └── fd: (1)-->(2-5) │ └── filters [type=bool, outer=(2), constraints=(/2: [/100 - /100]), fd=()-->(2)] │ ├── $1 > '2000-01-01T1:00:00' [type=bool] - │ └── a.i = 100 [type=bool, outer=(2), constraints=(/2: [/100 - /100]; tight)] + │ └── i = 100 [type=bool, outer=(2), constraints=(/2: [/100 - /100]; tight)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── xy.x = a.k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── $1 > '2000-01-01T1:00:00' [type=bool] # -------------------------------------------------- @@ -912,11 +912,11 @@ group-by │ ├── scan a │ │ └── columns: i:2(int) │ └── filters [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] - │ └── a.i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + │ └── i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] └── aggregations [outer=(2)] ├── count-rows [type=int] └── const-agg [type=int, outer=(2)] - └── variable: a.i [type=int, outer=(2)] + └── variable: i [type=int, outer=(2)] # Push down into GroupBy with no aggregations. opt @@ -933,10 +933,10 @@ distinct-on │ ├── scan a │ │ └── columns: i:2(int) │ └── filters [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] - │ └── a.i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + │ └── i = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] └── aggregations [outer=(2)] └── const-agg [type=int, outer=(2)] - └── variable: a.i [type=int, outer=(2)] + └── variable: i [type=int, outer=(2)] # Push down only conditions that do not depend on aggregations. opt @@ -960,12 +960,12 @@ select │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2,4) │ │ └── filters [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] - │ │ └── a.i = a.k [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] + │ │ └── i = k [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] │ └── aggregations [outer=(2,4)] │ ├── max [type=string, outer=(4)] - │ │ └── variable: a.s [type=string, outer=(4)] + │ │ └── variable: s [type=string, outer=(4)] │ └── const-agg [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] └── filters [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] └── m = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight)] @@ -983,10 +983,10 @@ distinct-on │ ├── scan a │ │ └── columns: i:2(int) f:3(float) s:4(string) │ └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] - │ └── a.i > a.f [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + │ └── i > f [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] └── aggregations [outer=(4)] └── first-agg [type=string, outer=(4)] - └── variable: a.s [type=string, outer=(4)] + └── variable: s [type=string, outer=(4)] # DistinctOn case with a ConstAgg. opt @@ -1014,16 +1014,16 @@ distinct-on │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2,3) │ │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] - │ │ └── a.k > a.f [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ │ └── k > f [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ └── a.i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ └── i = y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] └── aggregations [outer=(2,3,6)] ├── first-agg [type=int, outer=(2)] - │ └── variable: a.i [type=int, outer=(2)] + │ └── variable: i [type=int, outer=(2)] ├── first-agg [type=int, outer=(6)] - │ └── variable: xy.x [type=int, outer=(6)] + │ └── variable: x [type=int, outer=(6)] └── const-agg [type=float, outer=(3)] - └── variable: a.f [type=float, outer=(3)] + └── variable: f [type=float, outer=(3)] # Do *not* push down into scalar GroupBy. opt @@ -1092,8 +1092,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2) └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - /99]; tight)] - ├── b.i < 100 [type=bool, outer=(2), constraints=(/2: (/NULL - /99]; tight)] - └── b.i IS NOT NULL [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] + ├── i < 100 [type=bool, outer=(2), constraints=(/2: (/NULL - /99]; tight)] + └── i IS NOT NULL [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] opt SELECT k,s FROM b WHERE k IS NOT NULL AND s IS NOT NULL @@ -1116,7 +1116,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2,4) └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] - └── b.i IS NOT NULL [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] + └── i IS NOT NULL [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] # RemoveNotNullCondition rule is not applied opt @@ -1127,7 +1127,7 @@ select ├── scan b │ └── columns: i:2(int) └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] - └── b.i IS NOT NULL [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] + └── i IS NOT NULL [type=bool, outer=(2), constraints=(/2: (/NULL - ]; tight)] # RemoveNotNullCondition rule is not applied opt @@ -1145,7 +1145,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── filters [type=bool, outer=(1,2)] - └── (b.i + b.k) IS NOT NULL [type=bool, outer=(1,2)] + └── (i + k) IS NOT NULL [type=bool, outer=(1,2)] # -------------------------------------------------- # DetectSelectContradiction diff --git a/pkg/sql/opt/norm/testdata/rules/side_effects b/pkg/sql/opt/norm/testdata/rules/side_effects index b63b35006fb9..05f7c759881f 100644 --- a/pkg/sql/opt/norm/testdata/rules/side_effects +++ b/pkg/sql/opt/norm/testdata/rules/side_effects @@ -74,12 +74,12 @@ project │ │ ├── scan a │ │ │ └── columns: i:2(int) f:3(float) │ │ └── filters [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] - │ │ └── a.i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] + │ │ └── i = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight)] │ └── projections [outer=(2,3), side-effects] - │ └── a.i + (random() * 10.0)::INT [type=int, outer=(2), side-effects] + │ └── i + (random() * 10.0)::INT [type=int, outer=(2), side-effects] └── aggregations [outer=(3)] └── avg [type=float, outer=(3)] - └── variable: a.f [type=float, outer=(3)] + └── variable: f [type=float, outer=(3)] # Allow elimination of side effecting expressions during column pruning. opt @@ -109,7 +109,7 @@ inner-join (merge) │ │ ├── fd: (1)-->(2-5) │ │ └── ordering: +1 │ └── filters [type=bool, outer=(1), side-effects, constraints=(/1: (/NULL - ])] - │ └── a.k = random() [type=bool, outer=(1), side-effects, constraints=(/1: (/NULL - ])] + │ └── k = random() [type=bool, outer=(1), side-effects, constraints=(/1: (/NULL - ])] ├── select │ ├── columns: x:6(int!null) y:7(int) │ ├── side-effects @@ -122,12 +122,12 @@ inner-join (merge) │ │ ├── fd: (6)-->(7) │ │ └── ordering: +6 │ └── filters [type=bool, outer=(6), side-effects, constraints=(/6: (/NULL - ])] - │ └── xy.x = random() [type=bool, outer=(6), side-effects, constraints=(/6: (/NULL - ])] + │ └── x = random() [type=bool, outer=(6), side-effects, constraints=(/6: (/NULL - ])] └── merge-on ├── left ordering: +1 ├── right ordering: +6 └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── a.k = xy.x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Decorrelate CASE WHEN branch if there are no side effects. opt @@ -145,9 +145,9 @@ project │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] - │ └── xy.x = a.i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + │ └── x = i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] └── projections [outer=(2,7)] - └── CASE WHEN a.i < 0 THEN xy.y ELSE 5 END [type=int, outer=(2,7)] + └── CASE WHEN i < 0 THEN y ELSE 5 END [type=int, outer=(2,7)] # Decorrelate CASE ELSE branch if there are no side effects. opt @@ -173,9 +173,9 @@ project │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] - │ └── xy.x = a.i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + │ └── x = i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] └── filters [type=bool, outer=(1,2,7), constraints=(/1: (/NULL - ])] - └── a.k = CASE WHEN a.i < 0 THEN 5 ELSE xy.y END [type=bool, outer=(1,2,7), constraints=(/1: (/NULL - ])] + └── k = CASE WHEN i < 0 THEN 5 ELSE y END [type=bool, outer=(1,2,7), constraints=(/1: (/NULL - ])] # Don't decorrelate CASE WHEN branch if there are side effects. opt @@ -190,7 +190,7 @@ project └── case [type=int, outer=(2), side-effects] ├── true [type=bool] ├── when [type=int, outer=(2), side-effects] - │ ├── a.i < 0 [type=bool, outer=(2)] + │ ├── i < 0 [type=bool, outer=(2)] │ └── subquery [type=int, outer=(2), side-effects] │ └── project │ ├── columns: y:7(int) @@ -217,7 +217,7 @@ project │ │ │ ├── key: (6) │ │ │ └── fd: (6)-->(7) │ │ └── filters [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] - │ │ └── xy.x = a.i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + │ │ └── x = i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] │ └── (random() * 10.0)::INT [type=int, side-effects] └── const: 5 [type=int] @@ -236,11 +236,11 @@ select │ └── fd: (1)-->(2-5) └── filters [type=bool, outer=(1,2), side-effects, constraints=(/1: (/NULL - ])] └── eq [type=bool, outer=(1,2), side-effects, constraints=(/1: (/NULL - ])] - ├── variable: a.k [type=int, outer=(1)] + ├── variable: k [type=int, outer=(1)] └── case [type=int, outer=(2), side-effects] ├── true [type=bool] ├── when [type=int, outer=(2)] - │ ├── a.i < 0 [type=bool, outer=(2)] + │ ├── i < 0 [type=bool, outer=(2)] │ └── const: 5 [type=int] └── subquery [type=int, outer=(2), side-effects] └── project @@ -262,5 +262,5 @@ select │ ├── key: (6) │ └── fd: (6)-->(7) └── filters [type=bool, outer=(2,6,7), side-effects, constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] - ├── xy.x = a.i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] - └── (5 / xy.y) > 1 [type=bool, outer=(7), side-effects] + ├── x = i [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + └── (5 / y) > 1 [type=bool, outer=(7), side-effects] diff --git a/pkg/sql/opt/optbuilder/builder_test.go b/pkg/sql/opt/optbuilder/builder_test.go index eedaa511acea..a41aaecd01db 100644 --- a/pkg/sql/opt/optbuilder/builder_test.go +++ b/pkg/sql/opt/optbuilder/builder_test.go @@ -21,7 +21,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder" "github.com/cockroachdb/cockroach/pkg/sql/opt/props" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils" @@ -67,7 +67,7 @@ func TestBuilder(t *testing.T) { var err error tester := testutils.NewOptTester(catalog, d.Input) - tester.Flags.ExprFormat = opt.ExprFmtHideAll ^ opt.ExprFmtHideScalars + tester.Flags.ExprFormat = memo.ExprFmtHideAll ^ memo.ExprFmtHideScalars for _, arg := range d.CmdArgs { key, vals := arg.Key, arg.Vals diff --git a/pkg/sql/opt/optbuilder/orderby.go b/pkg/sql/opt/optbuilder/orderby.go index 283b08515f92..75a191289e03 100644 --- a/pkg/sql/opt/optbuilder/orderby.go +++ b/pkg/sql/opt/optbuilder/orderby.go @@ -123,7 +123,7 @@ func (b *Builder) buildOrderByIndex( panic(err) } - colItem := tree.NewColumnItem(tab.Name(), tree.Name(col.Column.ColName())) + colItem := tree.NewColumnItem(tab.Name(), col.Column.ColName()) expr := inScope.resolveType(colItem, types.Any) b.addExtraColumn(expr, inScope, projectionsScope, orderByScope) } diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 374a8b8ee243..cee9dac47988 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -254,9 +254,7 @@ func (b *Builder) buildScan( tab opt.Table, tn *tree.TableName, ordinals []int, indexFlags *tree.IndexFlags, inScope *scope, ) (outScope *scope) { md := b.factory.Metadata() - - tabName := tree.AsStringWithFlags(tn, b.FmtFlags) - tabID := md.AddTableWithName(tab, tabName) + tabID := md.AddTable(tab) colCount := len(ordinals) if colCount == 0 { @@ -265,7 +263,7 @@ func (b *Builder) buildScan( var tabColIDs opt.ColSet outScope = inScope.push() - outScope.cols = make([]scopeColumn, 0, colCount) + outScope.cols = make([]scopeColumn, colCount) for i := 0; i < colCount; i++ { ord := i if ordinals != nil { @@ -275,15 +273,15 @@ func (b *Builder) buildScan( col := tab.Column(ord) colID := tabID.ColumnID(ord) tabColIDs.Add(int(colID)) - name := tree.Name(col.ColName()) - outScope.cols = append(outScope.cols, scopeColumn{ + name := col.ColName() + outScope.cols[i] = scopeColumn{ id: colID, origName: name, name: name, table: *tn, typ: col.DatumType(), hidden: col.IsHidden(), - }) + } } if tab.IsVirtualTable() { diff --git a/pkg/sql/opt/optbuilder/testdata/aggregate b/pkg/sql/opt/optbuilder/testdata/aggregate index c5a2df5522f1..ba36c1da3727 100644 --- a/pkg/sql/opt/optbuilder/testdata/aggregate +++ b/pkg/sql/opt/optbuilder/testdata/aggregate @@ -68,27 +68,27 @@ scalar-group-by │ └── projections │ ├── const: 1 [type=int] │ ├── eq [type=bool] - │ │ ├── variable: kv.v [type=int] + │ │ ├── variable: v [type=int] │ │ └── const: 1 [type=int] │ └── cast: BYTES [type=bytes] - │ └── variable: kv.s [type=string] + │ └── variable: s [type=string] └── aggregations ├── min [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] ├── max [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] ├── count [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] ├── sum-int [type=int] │ └── variable: column8 [type=int] ├── avg [type=decimal] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] ├── sum [type=decimal] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] ├── std-dev [type=decimal] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] ├── variance [type=decimal] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] ├── bool-and [type=bool] │ └── variable: column14 [type=bool] └── xor-agg [type=bytes] @@ -167,7 +167,7 @@ scalar-group-by │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── json-agg [type=jsonb] - └── variable: kv.v [type=int] + └── variable: v [type=int] build SELECT jsonb_agg(1) @@ -438,7 +438,7 @@ group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── function: upper [type=string] - │ └── variable: kv.s [type=string] + │ └── variable: s [type=string] └── aggregations └── count-rows [type=int] @@ -495,7 +495,7 @@ project │ └── count-rows [type=int] └── projections └── function: upper [type=string] - └── variable: kv.s [type=string] + └── variable: s [type=string] # Selecting a value that is not grouped, even if a function of it it, does not work. build @@ -516,8 +516,8 @@ group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── plus [type=int] - │ ├── variable: kv.k [type=int] - │ └── variable: kv.v [type=int] + │ ├── variable: k [type=int] + │ └── variable: v [type=int] └── aggregations └── count-rows [type=int] @@ -539,8 +539,8 @@ project │ └── count-rows [type=int] └── projections └── plus [type=int] - ├── variable: kv.k [type=int] - └── variable: kv.v [type=int] + ├── variable: k [type=int] + └── variable: v [type=int] build SELECT count(*), k+v FROM kv GROUP BY k @@ -580,11 +580,11 @@ group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── plus [type=int] - │ ├── variable: kv.v [type=int] - │ └── variable: kv.w [type=int] + │ ├── variable: v [type=int] + │ └── variable: w [type=int] └── aggregations └── count [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT count(*) @@ -607,7 +607,7 @@ scalar-group-by │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── count [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT count(1) @@ -659,7 +659,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── count [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT v, count(k) FROM kv GROUP BY v ORDER BY v DESC @@ -676,7 +676,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── count [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT v, count(k) FROM kv GROUP BY v ORDER BY count(k) DESC @@ -693,7 +693,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── count [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT v, count(k) FROM kv GROUP BY v ORDER BY v-count(k) @@ -712,10 +712,10 @@ sort │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ └── count [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── projections └── minus [type=int] - ├── variable: kv.v [type=int] + ├── variable: v [type=int] └── variable: count [type=int] build @@ -733,7 +733,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── sum [type=decimal] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT v, count(k) FROM kv GROUP BY v ORDER BY 1 DESC @@ -750,7 +750,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── count [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT count(*), count(k), count(kv.v) FROM kv @@ -764,9 +764,9 @@ scalar-group-by └── aggregations ├── count-rows [type=int] ├── count [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── count [type=int] - └── variable: kv.v [type=int] + └── variable: v [type=int] build SELECT count(kv.*) FROM kv @@ -779,10 +779,10 @@ scalar-group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── tuple [type=tuple{int AS k, int AS v, int AS w, string AS s}] - │ ├── variable: kv.k [type=int] - │ ├── variable: kv.v [type=int] - │ ├── variable: kv.w [type=int] - │ └── variable: kv.s [type=string] + │ ├── variable: k [type=int] + │ ├── variable: v [type=int] + │ ├── variable: w [type=int] + │ └── variable: s [type=string] └── aggregations └── count [type=int] └── variable: column5 [type=tuple{int AS k, int AS v, int AS w, string AS s}] @@ -799,10 +799,10 @@ scalar-group-by └── aggregations ├── count [type=int] │ └── agg-distinct [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── count [type=int] └── agg-distinct [type=int] - └── variable: kv.v [type=int] + └── variable: v [type=int] build SELECT upper(s), count(DISTINCT k), count(DISTINCT v), count(DISTINCT (v)) FROM kv GROUP BY upper(s) @@ -816,14 +816,14 @@ group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── function: upper [type=string] - │ └── variable: kv.s [type=string] + │ └── variable: s [type=string] └── aggregations ├── count [type=int] │ └── agg-distinct [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── count [type=int] └── agg-distinct [type=int] - └── variable: kv.v [type=int] + └── variable: v [type=int] build SELECT count((k, v)) FROM kv @@ -836,8 +836,8 @@ scalar-group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── tuple [type=tuple{int, int}] - │ ├── variable: kv.k [type=int] - │ └── variable: kv.v [type=int] + │ ├── variable: k [type=int] + │ └── variable: v [type=int] └── aggregations └── count [type=int] └── variable: column5 [type=tuple{int, int}] @@ -853,8 +853,8 @@ scalar-group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── tuple [type=tuple{int, int}] - │ ├── variable: kv.k [type=int] - │ └── variable: kv.v [type=int] + │ ├── variable: k [type=int] + │ └── variable: v [type=int] └── aggregations └── count [type=int] └── agg-distinct [type=tuple{int, int}] @@ -871,8 +871,8 @@ scalar-group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── tuple [type=tuple{int, int}] - │ ├── variable: kv.k [type=int] - │ └── variable: kv.v [type=int] + │ ├── variable: k [type=int] + │ └── variable: v [type=int] └── aggregations └── count [type=int] └── agg-distinct [type=tuple{int, int}] @@ -907,8 +907,8 @@ limit │ │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ │ └── projections │ │ └── tuple [type=tuple{int, int}] - │ │ ├── variable: kv.k [type=int] - │ │ └── variable: kv.v [type=int] + │ │ ├── variable: k [type=int] + │ │ └── variable: v [type=int] │ └── aggregations │ └── count [type=int] │ └── variable: column5 [type=tuple{int, int}] @@ -927,8 +927,8 @@ offset │ │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ │ └── projections │ │ └── tuple [type=tuple{int, int}] - │ │ ├── variable: kv.k [type=int] - │ │ └── variable: kv.v [type=int] + │ │ ├── variable: k [type=int] + │ │ └── variable: v [type=int] │ └── aggregations │ └── count [type=int] │ └── variable: column5 [type=tuple{int, int}] @@ -947,9 +947,9 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ ├── count [type=int] - │ │ └── variable: kv.k [type=int] + │ │ └── variable: k [type=int] │ └── count [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] └── projections └── plus [type=int] ├── variable: column5 [type=int] @@ -987,13 +987,13 @@ scalar-group-by │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations ├── min [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] ├── max [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] ├── min [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] └── max [type=int] - └── variable: kv.v [type=int] + └── variable: v [type=int] build SELECT min(k), max(k), min(v), max(v) FROM kv WHERE k > 8 @@ -1008,17 +1008,17 @@ scalar-group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── filters [type=bool] │ └── gt [type=bool] - │ ├── variable: kv.k [type=int] + │ ├── variable: k [type=int] │ └── const: 8 [type=int] └── aggregations ├── min [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] ├── max [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] ├── min [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] └── max [type=int] - └── variable: kv.v [type=int] + └── variable: v [type=int] build SELECT array_agg(k), array_agg(s) FROM (SELECT k, s FROM kv ORDER BY k) @@ -1034,9 +1034,9 @@ scalar-group-by │ └── ordering: +1 └── aggregations ├── array-agg [type=int[]] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── array-agg [type=string[]] - └── variable: kv.s [type=string] + └── variable: s [type=string] build SELECT array_agg(k) FROM (SELECT k FROM kv ORDER BY s) @@ -1053,7 +1053,7 @@ scalar-group-by │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── array-agg [type=int[]] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT max(k) FROM (SELECT k FROM kv ORDER BY s) @@ -1068,7 +1068,7 @@ scalar-group-by │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── max [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build @@ -1088,7 +1088,7 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ └── array-agg [type=int[]] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── projections └── concat [type=int[]] ├── variable: column5 [type=int[]] @@ -1107,11 +1107,11 @@ scalar-group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── filters [type=bool] │ └── is [type=bool] - │ ├── variable: kv.s [type=string] + │ ├── variable: s [type=string] │ └── null [type=unknown] └── aggregations └── array-agg [type=string[]] - └── variable: kv.s [type=string] + └── variable: s [type=string] build SELECT avg(k), avg(v), sum(k), sum(v) FROM kv @@ -1124,13 +1124,13 @@ scalar-group-by │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations ├── avg [type=decimal] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] ├── avg [type=decimal] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] ├── sum [type=decimal] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── sum [type=decimal] - └── variable: kv.v [type=int] + └── variable: v [type=int] build SELECT avg(k::decimal), avg(v::decimal), sum(k::decimal), sum(v::decimal) FROM kv @@ -1143,9 +1143,9 @@ scalar-group-by │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ ├── cast: DECIMAL [type=decimal] - │ │ └── variable: kv.k [type=int] + │ │ └── variable: k [type=int] │ └── cast: DECIMAL [type=decimal] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] └── aggregations ├── avg [type=decimal] │ └── variable: column5 [type=decimal] @@ -1168,16 +1168,16 @@ scalar-group-by └── aggregations ├── avg [type=decimal] │ └── agg-distinct [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] ├── avg [type=decimal] │ └── agg-distinct [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] ├── sum [type=decimal] │ └── agg-distinct [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── sum [type=decimal] └── agg-distinct [type=int] - └── variable: kv.v [type=int] + └── variable: v [type=int] build SELECT avg(k) * 2.0 + max(v)::DECIMAL AS r FROM kv @@ -1192,9 +1192,9 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ ├── avg [type=decimal] - │ │ └── variable: kv.k [type=int] + │ │ └── variable: k [type=int] │ └── max [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] └── projections └── plus [type=decimal] ├── mult [type=decimal] @@ -1219,14 +1219,14 @@ project │ │ └── filters [type=bool] │ │ └── eq [type=bool] │ │ ├── mult [type=int] - │ │ │ ├── variable: kv.w [type=int] + │ │ │ ├── variable: w [type=int] │ │ │ └── const: 2 [type=int] - │ │ └── variable: kv.k [type=int] + │ │ └── variable: k [type=int] │ └── aggregations │ ├── avg [type=decimal] - │ │ └── variable: kv.k [type=int] + │ │ └── variable: k [type=int] │ └── max [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] └── projections └── plus [type=decimal] ├── mult [type=decimal] @@ -1260,13 +1260,13 @@ scalar-group-by │ └── columns: a:1(string!null) b:2(float) c:3(bool) d:4(decimal) └── aggregations ├── min [type=string] - │ └── variable: abc.a [type=string] + │ └── variable: a [type=string] ├── min [type=float] - │ └── variable: abc.b [type=float] + │ └── variable: b [type=float] ├── min [type=bool] - │ └── variable: abc.c [type=bool] + │ └── variable: c [type=bool] └── min [type=decimal] - └── variable: abc.d [type=decimal] + └── variable: d [type=decimal] build SELECT max(a), max(b), max(c), max(d) FROM abc @@ -1277,13 +1277,13 @@ scalar-group-by │ └── columns: a:1(string!null) b:2(float) c:3(bool) d:4(decimal) └── aggregations ├── max [type=string] - │ └── variable: abc.a [type=string] + │ └── variable: a [type=string] ├── max [type=float] - │ └── variable: abc.b [type=float] + │ └── variable: b [type=float] ├── max [type=bool] - │ └── variable: abc.c [type=bool] + │ └── variable: c [type=bool] └── max [type=decimal] - └── variable: abc.d [type=decimal] + └── variable: d [type=decimal] build SELECT avg(b), sum(b), avg(d), sum(d) FROM abc @@ -1296,13 +1296,13 @@ scalar-group-by │ └── columns: a:1(string!null) b:2(float) c:3(bool) d:4(decimal) └── aggregations ├── avg [type=float] - │ └── variable: abc.b [type=float] + │ └── variable: b [type=float] ├── sum [type=float] - │ └── variable: abc.b [type=float] + │ └── variable: b [type=float] ├── avg [type=decimal] - │ └── variable: abc.d [type=decimal] + │ └── variable: d [type=decimal] └── sum [type=decimal] - └── variable: abc.d [type=decimal] + └── variable: d [type=decimal] # Verify summing of intervals exec-ddl @@ -1324,7 +1324,7 @@ scalar-group-by │ └── columns: a:1(interval!null) └── aggregations └── sum [type=interval] - └── variable: intervals.a [type=interval] + └── variable: a [type=interval] build SELECT avg(a) FROM abc @@ -1393,7 +1393,7 @@ scalar-group-by │ └── columns: x:1(int!null) y:2(int) z:3(float) └── aggregations └── min [type=int] - └── variable: xyz.x [type=int] + └── variable: x [type=int] build SELECT min(x) FROM xyz WHERE x in (0, 4, 7) @@ -1408,14 +1408,14 @@ scalar-group-by │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ └── filters [type=bool] │ └── in [type=bool] - │ ├── variable: xyz.x [type=int] + │ ├── variable: x [type=int] │ └── tuple [type=tuple{int, int, int}] │ ├── const: 0 [type=int] │ ├── const: 4 [type=int] │ └── const: 7 [type=int] └── aggregations └── min [type=int] - └── variable: xyz.x [type=int] + └── variable: x [type=int] build SELECT max(x) FROM xyz @@ -1428,7 +1428,7 @@ scalar-group-by │ └── columns: x:1(int!null) y:2(int) z:3(float) └── aggregations └── max [type=int] - └── variable: xyz.x [type=int] + └── variable: x [type=int] build SELECT max(y) FROM xyz WHERE x = 1 @@ -1443,11 +1443,11 @@ scalar-group-by │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ └── filters [type=bool] │ └── eq [type=bool] - │ ├── variable: xyz.x [type=int] + │ ├── variable: x [type=int] │ └── const: 1 [type=int] └── aggregations └── max [type=int] - └── variable: xyz.y [type=int] + └── variable: y [type=int] build SELECT min(y) FROM xyz WHERE x = 7 @@ -1462,11 +1462,11 @@ scalar-group-by │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ └── filters [type=bool] │ └── eq [type=bool] - │ ├── variable: xyz.x [type=int] + │ ├── variable: x [type=int] │ └── const: 7 [type=int] └── aggregations └── min [type=int] - └── variable: xyz.y [type=int] + └── variable: y [type=int] build SELECT min(x) FROM xyz WHERE (y, z) = (2, 3.0) @@ -1482,14 +1482,14 @@ scalar-group-by │ └── filters [type=bool] │ └── eq [type=bool] │ ├── tuple [type=tuple{int, float}] - │ │ ├── variable: xyz.y [type=int] - │ │ └── variable: xyz.z [type=float] + │ │ ├── variable: y [type=int] + │ │ └── variable: z [type=float] │ └── tuple [type=tuple{int, float}] │ ├── const: 2 [type=int] │ └── const: 3.0 [type=float] └── aggregations └── min [type=int] - └── variable: xyz.x [type=int] + └── variable: x [type=int] build SELECT max(x) FROM xyz WHERE (z, y) = (3.0, 2) @@ -1505,14 +1505,14 @@ scalar-group-by │ └── filters [type=bool] │ └── eq [type=bool] │ ├── tuple [type=tuple{float, int}] - │ │ ├── variable: xyz.z [type=float] - │ │ └── variable: xyz.y [type=int] + │ │ ├── variable: z [type=float] + │ │ └── variable: y [type=int] │ └── tuple [type=tuple{float, int}] │ ├── const: 3.0 [type=float] │ └── const: 2 [type=int] └── aggregations └── max [type=int] - └── variable: xyz.x [type=int] + └── variable: x [type=int] # VARIANCE/STDDEV @@ -1530,14 +1530,14 @@ project │ │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ │ └── projections │ │ └── cast: DECIMAL [type=decimal] - │ │ └── variable: xyz.y [type=int] + │ │ └── variable: y [type=int] │ └── aggregations │ ├── variance [type=decimal] - │ │ └── variable: xyz.x [type=int] + │ │ └── variable: x [type=int] │ ├── variance [type=decimal] │ │ └── variable: column5 [type=decimal] │ └── variance [type=float] - │ └── variable: xyz.z [type=float] + │ └── variable: z [type=float] └── projections └── function: round [type=float] ├── variable: column7 [type=float] @@ -1556,11 +1556,11 @@ scalar-group-by │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ └── filters [type=bool] │ └── eq [type=bool] - │ ├── variable: xyz.x [type=int] + │ ├── variable: x [type=int] │ └── const: 10 [type=int] └── aggregations └── variance [type=decimal] - └── variable: xyz.x [type=int] + └── variable: x [type=int] build SELECT stddev(x), stddev(y::decimal), round(stddev(z), 14) FROM xyz @@ -1575,14 +1575,14 @@ project │ │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ │ └── projections │ │ └── cast: DECIMAL [type=decimal] - │ │ └── variable: xyz.y [type=int] + │ │ └── variable: y [type=int] │ └── aggregations │ ├── std-dev [type=decimal] - │ │ └── variable: xyz.x [type=int] + │ │ └── variable: x [type=int] │ ├── std-dev [type=decimal] │ │ └── variable: column5 [type=decimal] │ └── std-dev [type=float] - │ └── variable: xyz.z [type=float] + │ └── variable: z [type=float] └── projections └── function: round [type=float] ├── variable: column7 [type=float] @@ -1601,11 +1601,11 @@ scalar-group-by │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ └── filters [type=bool] │ └── eq [type=bool] - │ ├── variable: xyz.x [type=int] + │ ├── variable: x [type=int] │ └── const: 1 [type=int] └── aggregations └── std-dev [type=decimal] - └── variable: xyz.x [type=int] + └── variable: x [type=int] build SELECT avg(1::int)::float, avg(2::float)::float, avg(3::decimal)::float @@ -1748,7 +1748,7 @@ limit │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ └── projections │ └── gt [type=bool] - │ ├── variable: xyz.x [type=int] + │ ├── variable: x [type=int] │ └── subquery [type=decimal] │ └── max1-row │ ├── columns: avg:5(decimal) @@ -1811,9 +1811,9 @@ scalar-group-by │ └── columns: b:1(bool) rowid:2(int!null) └── aggregations ├── bool-and [type=bool] - │ └── variable: bools.b [type=bool] + │ └── variable: b [type=bool] └── bool-or [type=bool] - └── variable: bools.b [type=bool] + └── variable: b [type=bool] # Tests with * inside GROUP BY. @@ -1854,9 +1854,9 @@ project │ │ └── columns: a:1(bytes) b:2(int) c:3(int) rowid:4(int!null) │ └── aggregations │ ├── xor-agg [type=bytes] - │ │ └── variable: xor_bytes.a [type=bytes] + │ │ └── variable: a [type=bytes] │ └── xor-agg [type=int] - │ └── variable: xor_bytes.c [type=int] + │ └── variable: c [type=int] └── projections └── function: to_hex [type=string] └── variable: column5 [type=bytes] @@ -1878,9 +1878,9 @@ sort │ │ └── columns: a:1(bytes) b:2(int) c:3(int) rowid:4(int!null) │ └── aggregations │ ├── xor-agg [type=bytes] - │ │ └── variable: xor_bytes.a [type=bytes] + │ │ └── variable: a [type=bytes] │ └── xor-agg [type=int] - │ └── variable: xor_bytes.c [type=int] + │ └── variable: c [type=int] └── projections └── function: to_hex [type=string] └── variable: column5 [type=bytes] @@ -1933,7 +1933,7 @@ scalar-group-by │ └── ordering: +1 └── aggregations └── concat-agg [type=string] - └── variable: kv.s [type=string] + └── variable: s [type=string] build SELECT json_agg(s) FROM (SELECT s FROM kv ORDER BY k) @@ -1949,7 +1949,7 @@ scalar-group-by │ └── ordering: +1 └── aggregations └── json-agg [type=jsonb] - └── variable: kv.s [type=string] + └── variable: s [type=string] build SELECT jsonb_agg(s) FROM (SELECT s FROM kv ORDER BY k) @@ -1965,7 +1965,7 @@ scalar-group-by │ └── ordering: +1 └── aggregations └── jsonb-agg [type=jsonb] - └── variable: kv.s [type=string] + └── variable: s [type=string] exec-ddl CREATE TABLE ab ( @@ -2004,8 +2004,8 @@ project │ └── columns: a:1(int!null) b:2(int) └── projections └── tuple [type=tuple{int, int}] - ├── variable: ab.b [type=int] - └── variable: ab.a [type=int] + ├── variable: b [type=int] + └── variable: a [type=int] build SELECT min(y), (b, a) AS r @@ -2027,11 +2027,11 @@ project │ │ └── true [type=bool] │ └── aggregations │ └── min [type=string] - │ └── variable: xy.y [type=string] + │ └── variable: y [type=string] └── projections └── tuple [type=tuple{int, int}] - ├── variable: ab.b [type=int] - └── variable: ab.a [type=int] + ├── variable: b [type=int] + └── variable: a [type=int] build SELECT v, count(k) FROM kv GROUP BY v ORDER BY count(k) @@ -2048,7 +2048,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── count [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT v, count(*) FROM kv GROUP BY v ORDER BY count(*) @@ -2099,11 +2099,11 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ ├── plus [type=int] - │ │ ├── variable: kv.k [type=int] - │ │ └── variable: kv.v [type=int] + │ │ ├── variable: k [type=int] + │ │ └── variable: v [type=int] │ └── plus [type=int] - │ ├── variable: kv.v [type=int] - │ └── variable: kv.w [type=int] + │ ├── variable: v [type=int] + │ └── variable: w [type=int] └── projections └── div [type=decimal] ├── variable: column5 [type=int] @@ -2182,7 +2182,7 @@ project │ └── projections │ └── and [type=bool] │ ├── variable: bools.b [type=bool] - │ └── variable: abc.c [type=bool] + │ └── variable: c [type=bool] └── projections └── and [type=bool] ├── variable: column9 [type=bool] @@ -2218,7 +2218,7 @@ project │ └── projections │ └── or [type=bool] │ ├── variable: bools.b [type=bool] - │ └── variable: abc.c [type=bool] + │ └── variable: c [type=bool] └── projections └── or [type=bool] ├── variable: column9 [type=bool] @@ -2243,12 +2243,12 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── mod [type=int] - │ ├── variable: kv.k [type=int] - │ └── variable: kv.w [type=int] + │ ├── variable: k [type=int] + │ └── variable: w [type=int] └── projections └── mod [type=int] ├── variable: column5 [type=int] - └── variable: kv.v [type=int] + └── variable: v [type=int] build SELECT concat(concat(s, a), a) FROM kv, abc GROUP BY concat(s, a), a @@ -2269,12 +2269,12 @@ project │ │ └── true [type=bool] │ └── projections │ └── function: concat [type=string] - │ ├── variable: kv.s [type=string] - │ └── variable: abc.a [type=string] + │ ├── variable: s [type=string] + │ └── variable: a [type=string] └── projections └── function: concat [type=string] ├── variable: column9 [type=string] - └── variable: abc.a [type=string] + └── variable: a [type=string] build SELECT concat(concat(s, a), s) FROM kv, abc GROUP BY concat(s, a), a @@ -2295,13 +2295,13 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── lt [type=bool] - │ ├── variable: kv.k [type=int] - │ └── variable: kv.w [type=int] + │ ├── variable: k [type=int] + │ └── variable: w [type=int] └── projections └── and [type=bool] ├── variable: column5 [type=bool] └── ne [type=bool] - ├── variable: kv.v [type=int] + ├── variable: v [type=int] └── const: 5 [type=int] build @@ -2436,7 +2436,7 @@ group-by │ └── columns: b:1(bool) rowid:2(int!null) └── projections └── not [type=bool] - └── variable: bools.b [type=bool] + └── variable: b [type=bool] build SELECT b FROM bools GROUP BY NOT b @@ -2457,7 +2457,7 @@ project │ └── columns: b:1(bool) rowid:2(int!null) └── projections └── not [type=bool] - └── variable: bools.b [type=bool] + └── variable: b [type=bool] build SELECT +k * (-w) AS r FROM kv GROUP BY +k, -w @@ -2473,10 +2473,10 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── unary-minus [type=int] - │ └── variable: kv.w [type=int] + │ └── variable: w [type=int] └── projections └── mult [type=int] - ├── variable: kv.k [type=int] + ├── variable: k [type=int] └── variable: column5 [type=int] build @@ -2493,10 +2493,10 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── unary-minus [type=int] - │ └── variable: kv.w [type=int] + │ └── variable: w [type=int] └── projections └── mult [type=int] - ├── variable: kv.k [type=int] + ├── variable: k [type=int] └── variable: column5 [type=int] build @@ -2513,9 +2513,9 @@ project │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── projections └── mult [type=int] - ├── variable: kv.k [type=int] + ├── variable: k [type=int] └── unary-minus [type=int] - └── variable: kv.w [type=int] + └── variable: w [type=int] build SELECT 1 + min(v*2) AS r FROM kv GROUP BY k+3 @@ -2531,10 +2531,10 @@ project │ │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ │ └── projections │ │ ├── plus [type=int] - │ │ │ ├── variable: kv.k [type=int] + │ │ │ ├── variable: k [type=int] │ │ │ └── const: 3 [type=int] │ │ └── mult [type=int] - │ │ ├── variable: kv.v [type=int] + │ │ ├── variable: v [type=int] │ │ └── const: 2 [type=int] │ └── aggregations │ └── min [type=int] @@ -2573,7 +2573,7 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ └── function: upper [type=string] - │ └── variable: kv.s [type=string] + │ └── variable: s [type=string] └── aggregations └── count [type=int] └── variable: column5 [type=string] @@ -2596,11 +2596,11 @@ project │ │ └── columns: a:5(string!null) b:6(float) c:7(bool) d:8(decimal) │ └── filters [type=bool] │ └── ge [type=bool] - │ ├── variable: kv.k [type=int] - │ └── variable: abc.d [type=decimal] + │ ├── variable: k [type=int] + │ └── variable: d [type=decimal] └── aggregations └── sum [type=decimal] - └── variable: abc.d [type=decimal] + └── variable: d [type=decimal] build SELECT sum(DISTINCT abc.d) FROM abc @@ -2614,7 +2614,7 @@ scalar-group-by └── aggregations └── sum [type=decimal] └── agg-distinct [type=decimal] - └── variable: abc.d [type=decimal] + └── variable: d [type=decimal] build SELECT sum(abc.d) FILTER (WHERE abc.d > 0) FROM abc @@ -2639,7 +2639,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── max [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT max(k) AS mk FROM kv GROUP BY v ORDER BY max(k) @@ -2658,7 +2658,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── max [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT max(k) AS mk1, max(k) AS mk2 FROM kv GROUP BY v ORDER BY mk1 @@ -2677,7 +2677,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── max [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT max(k) AS mk1, max(k) AS mk2 FROM kv GROUP BY v ORDER BY mk2 @@ -2696,7 +2696,7 @@ sort │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) └── aggregations └── max [type=int] - └── variable: kv.k [type=int] + └── variable: k [type=int] build SELECT max(k) AS mk1, max(k)/5 AS mk2 FROM kv GROUP BY v ORDER BY mk2 @@ -2715,7 +2715,7 @@ sort │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ └── max [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── projections └── div [type=decimal] ├── variable: mk1 [type=int] @@ -2738,18 +2738,18 @@ project │ │ │ └── columns: k:1(int!null) v:2(int) w:3(int) kv.s:4(string) │ │ └── projections │ │ ├── plus [type=int] - │ │ │ ├── variable: kv.k [type=int] - │ │ │ └── variable: kv.v [type=int] + │ │ │ ├── variable: k [type=int] + │ │ │ └── variable: v [type=int] │ │ ├── minus [type=int] - │ │ │ ├── variable: kv.k [type=int] - │ │ │ └── variable: kv.v [type=int] + │ │ │ ├── variable: k [type=int] + │ │ │ └── variable: v [type=int] │ │ └── div [type=decimal] │ │ ├── plus [type=int] - │ │ │ ├── variable: kv.k [type=int] - │ │ │ └── variable: kv.v [type=int] + │ │ │ ├── variable: k [type=int] + │ │ │ └── variable: v [type=int] │ │ └── minus [type=int] - │ │ ├── variable: kv.k [type=int] - │ │ └── variable: kv.v [type=int] + │ │ ├── variable: k [type=int] + │ │ └── variable: v [type=int] │ └── aggregations │ └── max [type=decimal] │ └── variable: column7 [type=decimal] @@ -2772,22 +2772,22 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── projections │ ├── plus [type=int] - │ │ ├── variable: kv.k [type=int] - │ │ └── variable: kv.v [type=int] + │ │ ├── variable: k [type=int] + │ │ └── variable: v [type=int] │ ├── div [type=decimal] │ │ ├── plus [type=int] - │ │ │ ├── variable: kv.k [type=int] - │ │ │ └── variable: kv.v [type=int] + │ │ │ ├── variable: k [type=int] + │ │ │ └── variable: v [type=int] │ │ └── minus [type=int] - │ │ ├── variable: kv.k [type=int] - │ │ └── variable: kv.v [type=int] + │ │ ├── variable: k [type=int] + │ │ └── variable: v [type=int] │ └── mult [type=int] │ ├── plus [type=int] - │ │ ├── variable: kv.k [type=int] - │ │ └── variable: kv.v [type=int] + │ │ ├── variable: k [type=int] + │ │ └── variable: v [type=int] │ └── minus [type=int] - │ ├── variable: kv.k [type=int] - │ └── variable: kv.v [type=int] + │ ├── variable: k [type=int] + │ └── variable: v [type=int] └── aggregations └── max [type=decimal] └── variable: column6 [type=decimal] @@ -2809,7 +2809,7 @@ sort │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ └── max [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] └── projections └── const: 123 [type=int] @@ -2831,11 +2831,11 @@ scalar-group-by │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ └── projections │ └── plus [type=int] - │ ├── variable: xyz.x [type=int] - │ └── variable: xyz.y [type=int] + │ ├── variable: x [type=int] + │ └── variable: y [type=int] └── aggregations └── array-agg [type=int[]] - └── variable: xyz.y [type=int] + └── variable: y [type=int] build SELECT array_agg(y) FROM (SELECT * FROM xyz ORDER BY x DESC) @@ -2851,4 +2851,4 @@ scalar-group-by │ └── ordering: -1 └── aggregations └── array-agg [type=int[]] - └── variable: xyz.y [type=int] + └── variable: y [type=int] diff --git a/pkg/sql/opt/optbuilder/testdata/distinct b/pkg/sql/opt/optbuilder/testdata/distinct index 8129855d9279..90a60fdac025 100644 --- a/pkg/sql/opt/optbuilder/testdata/distinct +++ b/pkg/sql/opt/optbuilder/testdata/distinct @@ -115,8 +115,8 @@ sort │ └── columns: x:1(int!null) y:2(int) z:3(float) └── projections └── plus [type=int] - ├── variable: xyz.y [type=int] - └── variable: xyz.x [type=int] + ├── variable: y [type=int] + └── variable: x [type=int] build SELECT DISTINCT y + x AS r FROM xyz ORDER BY y + x @@ -133,8 +133,8 @@ sort │ └── columns: x:1(int!null) y:2(int) z:3(float) └── projections └── plus [type=int] - ├── variable: xyz.y [type=int] - └── variable: xyz.x [type=int] + ├── variable: y [type=int] + └── variable: x [type=int] build SELECT DISTINCT y + z FROM xyz ORDER BY y + z @@ -176,8 +176,8 @@ distinct-on │ └── columns: x:1(int!null) y:2(int) z:3(float) └── projections └── tuple [type=tuple{int, float}] - ├── variable: xyz.y [type=int] - └── variable: xyz.z [type=float] + ├── variable: y [type=int] + └── variable: z [type=float] build SELECT count(*) FROM (SELECT DISTINCT y FROM xyz) @@ -209,7 +209,7 @@ distinct-on │ └── columns: x:1(int!null) y:2(int) z:3(float) └── filters [type=bool] └── gt [type=bool] - ├── variable: xyz.x [type=int] + ├── variable: x [type=int] └── const: 0 [type=int] build @@ -226,7 +226,7 @@ distinct-on │ └── columns: x:1(int!null) y:2(int) z:3(float) └── filters [type=bool] └── gt [type=bool] - ├── variable: xyz.x [type=int] + ├── variable: x [type=int] └── const: 0 [type=int] build @@ -246,7 +246,7 @@ distinct-on │ └── columns: x:1(int!null) y:2(int) z:3(float) └── aggregations └── max [type=int] - └── variable: xyz.x [type=int] + └── variable: x [type=int] build SELECT DISTINCT x+y AS r FROM xyz @@ -260,8 +260,8 @@ distinct-on │ └── columns: x:1(int!null) y:2(int) z:3(float) └── projections └── plus [type=int] - ├── variable: xyz.x [type=int] - └── variable: xyz.y [type=int] + ├── variable: x [type=int] + └── variable: y [type=int] build SELECT DISTINCT 3 r FROM xyz @@ -306,15 +306,15 @@ distinct-on │ │ │ └── columns: x:1(int!null) y:2(int) z:3(float) │ │ └── aggregations │ │ └── max [type=float] - │ │ └── variable: xyz.z [type=float] + │ │ └── variable: z [type=float] │ └── filters [type=bool] │ └── gt [type=bool] - │ ├── variable: xyz.y [type=int] + │ ├── variable: y [type=int] │ └── const: 4 [type=int] └── projections ├── plus [type=int] - │ ├── variable: xyz.x [type=int] - │ └── variable: xyz.y [type=int] + │ ├── variable: x [type=int] + │ └── variable: y [type=int] └── const: 3 [type=int] exec-ddl diff --git a/pkg/sql/opt/optbuilder/testdata/distinct_on b/pkg/sql/opt/optbuilder/testdata/distinct_on index 15c8205a7936..59277012b32e 100644 --- a/pkg/sql/opt/optbuilder/testdata/distinct_on +++ b/pkg/sql/opt/optbuilder/testdata/distinct_on @@ -155,7 +155,7 @@ distinct-on │ └── columns: a:1(string!null) b:2(string!null) c:3(string!null) └── aggregations └── first-agg [type=string] - └── variable: abc.b [type=string] + └── variable: b [type=string] build SELECT DISTINCT ON (c, a) b, c, a FROM abc @@ -167,7 +167,7 @@ distinct-on │ └── columns: a:1(string!null) b:2(string!null) c:3(string!null) └── aggregations └── first-agg [type=string] - └── variable: abc.b [type=string] + └── variable: b [type=string] ################# # With ORDER BY # @@ -202,7 +202,7 @@ sort │ └── columns: x:1(int) y:2(int) z:3(int) pk1:4(int!null) pk2:5(int!null) └── aggregations └── first-agg [type=int] - └── variable: xyz.y [type=int] + └── variable: y [type=int] build SELECT DISTINCT ON (x) y, z, x FROM xyz ORDER BY x ASC, z DESC, y DESC @@ -221,9 +221,9 @@ distinct-on │ └── columns: x:1(int) y:2(int) z:3(int) pk1:4(int!null) pk2:5(int!null) └── aggregations ├── first-agg [type=int] - │ └── variable: xyz.y [type=int] + │ └── variable: y [type=int] └── first-agg [type=int] - └── variable: xyz.z [type=int] + └── variable: z [type=int] ##################### # With aggregations # @@ -243,9 +243,9 @@ distinct-on │ │ └── columns: x:1(int) y:2(int) z:3(int) pk1:4(int!null) pk2:5(int!null) │ └── aggregations │ ├── max [type=int] - │ │ └── variable: xyz.x [type=int] + │ │ └── variable: x [type=int] │ └── max [type=int] - │ └── variable: xyz.y [type=int] + │ └── variable: y [type=int] └── aggregations └── first-agg [type=int] └── variable: max [type=int] @@ -262,13 +262,13 @@ distinct-on │ │ └── columns: a:1(string!null) b:2(string!null) c:3(string!null) │ └── aggregations │ ├── max [type=string] - │ │ └── variable: abc.a [type=string] + │ │ └── variable: a [type=string] │ ├── min [type=string] - │ │ └── variable: abc.a [type=string] + │ │ └── variable: a [type=string] │ ├── max [type=string] - │ │ └── variable: abc.b [type=string] + │ │ └── variable: b [type=string] │ └── min [type=string] - │ └── variable: abc.c [type=string] + │ └── variable: c [type=string] └── aggregations └── first-agg [type=string] └── variable: max [type=string] @@ -292,7 +292,7 @@ distinct-on │ │ └── columns: x:1(int) y:2(int) z:3(int) pk1:4(int!null) pk2:5(int!null) │ └── aggregations │ └── min [type=int] - │ └── variable: xyz.x [type=int] + │ └── variable: x [type=int] └── aggregations └── first-agg [type=int] └── variable: min [type=int] @@ -316,7 +316,7 @@ distinct-on │ │ └── columns: x:1(int) y:2(int) z:3(int) pk1:4(int!null) pk2:5(int!null) │ └── aggregations │ └── min [type=int] - │ └── variable: xyz.x [type=int] + │ └── variable: x [type=int] └── filters [type=bool] └── eq [type=bool] ├── variable: column6 [type=int] @@ -347,9 +347,9 @@ distinct-on │ └── columns: x:1(int) y:2(int) z:3(int) pk1:4(int!null) pk2:5(int!null) └── aggregations ├── first-agg [type=int] - │ └── variable: xyz.y [type=int] + │ └── variable: y [type=int] └── first-agg [type=int] - └── variable: xyz.z [type=int] + └── variable: z [type=int] build SELECT DISTINCT ON (1,2,3) a, b, c FROM abc @@ -377,7 +377,7 @@ distinct-on │ └── columns: x:1(int) y:2(int) z:3(int) pk1:4(int!null) pk2:5(int!null) └── aggregations └── first-agg [type=int] - └── variable: xyz.y [type=int] + └── variable: y [type=int] # Ignores the alias. build @@ -423,7 +423,7 @@ sort │ └── columns: x:1(int) y:2(int) z:3(int) pk1:4(int!null) pk2:5(int!null) └── aggregations └── first-agg [type=int] - └── variable: xyz.z [type=int] + └── variable: z [type=int] build SELECT DISTINCT ON (x, y, z) pk1 FROM xyz ORDER BY x @@ -440,4 +440,4 @@ sort │ └── columns: x:1(int) y:2(int) z:3(int) pk1:4(int!null) pk2:5(int!null) └── aggregations └── first-agg [type=int] - └── variable: xyz.pk1 [type=int] + └── variable: pk1 [type=int] diff --git a/pkg/sql/opt/optbuilder/testdata/explain b/pkg/sql/opt/optbuilder/testdata/explain index 1666647e1421..25fbcdca75fa 100644 --- a/pkg/sql/opt/optbuilder/testdata/explain +++ b/pkg/sql/opt/optbuilder/testdata/explain @@ -70,7 +70,7 @@ explain │ └── const: 4 [type=int] └── filters [type=bool] └── eq [type=bool] - ├── variable: xy.x [type=int] + ├── variable: x [type=int] └── variable: column1 [type=int] build diff --git a/pkg/sql/opt/optbuilder/testdata/having b/pkg/sql/opt/optbuilder/testdata/having index 8e44fe3d5e0d..d1192ac0a578 100644 --- a/pkg/sql/opt/optbuilder/testdata/having +++ b/pkg/sql/opt/optbuilder/testdata/having @@ -64,9 +64,9 @@ select │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ ├── min [type=int] - │ │ └── variable: kv.v [type=int] + │ │ └── variable: v [type=int] │ └── max [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── filters [type=bool] └── gt [type=bool] ├── variable: column5 [type=int] @@ -87,11 +87,11 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ ├── max [type=int] - │ │ └── variable: kv.v [type=int] + │ │ └── variable: v [type=int] │ ├── max [type=int] - │ │ └── variable: kv.k [type=int] + │ │ └── variable: k [type=int] │ └── min [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] └── filters [type=bool] └── gt [type=bool] ├── variable: column5 [type=int] @@ -139,8 +139,8 @@ select │ │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ │ └── projections │ │ └── plus [type=int] - │ │ ├── variable: kv.k [type=int] - │ │ └── variable: kv.w [type=int] + │ │ ├── variable: k [type=int] + │ │ └── variable: w [type=int] │ └── aggregations │ └── count-rows [type=int] └── filters [type=bool] @@ -170,10 +170,10 @@ project │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ └── max [type=int] - │ └── variable: kv.v [type=int] + │ └── variable: v [type=int] └── filters [type=bool] └── gt [type=bool] - ├── variable: kv.v [type=int] + ├── variable: v [type=int] └── const: 5 [type=int] build @@ -192,10 +192,10 @@ project │ │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ │ └── projections │ │ └── function: lower [type=string] - │ │ └── variable: kv.s [type=string] + │ │ └── variable: s [type=string] │ └── aggregations │ └── sum [type=decimal] - │ └── variable: kv.w [type=int] + │ └── variable: w [type=int] └── filters [type=bool] └── like [type=bool] ├── variable: column5 [type=string] @@ -217,10 +217,10 @@ project │ │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ │ └── projections │ │ └── function: lower [type=string] - │ │ └── variable: kv.s [type=string] + │ │ └── variable: s [type=string] │ └── aggregations │ └── sum [type=decimal] - │ └── variable: kv.w [type=int] + │ └── variable: w [type=int] └── filters [type=bool] └── in [type=bool] ├── variable: column6 [type=decimal] @@ -302,7 +302,7 @@ sort │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ └── sum [type=decimal] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── filters [type=bool] └── eq [type=bool] ├── variable: column5 [type=decimal] @@ -327,9 +327,9 @@ sort │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ ├── max [type=int] - │ │ └── variable: kv.k [type=int] + │ │ └── variable: k [type=int] │ └── sum [type=decimal] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── filters [type=bool] └── gt [type=bool] ├── variable: column5 [type=int] @@ -354,10 +354,10 @@ sort │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ └── sum [type=decimal] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── filters [type=bool] └── gt [type=bool] - ├── variable: kv.v [type=int] + ├── variable: v [type=int] └── const: 10 [type=int] build @@ -379,7 +379,7 @@ sort │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ └── max [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── filters [type=bool] └── gt [type=bool] ├── variable: column5 [type=int] @@ -404,7 +404,7 @@ sort │ │ └── columns: k:1(int!null) v:2(int) w:3(int) s:4(string) │ └── aggregations │ └── max [type=int] - │ └── variable: kv.k [type=int] + │ └── variable: k [type=int] └── filters [type=bool] └── gt [type=bool] ├── variable: column5 [type=int] diff --git a/pkg/sql/opt/optbuilder/testdata/join b/pkg/sql/opt/optbuilder/testdata/join index 793c26580bc7..180c69b60ce7 100644 --- a/pkg/sql/opt/optbuilder/testdata/join +++ b/pkg/sql/opt/optbuilder/testdata/join @@ -364,7 +364,7 @@ limit │ │ └── const: 42 [type=int] │ └── filters [type=bool] │ └── eq [type=bool] - │ ├── variable: onecolumn.x [type=int] + │ ├── variable: x [type=int] │ └── variable: column1 [type=int] └── const: 1 [type=int] @@ -748,7 +748,7 @@ project └── filters [type=bool] └── eq [type=bool] ├── variable: onecolumn.x [type=int] - └── variable: twocolumn.y [type=int] + └── variable: y [type=int] build SELECT * FROM onecolumn JOIN twocolumn ON onecolumn.x = twocolumn.y @@ -764,7 +764,7 @@ project └── filters [type=bool] └── eq [type=bool] ├── variable: onecolumn.x [type=int] - └── variable: twocolumn.y [type=int] + └── variable: y [type=int] # Inner join with filter predicate build @@ -800,7 +800,7 @@ project │ ├── variable: onecolumn.x [type=int] │ └── variable: twocolumn.x [type=int] └── eq [type=bool] - ├── variable: twocolumn.y [type=int] + ├── variable: y [type=int] └── const: 53 [type=int] # Outer joins with filter predicate @@ -821,7 +821,7 @@ project │ ├── variable: onecolumn.x [type=int] │ └── variable: twocolumn.x [type=int] └── eq [type=bool] - ├── variable: twocolumn.y [type=int] + ├── variable: y [type=int] └── const: 53 [type=int] build @@ -1083,7 +1083,7 @@ project │ └── const: 43 [type=int] └── filters [type=bool] └── eq [type=bool] - ├── variable: onecolumn.x [type=int] + ├── variable: x [type=int] └── variable: column1 [type=int] build @@ -1399,8 +1399,8 @@ project │ └── true [type=bool] └── filters [type=bool] └── eq [type=bool] - ├── variable: pairs.b [type=int] - └── variable: square.n [type=int] + ├── variable: b [type=int] + └── variable: n [type=int] # The filter expression becomes an ON predicate. build @@ -1420,9 +1420,9 @@ project └── filters [type=bool] └── eq [type=bool] ├── plus [type=int] - │ ├── variable: pairs.a [type=int] - │ └── variable: pairs.b [type=int] - └── variable: square.sq [type=int] + │ ├── variable: a [type=int] + │ └── variable: b [type=int] + └── variable: sq [type=int] # Query similar to the one above, but the filter refers to a rendered # expression and can't "break through". See the comment for propagateFilters @@ -1445,12 +1445,12 @@ project │ │ └── true [type=bool] │ └── projections │ └── plus [type=int] - │ ├── variable: pairs.a [type=int] - │ └── variable: pairs.b [type=int] + │ ├── variable: a [type=int] + │ └── variable: b [type=int] └── filters [type=bool] └── eq [type=bool] ├── variable: sum [type=int] - └── variable: square.sq [type=int] + └── variable: sq [type=int] # The filter expression must stay on top of the outer join. build @@ -1467,9 +1467,9 @@ project └── filters [type=bool] └── eq [type=bool] ├── plus [type=int] - │ ├── variable: pairs.a [type=int] - │ └── variable: pairs.b [type=int] - └── variable: square.sq [type=int] + │ ├── variable: a [type=int] + │ └── variable: b [type=int] + └── variable: sq [type=int] build SELECT * FROM pairs FULL OUTER JOIN square ON pairs.a + pairs.b = square.sq WHERE pairs.b%2 <> square.sq%2 @@ -1487,16 +1487,16 @@ project │ └── filters [type=bool] │ └── eq [type=bool] │ ├── plus [type=int] - │ │ ├── variable: pairs.a [type=int] - │ │ └── variable: pairs.b [type=int] - │ └── variable: square.sq [type=int] + │ │ ├── variable: a [type=int] + │ │ └── variable: b [type=int] + │ └── variable: sq [type=int] └── filters [type=bool] └── ne [type=bool] ├── mod [type=int] - │ ├── variable: pairs.b [type=int] + │ ├── variable: b [type=int] │ └── const: 2 [type=int] └── mod [type=int] - ├── variable: square.sq [type=int] + ├── variable: sq [type=int] └── const: 2 [type=int] # Filter propagation through outer joins. @@ -1520,34 +1520,34 @@ select │ └── and [type=bool] │ ├── and [type=bool] │ │ ├── eq [type=bool] - │ │ │ ├── variable: pairs.b [type=int] - │ │ │ └── variable: square.sq [type=int] + │ │ │ ├── variable: b [type=int] + │ │ │ └── variable: sq [type=int] │ │ └── gt [type=bool] - │ │ ├── variable: pairs.a [type=int] + │ │ ├── variable: a [type=int] │ │ └── const: 1 [type=int] │ └── lt [type=bool] - │ ├── variable: square.n [type=int] + │ ├── variable: n [type=int] │ └── const: 6 [type=int] └── filters [type=bool] └── and [type=bool] ├── and [type=bool] │ ├── gt [type=bool] - │ │ ├── variable: pairs.b [type=int] + │ │ ├── variable: b [type=int] │ │ └── const: 1 [type=int] │ └── or [type=bool] │ ├── is [type=bool] - │ │ ├── variable: square.n [type=int] + │ │ ├── variable: n [type=int] │ │ └── null [type=unknown] │ └── gt [type=bool] - │ ├── variable: square.n [type=int] + │ ├── variable: n [type=int] │ └── const: 1 [type=int] └── or [type=bool] ├── is [type=bool] - │ ├── variable: square.n [type=int] + │ ├── variable: n [type=int] │ └── null [type=unknown] └── lt [type=bool] - ├── variable: pairs.a [type=int] - └── variable: square.sq [type=int] + ├── variable: a [type=int] + └── variable: sq [type=int] build SELECT * @@ -1568,34 +1568,34 @@ select │ └── and [type=bool] │ ├── and [type=bool] │ │ ├── eq [type=bool] - │ │ │ ├── variable: pairs.b [type=int] - │ │ │ └── variable: square.sq [type=int] + │ │ │ ├── variable: b [type=int] + │ │ │ └── variable: sq [type=int] │ │ └── gt [type=bool] - │ │ ├── variable: pairs.a [type=int] + │ │ ├── variable: a [type=int] │ │ └── const: 1 [type=int] │ └── lt [type=bool] - │ ├── variable: square.n [type=int] + │ ├── variable: n [type=int] │ └── const: 6 [type=int] └── filters [type=bool] └── and [type=bool] ├── and [type=bool] │ ├── or [type=bool] │ │ ├── is [type=bool] - │ │ │ ├── variable: pairs.a [type=int] + │ │ │ ├── variable: a [type=int] │ │ │ └── null [type=unknown] │ │ └── gt [type=bool] - │ │ ├── variable: pairs.a [type=int] + │ │ ├── variable: a [type=int] │ │ └── const: 2 [type=int] │ └── gt [type=bool] - │ ├── variable: square.n [type=int] + │ ├── variable: n [type=int] │ └── const: 1 [type=int] └── or [type=bool] ├── is [type=bool] - │ ├── variable: pairs.a [type=int] + │ ├── variable: a [type=int] │ └── null [type=unknown] └── lt [type=bool] - ├── variable: pairs.a [type=int] - └── variable: square.sq [type=int] + ├── variable: a [type=int] + └── variable: sq [type=int] # The simpler plan for an inner join, to compare. build @@ -1617,34 +1617,34 @@ select │ └── and [type=bool] │ ├── and [type=bool] │ │ ├── eq [type=bool] - │ │ │ ├── variable: pairs.b [type=int] - │ │ │ └── variable: square.sq [type=int] + │ │ │ ├── variable: b [type=int] + │ │ │ └── variable: sq [type=int] │ │ └── gt [type=bool] - │ │ ├── variable: pairs.a [type=int] + │ │ ├── variable: a [type=int] │ │ └── const: 1 [type=int] │ └── lt [type=bool] - │ ├── variable: square.n [type=int] + │ ├── variable: n [type=int] │ └── const: 6 [type=int] └── filters [type=bool] └── and [type=bool] ├── and [type=bool] │ ├── or [type=bool] │ │ ├── is [type=bool] - │ │ │ ├── variable: pairs.a [type=int] + │ │ │ ├── variable: a [type=int] │ │ │ └── null [type=unknown] │ │ └── gt [type=bool] - │ │ ├── variable: pairs.a [type=int] + │ │ ├── variable: a [type=int] │ │ └── const: 2 [type=int] │ └── gt [type=bool] - │ ├── variable: square.n [type=int] + │ ├── variable: n [type=int] │ └── const: 1 [type=int] └── or [type=bool] ├── is [type=bool] - │ ├── variable: pairs.a [type=int] + │ ├── variable: a [type=int] │ └── null [type=unknown] └── lt [type=bool] - ├── variable: pairs.a [type=int] - └── variable: square.sq [type=int] + ├── variable: a [type=int] + └── variable: sq [type=int] exec-ddl @@ -2605,7 +2605,7 @@ project ├── tuple [type=tuple{int, int, int}] │ ├── variable: xyu.x [type=int] │ ├── variable: xyu.y [type=int] - │ └── variable: xyu.u [type=int] + │ └── variable: u [type=int] └── tuple [type=tuple{int, int, int}] ├── const: 1 [type=int] ├── const: 2 [type=int] @@ -2795,20 +2795,20 @@ project │ │ ├── const: 1 [type=int] │ │ └── const: 2 [type=int] │ └── lt [type=bool] - │ ├── variable: abcdef.c [type=int] + │ ├── variable: c [type=int] │ └── const: 6 [type=int] └── and [type=bool] ├── eq [type=bool] │ ├── tuple [type=tuple{int, int, int}] │ │ ├── variable: abcdef.a [type=int] │ │ ├── variable: abcdef.b [type=int] - │ │ └── variable: abcdef.c [type=int] + │ │ └── variable: c [type=int] │ └── tuple [type=tuple{int, int, int}] │ ├── const: 1 [type=int] │ ├── const: 2 [type=int] │ └── const: 6 [type=int] └── gt [type=bool] - ├── variable: abcdef.d [type=int] + ├── variable: d [type=int] └── const: 8 [type=int] # Regression tests for mixed-type equality columns (#22514). diff --git a/pkg/sql/opt/optbuilder/testdata/limit b/pkg/sql/opt/optbuilder/testdata/limit index 244f4727dc82..5848961cf9d5 100644 --- a/pkg/sql/opt/optbuilder/testdata/limit +++ b/pkg/sql/opt/optbuilder/testdata/limit @@ -149,7 +149,7 @@ limit │ │ └── columns: k:1(int!null) v:2(int) w:3(int) │ └── aggregations │ └── sum [type=decimal] - │ └── variable: t.w [type=int] + │ └── variable: w [type=int] └── const: 10 [type=int] build diff --git a/pkg/sql/opt/optbuilder/testdata/orderby b/pkg/sql/opt/optbuilder/testdata/orderby index 78646f09532c..ce51019d684d 100644 --- a/pkg/sql/opt/optbuilder/testdata/orderby +++ b/pkg/sql/opt/optbuilder/testdata/orderby @@ -142,7 +142,7 @@ build SELECT a AS "foo.bar", b FROM t ORDER BY "foo.bar" DESC ---- project - ├── columns: bar:1(int!null) b:2(int) + ├── columns: foo.bar:1(int!null) b:2(int) ├── ordering: -1 └── scan t,rev ├── columns: a:1(int!null) b:2(int) c:3(bool) @@ -265,7 +265,7 @@ sort │ └── columns: a:1(int!null) b:2(int) c:3(bool) └── filters [type=bool] └── eq [type=bool] - ├── variable: t.b [type=int] + ├── variable: b [type=int] └── const: 7 [type=int] build @@ -293,12 +293,12 @@ sort │ │ └── columns: a:1(int!null) b:2(int) c:3(bool) │ └── filters [type=bool] │ └── eq [type=bool] - │ ├── variable: t.b [type=int] + │ ├── variable: b [type=int] │ └── const: 7 [type=int] └── projections └── plus [type=int] - ├── variable: t.a [type=int] - └── variable: t.b [type=int] + ├── variable: a [type=int] + └── variable: b [type=int] build SELECT a FROM t ORDER BY a+b DESC, a @@ -312,8 +312,8 @@ sort │ └── columns: a:1(int!null) b:2(int) c:3(bool) └── projections └── plus [type=int] - ├── variable: t.a [type=int] - └── variable: t.b [type=int] + ├── variable: a [type=int] + └── variable: b [type=int] build SELECT a FROM t ORDER BY (((a))) @@ -524,7 +524,7 @@ sort │ └── columns: a:1(int!null) b:2(int) c:3(bool) └── projections └── plus [type=int] - ├── variable: t.b [type=int] + ├── variable: b [type=int] └── const: 2 [type=int] # Check that the sort picks up a renamed render properly. @@ -540,7 +540,7 @@ sort │ └── columns: a:1(int!null) b:2(int) c:3(bool) └── projections └── plus [type=int] - ├── variable: t.b [type=int] + ├── variable: b [type=int] └── const: 2 [type=int] build @@ -555,7 +555,7 @@ sort │ └── columns: a:1(int!null) b:2(int) c:3(bool) └── projections └── plus [type=int] - ├── variable: t.b [type=int] + ├── variable: b [type=int] └── const: 2 [type=int] build @@ -569,7 +569,7 @@ sort ├── scan t │ └── columns: a:1(int!null) b:2(int) c:3(bool) └── projections - └── variable: t.b [type=int] + └── variable: b [type=int] build SELECT b, c FROM t ORDER BY @4 @@ -625,7 +625,7 @@ sort │ └── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(string) └── projections └── function: lower [type=string] - └── variable: abc.d [type=string] + └── variable: d [type=string] build SELECT * FROM abc ORDER BY a @@ -721,7 +721,7 @@ sort │ └── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(string) └── filters [type=bool] └── eq [type=bool] - ├── variable: abc.b [type=int] + ├── variable: b [type=int] └── const: 2 [type=int] build @@ -738,7 +738,7 @@ sort │ └── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(string) └── filters [type=bool] └── eq [type=bool] - ├── variable: abc.b [type=int] + ├── variable: b [type=int] └── const: 2 [type=int] build @@ -757,7 +757,7 @@ sort │ └── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(string) └── filters [type=bool] └── eq [type=bool] - ├── variable: abc.a [type=int] + ├── variable: a [type=int] └── const: 1 [type=int] build @@ -841,8 +841,8 @@ project │ └── columns: a:1(int!null) b:2(int) c:3(int) d:4(int) └── projections └── plus [type=int] - ├── variable: abcd.a [type=int] - └── variable: abcd.b [type=int] + ├── variable: a [type=int] + └── variable: b [type=int] build SELECT b+d AS r FROM (SELECT * FROM abcd ORDER BY a,d) @@ -853,8 +853,8 @@ project │ └── columns: a:1(int!null) b:2(int) c:3(int) d:4(int) └── projections └── plus [type=int] - ├── variable: abcd.b [type=int] - └── variable: abcd.d [type=int] + ├── variable: b [type=int] + └── variable: d [type=int] build SELECT * FROM (VALUES ('a'), ('b'), ('c')) AS c(x) ORDER BY x diff --git a/pkg/sql/opt/optbuilder/testdata/project b/pkg/sql/opt/optbuilder/testdata/project index 3475f93c849b..e71be4e56095 100644 --- a/pkg/sql/opt/optbuilder/testdata/project +++ b/pkg/sql/opt/optbuilder/testdata/project @@ -72,7 +72,7 @@ project │ └── columns: x:1(int!null) y:2(float) └── projections ├── plus [type=int] - │ ├── variable: a.x [type=int] + │ ├── variable: x [type=int] │ └── const: 3 [type=int] └── false [type=bool] @@ -86,10 +86,10 @@ project └── projections └── or [type=bool] ├── lt [type=bool] - │ ├── variable: a.x [type=int] - │ └── variable: a.y [type=float] + │ ├── variable: x [type=int] + │ └── variable: y [type=float] └── gt [type=bool] - ├── variable: a.x [type=int] + ├── variable: x [type=int] └── const: 1000 [type=int] build @@ -113,10 +113,10 @@ project │ │ └── columns: x:1(int!null) y:2(float) │ └── projections │ ├── plus [type=int] - │ │ ├── variable: a.x [type=int] + │ │ ├── variable: x [type=int] │ │ └── const: 3 [type=int] │ └── plus [type=float] - │ ├── variable: a.y [type=float] + │ ├── variable: y [type=float] │ └── const: 1.0 [type=float] └── projections ├── plus [type=int] @@ -187,7 +187,7 @@ select │ └── columns: x:1(int) y:2(float) rowid:3(int!null) └── filters [type=bool] └── gt [type=bool] - ├── variable: c.rowid [type=int] + ├── variable: rowid [type=int] └── const: 0 [type=int] build @@ -276,6 +276,6 @@ project │ └── columns: k:1(int!null) v:2(int) └── projections └── tuple [type=tuple{int AS a, int AS b, int AS c}] - ├── variable: kv.v [type=int] - ├── variable: kv.v [type=int] - └── variable: kv.v [type=int] + ├── variable: v [type=int] + ├── variable: v [type=int] + └── variable: v [type=int] diff --git a/pkg/sql/opt/optbuilder/testdata/scalar b/pkg/sql/opt/optbuilder/testdata/scalar index 14e4d26e3a31..9d200d8f8b91 100644 --- a/pkg/sql/opt/optbuilder/testdata/scalar +++ b/pkg/sql/opt/optbuilder/testdata/scalar @@ -803,11 +803,11 @@ project │ │ │ └── columns: a:2(int!null) │ │ └── filters [type=bool] │ │ └── eq [type=bool] - │ │ ├── variable: x.a [type=int] - │ │ └── variable: y.b [type=int] + │ │ ├── variable: a [type=int] + │ │ └── variable: b [type=int] │ └── aggregations │ └── array-agg [type=int[]] - │ └── variable: x.a [type=int] + │ └── variable: a [type=int] └── array: [type=int[]] build @@ -828,7 +828,7 @@ project │ │ └── ordering: +2 │ └── aggregations │ └── array-agg [type=int[]] - │ └── variable: x.a [type=int] + │ └── variable: a [type=int] └── array: [type=int[]] build diff --git a/pkg/sql/opt/optbuilder/testdata/select b/pkg/sql/opt/optbuilder/testdata/select index 4798416aff2e..f80669e63206 100644 --- a/pkg/sql/opt/optbuilder/testdata/select +++ b/pkg/sql/opt/optbuilder/testdata/select @@ -153,9 +153,9 @@ project └── projections └── plus [type=int] ├── plus [type=int] - │ ├── variable: abc.a [type=int] - │ └── variable: abc.b [type=int] - └── variable: abc.c [type=int] + │ ├── variable: a [type=int] + │ └── variable: b [type=int] + └── variable: c [type=int] build allow-unsupported SELECT a,b FROM abc WHERE CASE WHEN a != 0 THEN b/a > 1.5 ELSE false END @@ -171,12 +171,12 @@ project ├── true [type=bool] ├── when [type=bool] │ ├── ne [type=bool] - │ │ ├── variable: abc.a [type=int] + │ │ ├── variable: a [type=int] │ │ └── const: 0 [type=int] │ └── gt [type=bool] │ ├── div [type=decimal] - │ │ ├── variable: abc.b [type=int] - │ │ └── variable: abc.a [type=int] + │ │ ├── variable: b [type=int] + │ │ └── variable: a [type=int] │ └── const: 1.5 [type=decimal] └── false [type=bool] @@ -212,7 +212,7 @@ project │ └── columns: k:1(string!null) v:2(string) └── projections └── concat [type=string] - ├── variable: kv.v [type=string] + ├── variable: v [type=string] └── const: 'foo' [type=string] build @@ -224,7 +224,7 @@ project │ └── columns: k:1(string!null) v:2(string) └── projections └── function: lower [type=string] - └── variable: kv.v [type=string] + └── variable: v [type=string] build SELECT k FROM kv @@ -255,8 +255,8 @@ project │ └── columns: k:1(string!null) v:2(string) └── projections └── tuple [type=tuple{string AS k, string AS v}] - ├── variable: kv.k [type=string] - └── variable: kv.v [type=string] + ├── variable: k [type=string] + └── variable: v [type=string] build SELECT foo.* FROM kv @@ -300,7 +300,7 @@ project │ └── columns: k:1(string!null) v:2(string) └── filters [type=bool] └── eq [type=bool] - ├── variable: kv.k [type=string] + ├── variable: k [type=string] └── const: 'a' [type=string] build @@ -314,7 +314,7 @@ project │ └── columns: k:1(string!null) v:2(string) └── filters [type=bool] └── eq [type=bool] - ├── variable: kv.k [type=string] + ├── variable: k [type=string] └── const: 'a' [type=string] exec-ddl @@ -425,8 +425,8 @@ project │ └── columns: x:1(int!null) y:2(int) z:3(int) w:4(int) └── projections └── tuple [type=tuple{int, int}] - ├── variable: xyzw.x [type=int] - └── variable: xyzw.y [type=int] + ├── variable: x [type=int] + └── variable: y [type=int] build SELECT * FROM xyzw LIMIT 0 @@ -602,13 +602,13 @@ project └── projections ├── mult [type=int] │ ├── const: 0 [type=int] - │ └── variable: abc.b [type=int] + │ └── variable: b [type=int] ├── mod [type=int] - │ ├── variable: abc.b [type=int] + │ ├── variable: b [type=int] │ └── const: 1 [type=int] └── mod [type=int] ├── const: 0 [type=int] - └── variable: abc.b [type=int] + └── variable: b [type=int] # Regression tests for #22670. build @@ -998,7 +998,7 @@ select │ └── columns: x:1(int!null) y:2(float) └── filters [type=bool] └── gt [type=bool] - ├── variable: a.x [type=int] + ├── variable: x [type=int] └── const: 10 [type=int] build @@ -1011,14 +1011,14 @@ select └── filters [type=bool] └── and [type=bool] ├── gt [type=bool] - │ ├── variable: a.x [type=int] + │ ├── variable: x [type=int] │ └── const: 10 [type=int] └── and [type=bool] ├── lt [type=bool] - │ ├── variable: a.x [type=int] + │ ├── variable: x [type=int] │ └── const: 20 [type=int] └── ne [type=bool] - ├── variable: a.x [type=int] + ├── variable: x [type=int] └── const: 13 [type=int] build @@ -1030,7 +1030,7 @@ select │ └── columns: x:1(int!null) y:2(float) └── filters [type=bool] └── in [type=bool] - ├── variable: a.x [type=int] + ├── variable: x [type=int] └── tuple [type=tuple{int, int, int}] ├── const: 1 [type=int] ├── const: 2 [type=int] @@ -1050,8 +1050,8 @@ project ├── scan a │ └── columns: x:1(int!null) y:2(float) └── projections - ├── variable: a.x [type=int] - └── variable: a.y [type=float] + ├── variable: x [type=int] + └── variable: y [type=float] build SELECT * FROM a WHERE (x > 10)::bool @@ -1063,7 +1063,7 @@ select └── filters [type=bool] └── cast: BOOL [type=bool] └── gt [type=bool] - ├── variable: a.x [type=int] + ├── variable: x [type=int] └── const: 10 [type=int] build @@ -1080,7 +1080,7 @@ select │ └── columns: x:1(int!null) y:2(float) └── filters [type=bool] └── eq [type=bool] - ├── variable: a.x [type=int] + ├── variable: x [type=int] └── placeholder: $1 [type=int] # This is slightly funky, because the AS OF SYSTEM TIME timestamp only gets diff --git a/pkg/sql/opt/optbuilder/testdata/srfs b/pkg/sql/opt/optbuilder/testdata/srfs index e889336fb0c7..935da00c7850 100644 --- a/pkg/sql/opt/optbuilder/testdata/srfs +++ b/pkg/sql/opt/optbuilder/testdata/srfs @@ -524,7 +524,7 @@ build SELECT information_schema._pg_expandarray(ARRAY['b', 'a']) ---- project - ├── columns: _pg_expandarray:3(tuple{string AS x, int AS n}) + ├── columns: information_schema._pg_expandarray:3(tuple{string AS x, int AS n}) ├── inner-join-apply │ ├── columns: x:1(string) n:2(int) │ ├── values @@ -748,12 +748,12 @@ project │ │ │ └── columns: a:1(string) t.rowid:2(int!null) │ │ └── projections │ │ └── cast: INT [type=int] - │ │ └── variable: t.a [type=string] + │ │ └── variable: a [type=string] │ └── aggregations │ ├── max [type=int] │ │ └── variable: column6 [type=int] │ └── max [type=string] - │ └── variable: t.a [type=string] + │ └── variable: a [type=string] └── filters [type=bool] └── gt [type=bool] ├── variable: column7 [type=int] @@ -773,7 +773,7 @@ project │ │ └── function: generate_series [type=int] │ │ ├── const: 0 [type=int] │ │ └── cast: INT [type=int] - │ │ └── variable: u.b [type=string] + │ │ └── variable: b [type=string] │ └── true [type=bool] └── const: 1 [type=int] @@ -810,7 +810,7 @@ project │ │ │ ├── variable: t.a [type=string] │ │ │ └── concat [type=string] │ │ │ ├── variable: t.a [type=string] - │ │ │ └── variable: u.b [type=string] + │ │ │ └── variable: b [type=string] │ │ └── true [type=bool] │ └── const: 100 [type=int] └── true [type=bool] diff --git a/pkg/sql/opt/optbuilder/testdata/subquery b/pkg/sql/opt/optbuilder/testdata/subquery index 4a97d7214e1f..5e11b333c317 100644 --- a/pkg/sql/opt/optbuilder/testdata/subquery +++ b/pkg/sql/opt/optbuilder/testdata/subquery @@ -660,8 +660,8 @@ project │ │ └── columns: a:1(int!null) b:2(int) c:3(int) │ └── projections │ └── tuple [type=tuple{int, int}] - │ ├── variable: abc.a [type=int] - │ └── variable: abc.b [type=int] + │ ├── variable: a [type=int] + │ └── variable: b [type=int] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] └── const: 2 [type=int] @@ -687,8 +687,8 @@ project │ │ └── false [type=bool] │ └── projections │ └── tuple [type=tuple{int, int}] - │ ├── variable: abc.a [type=int] - │ └── variable: abc.b [type=int] + │ ├── variable: a [type=int] + │ └── variable: b [type=int] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] └── const: 2 [type=int] @@ -788,7 +788,7 @@ select │ └── columns: a:1(int!null) b:2(int) c:3(int) └── filters [type=bool] └── eq [type=bool] - ├── variable: abc.a [type=int] + ├── variable: a [type=int] └── const: 7 [type=int] exec-ddl @@ -927,7 +927,7 @@ project │ │ └── columns: k:1(int!null) v:2(string) │ └── filters [type=bool] │ └── eq [type=bool] - │ ├── variable: kv.k [type=int] + │ ├── variable: k [type=int] │ └── const: 1 [type=int] └── projections └── const: 1 [type=int] @@ -949,7 +949,7 @@ project │ │ └── columns: k:1(int!null) v:2(string) │ └── filters [type=bool] │ └── eq [type=bool] - │ ├── variable: kv.k [type=int] + │ ├── variable: k [type=int] │ └── const: 2 [type=int] └── projections └── const: 1 [type=int] @@ -1041,7 +1041,7 @@ select │ └── columns: x:1(int!null) y:2(int) z:3(int) └── filters [type=bool] └── lt [type=bool] - ├── variable: xyz.x [type=int] + ├── variable: x [type=int] └── const: 7 [type=int] build @@ -1053,7 +1053,7 @@ select │ └── columns: x:1(int!null) y:2(int) z:3(int) └── filters [type=bool] └── lt [type=bool] - ├── variable: xyz.x [type=int] + ├── variable: x [type=int] └── const: 7 [type=int] build @@ -1065,7 +1065,7 @@ select │ └── columns: x:1(int!null) y:2(int) z:3(int) └── filters [type=bool] └── lt [type=bool] - ├── variable: xyz.x [type=int] + ├── variable: x [type=int] └── const: 7 [type=int] build @@ -1077,7 +1077,7 @@ select │ └── columns: x:1(int!null) y:2(int) z:3(int) └── filters [type=bool] └── lt [type=bool] - ├── variable: xyz.x [type=int] + ├── variable: x [type=int] └── const: 7 [type=int] build @@ -1092,7 +1092,7 @@ sort │ └── columns: x:1(int!null) y:2(int) z:3(int) └── filters [type=bool] └── lt [type=bool] - ├── variable: xyz.x [type=int] + ├── variable: x [type=int] └── const: 7 [type=int] build @@ -1941,7 +1941,7 @@ select │ │ └── columns: a:1(string) t.rowid:2(int!null) │ └── aggregations │ └── max [type=string] - │ └── variable: t.a [type=string] + │ └── variable: a [type=string] └── filters [type=bool] └── any: lt [type=bool] ├── project @@ -1963,7 +1963,7 @@ project │ │ └── columns: a:1(string) t.rowid:2(int!null) │ └── aggregations │ └── min [type=string] - │ └── variable: t.a [type=string] + │ └── variable: a [type=string] └── projections └── any: eq [type=bool] ├── project diff --git a/pkg/sql/opt/optbuilder/testdata/view b/pkg/sql/opt/optbuilder/testdata/view index f37a067c16bc..70d333816145 100644 --- a/pkg/sql/opt/optbuilder/testdata/view +++ b/pkg/sql/opt/optbuilder/testdata/view @@ -79,7 +79,7 @@ project │ └── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) └── filters [type=bool] └── eq [type=bool] - ├── variable: a.i [type=int] + ├── variable: i [type=int] └── const: 10 [type=int] # Sort used by group by because of presence of ARRAY_AGG. @@ -102,8 +102,8 @@ scalar-group-by │ │ └── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) │ └── filters [type=bool] │ └── eq [type=bool] - │ ├── variable: a.i [type=int] + │ ├── variable: i [type=int] │ └── const: 10 [type=int] └── aggregations └── array-agg [type=float[]] - └── variable: a.f [type=float] + └── variable: f [type=float] diff --git a/pkg/sql/opt/optbuilder/testdata/where b/pkg/sql/opt/optbuilder/testdata/where index c753ff7a37c4..39b113b23f3e 100644 --- a/pkg/sql/opt/optbuilder/testdata/where +++ b/pkg/sql/opt/optbuilder/testdata/where @@ -33,7 +33,7 @@ select │ └── columns: k:1(int!null) v:2(int) └── filters [type=bool] └── in [type=bool] - ├── variable: kv.k [type=int] + ├── variable: k [type=int] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] └── const: 3 [type=int] @@ -47,7 +47,7 @@ select │ └── columns: k:1(int!null) v:2(int) └── filters [type=bool] └── in [type=bool] - ├── variable: kv.v [type=int] + ├── variable: v [type=int] └── tuple [type=tuple{int}] └── const: 6 [type=int] @@ -106,12 +106,12 @@ project │ │ └── ordering: +1 │ └── filters [type=bool] │ └── like [type=bool] - │ ├── variable: kvstring.k [type=string] + │ ├── variable: k [type=string] │ └── const: 'like%' [type=string] └── projections └── like [type=bool] ├── const: 'hello' [type=string] - └── variable: kvstring.v [type=string] + └── variable: v [type=string] build SELECT 'hello' SIMILAR TO v AS r FROM kvString WHERE k SIMILAR TO 'like[1-2]' ORDER BY k @@ -127,12 +127,12 @@ project │ │ └── ordering: +1 │ └── filters [type=bool] │ └── similar-to [type=bool] - │ ├── variable: kvstring.k [type=string] + │ ├── variable: k [type=string] │ └── const: 'like[1-2]' [type=string] └── projections └── similar-to [type=bool] ├── const: 'hello' [type=string] - └── variable: kvstring.v [type=string] + └── variable: v [type=string] build SELECT 'hello' ~ replace(v, '%', '.*') AS r FROM kvString WHERE k ~ 'like[1-2]' ORDER BY k @@ -148,13 +148,13 @@ project │ │ └── ordering: +1 │ └── filters [type=bool] │ └── reg-match [type=bool] - │ ├── variable: kvstring.k [type=string] + │ ├── variable: k [type=string] │ └── const: 'like[1-2]' [type=string] └── projections └── reg-match [type=bool] ├── const: 'hello' [type=string] └── function: replace [type=string] - ├── variable: kvstring.v [type=string] + ├── variable: v [type=string] ├── const: '%' [type=string] └── const: '.*' [type=string] @@ -169,7 +169,7 @@ select │ └── columns: k:1(int!null) v:2(int) └── filters [type=bool] └── in [type=bool] - ├── variable: kv.k [type=int] + ├── variable: k [type=int] └── tuple [type=tuple{int, int, int}] ├── const: 1 [type=int] ├── const: 5 [type=int] @@ -197,7 +197,7 @@ project │ └── columns: a:1(int) b:2(int) rowid:3(int!null) └── filters [type=bool] └── in [type=bool] - ├── variable: ab.a [type=int] + ├── variable: a [type=int] └── tuple [type=tuple{int, int, int}] ├── const: 1 [type=int] ├── const: 3 [type=int] @@ -214,7 +214,7 @@ project │ └── columns: a:1(int) b:2(int) rowid:3(int!null) └── filters [type=bool] └── in [type=bool] - ├── variable: ab.a [type=int] + ├── variable: a [type=int] └── tuple [type=tuple{int, int, int, int}] ├── const: 1 [type=int] ├── const: 3 [type=int] @@ -233,8 +233,8 @@ project └── filters [type=bool] └── in [type=bool] ├── tuple [type=tuple{int, int}] - │ ├── variable: ab.a [type=int] - │ └── variable: ab.b [type=int] + │ ├── variable: a [type=int] + │ └── variable: b [type=int] └── tuple [type=tuple{tuple{int, int}, tuple{int, int}, tuple{int, int}}] ├── tuple [type=tuple{int, int}] │ ├── const: 1 [type=int] @@ -258,8 +258,8 @@ project └── filters [type=bool] └── in [type=bool] ├── tuple [type=tuple{int, int}] - │ ├── variable: ab.a [type=int] - │ └── variable: ab.b [type=int] + │ ├── variable: a [type=int] + │ └── variable: b [type=int] └── tuple [type=tuple{tuple{int, int}, tuple{int, int}, tuple{int, int}}] ├── tuple [type=tuple{int, int}] │ ├── const: 1 [type=int] diff --git a/pkg/sql/opt/props/logical.go b/pkg/sql/opt/props/logical.go index 8b52f921e248..4a5c2be25be6 100644 --- a/pkg/sql/opt/props/logical.go +++ b/pkg/sql/opt/props/logical.go @@ -15,15 +15,11 @@ package props import ( - "bytes" "fmt" - "strings" - "unicode" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" - "github.com/cockroachdb/cockroach/pkg/util/treeprinter" ) // Logical properties describe the content and characteristics of data returned @@ -471,112 +467,3 @@ func (p *Logical) Verify() { } } } - -// FormatColSet outputs the specified set of columns using FormatCol to format -// the output. -func (p *Logical) FormatColSet( - f *opt.ExprFmtCtx, tp treeprinter.Node, heading string, colSet opt.ColSet, -) { - if !colSet.Empty() { - var buf bytes.Buffer - buf.WriteString(heading) - colSet.ForEach(func(i int) { - p.FormatCol(f, &buf, "", opt.ColumnID(i)) - }) - tp.Child(buf.String()) - } -} - -// FormatColList outputs the specified list of columns using FormatCol to -// format the output. -func (p *Logical) FormatColList( - f *opt.ExprFmtCtx, tp treeprinter.Node, heading string, colList opt.ColList, -) { - if len(colList) > 0 { - var buf bytes.Buffer - buf.WriteString(heading) - for _, col := range colList { - p.FormatCol(f, &buf, "", col) - } - tp.Child(buf.String()) - } -} - -// FormatCol outputs the specified column using the following format: -// label:index(type) -// -// If the column is not nullable, then this is the format: -// label:index(type!null) -// -// If a label is given, then it is used. Otherwise, a "best effort" label is -// used from query metadata. -func (p *Logical) FormatCol(f *opt.ExprFmtCtx, buf *bytes.Buffer, label string, id opt.ColumnID) { - if label == "" { - label = f.Metadata().ColumnLabel(id) - } - - if !isSimpleColumnName(label) { - // Add quotations around the column name if it appears to be an - // expression. This also indicates that the column name is not eligible - // to be shortened. - label = "\"" + label + "\"" - } else if f.HasFlags(opt.ExprFmtHideQualifications) { - // If the label is qualified, try to shorten it. - if idx := strings.LastIndex(label, "."); idx != -1 { - short := label[idx+1:] - suffix := label[idx:] // includes the "." - // Check if shortening the label could cause ambiguity: is there another - // column that would be shortened to the same name? - ambiguous := false - for col := opt.ColumnID(1); int(col) <= f.Metadata().NumColumns(); col++ { - if col != id { - if l := f.Metadata().ColumnLabel(col); l == short || strings.HasSuffix(l, suffix) { - ambiguous = true - break - } - } - } - if !ambiguous { - label = short - } - } - } - - typ := f.Metadata().ColumnType(id) - buf.WriteByte(' ') - buf.WriteString(label) - buf.WriteByte(':') - fmt.Fprintf(buf, "%d", id) - buf.WriteByte('(') - buf.WriteString(typ.String()) - - if p.Relational.NotNullCols.Contains(int(id)) { - buf.WriteString("!null") - } - buf.WriteByte(')') -} - -// isSimpleColumnName returns true if the given label consists of only ASCII -// letters, numbers, underscores, quotation marks, and periods ("."). It is -// used to determine whether or not we can shorten a column label by removing -// the prefix up to the last ".". Although isSimpleColumnName excludes some -// valid table column names, it ensures that we don't shorten expressions such -// as "a.x + b.x" to "x". It is better to err on the side of not shortening -// than to incorrectly shorten a column name representing an expression. -func isSimpleColumnName(label string) bool { - for i, r := range label { - if r > unicode.MaxASCII { - return false - } - - if i == 0 { - if r != '"' && !unicode.IsLetter(r) { - // The first character must be a letter or quotation mark. - return false - } - } else if r != '.' && r != '_' && r != '"' && !unicode.IsNumber(r) && !unicode.IsLetter(r) { - return false - } - } - return true -} diff --git a/pkg/sql/opt/props/logical_test.go b/pkg/sql/opt/props/logical_test.go deleted file mode 100644 index c87c4a78b49e..000000000000 --- a/pkg/sql/opt/props/logical_test.go +++ /dev/null @@ -1,72 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or -// implied. See the License for the specific language governing -// permissions and limitations under the License. - -package props_test - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/sql/opt" - "github.com/cockroachdb/cockroach/pkg/sql/opt/props" - "github.com/cockroachdb/cockroach/pkg/sql/sem/types" - "github.com/cockroachdb/cockroach/pkg/util/treeprinter" -) - -func TestLogicalProps(t *testing.T) { - md := opt.NewMetadata() - i := md.AddColumn("i", types.Int) - d := md.AddColumn("d", types.Decimal) - s := md.AddColumn("s", types.String) - - outCols := opt.ColSet{} - outCols.Add(int(i)) - - outerCols := opt.ColSet{} - outerCols.Add(int(d)) - outerCols.Add(int(s)) - - colList := opt.ColList{s, i} - - tp := treeprinter.New() - nd := tp.Child("props") - - relational := &props.Logical{ - Relational: &props.Relational{ - OutputCols: outCols, - OuterCols: outerCols, - NotNullCols: outCols, - }, - } - - scalar := &props.Logical{ - Scalar: &props.Scalar{OuterCols: outerCols}, - } - - f := opt.MakeExprFmtCtx(md, 0 /* flags */) - relational.FormatColSet(&f, nd, "output:", relational.Relational.OutputCols) - relational.FormatColSet(&f, nd, "outer relational:", relational.OuterCols()) - relational.FormatColList(&f, nd, "list:", colList) - relational.FormatColSet(&f, nd, "outer scalar:", scalar.OuterCols()) - - expected := "props\n" + - " ├── output: i:1(int!null)\n" + - " ├── outer relational: d:2(decimal) s:3(string)\n" + - " ├── list: s:3(string) i:1(int!null)\n" + - " └── outer scalar: d:2(decimal) s:3(string)\n" - - actual := tp.String() - if actual != expected { - t.Fatalf("expected:\n%s\nactual:\n%s\n", expected, actual) - } -} diff --git a/pkg/sql/opt/testutils/format.go b/pkg/sql/opt/testutils/format.go index 00400e64226f..d0be484dcc8b 100644 --- a/pkg/sql/opt/testutils/format.go +++ b/pkg/sql/opt/testutils/format.go @@ -15,8 +15,6 @@ package testutils import ( - "bytes" - "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" @@ -33,8 +31,8 @@ func init() { // fmtInterceptor is a function suitable for memo.ExprFmtInterceptor. It detects // if an expression tree contains only scalar expressions; if so, it tries to // execbuild them and print the SQL expressions. -func fmtInterceptor(f *opt.ExprFmtCtx, tp treeprinter.Node, ev memo.ExprView) bool { - if !f.HasFlags(opt.ExprFmtHideScalars) || !onlyScalars(ev) { +func fmtInterceptor(f *memo.ExprFmtCtx, tp treeprinter.Node, ev memo.ExprView) bool { + if !f.HasFlags(memo.ExprFmtHideScalars) || !onlyScalars(ev) { return false } @@ -57,14 +55,16 @@ func fmtInterceptor(f *opt.ExprFmtCtx, tp treeprinter.Node, ev memo.ExprView) bo // Not all scalar operators are supported (e.g. Projections). return false } - var buf bytes.Buffer - fmtCtx := tree.MakeFmtCtx(&buf, tree.FmtSimple) + f.Buffer.Reset() + fmtCtx := tree.MakeFmtCtx(f.Buffer, tree.FmtSimple) fmtCtx.WithIndexedVarFormat(func(ctx *tree.FmtCtx, idx int) { - ctx.WriteString(md.ColumnLabel(opt.ColumnID(idx + 1))) + fullyQualify := !f.HasFlags(memo.ExprFmtHideQualifications) + label := md.QualifiedColumnLabel(opt.ColumnID(idx+1), fullyQualify) + ctx.WriteString(label) }) expr.Format(&fmtCtx) - ev.FormatScalarProps(f, &buf) - tp.Child(buf.String()) + ev.FormatScalarProps(f) + tp.Child(f.Buffer.String()) return true } diff --git a/pkg/sql/opt/testutils/opt_tester.go b/pkg/sql/opt/testutils/opt_tester.go index a08a85bd1a14..90758af73f9b 100644 --- a/pkg/sql/opt/testutils/opt_tester.go +++ b/pkg/sql/opt/testutils/opt_tester.go @@ -63,7 +63,7 @@ type OptTester struct { type OptTesterFlags struct { // Format controls the output detail of build / opt/ optsteps command // directives. - ExprFormat opt.ExprFmtFlags + ExprFormat memo.ExprFmtFlags // MemoFormat controls the output detail of memo command directives. MemoFormat memo.FmtFlags @@ -144,8 +144,9 @@ func NewOptTester(catalog opt.Catalog, sql string) *OptTester { // // - format: controls the formatting of expressions for build, opt, and // optsteps commands. Possible values: show-all, hide-all, or any combination -// of hide-cost, hide-stats, hide-constraints, hide-scalars. Example: -// build format={hide-cost,hide-stats} +// of hide-cost, hide-stats, hide-constraints, hide-scalars, hide-qual. +// For example: +// build format=(hide-cost,hide-stats) // // - raw-memo: show the raw memo groups, in the order they were originally // added, including any "orphaned" groups. @@ -269,14 +270,15 @@ func (f *OptTesterFlags) Set(arg datadriven.CmdArg) error { return fmt.Errorf("format flag requires value(s)") } for _, v := range arg.Vals { - m := map[string]opt.ExprFmtFlags{ - "show-all": opt.ExprFmtShowAll, - "hide-all": opt.ExprFmtHideAll, - "hide-stats": opt.ExprFmtHideStats, - "hide-cost": opt.ExprFmtHideCost, - "hide-constraints": opt.ExprFmtHideConstraints, - "hide-ruleprops": opt.ExprFmtHideRuleProps, - "hide-scalars": opt.ExprFmtHideScalars, + m := map[string]memo.ExprFmtFlags{ + "show-all": memo.ExprFmtShowAll, + "hide-all": memo.ExprFmtHideAll, + "hide-stats": memo.ExprFmtHideStats, + "hide-cost": memo.ExprFmtHideCost, + "hide-constraints": memo.ExprFmtHideConstraints, + "hide-ruleprops": memo.ExprFmtHideRuleProps, + "hide-scalars": memo.ExprFmtHideScalars, + "hide-qual": memo.ExprFmtHideQualifications, } if val, ok := m[v]; ok { f.ExprFormat |= val @@ -294,7 +296,7 @@ func (f *OptTesterFlags) Set(arg datadriven.CmdArg) error { case "fully-qualify-names": f.FullyQualifyNames = true // Hiding qualifications defeats the purpose. - f.ExprFormat &= ^opt.ExprFmtHideQualifications + f.ExprFormat &= ^memo.ExprFmtHideQualifications case "rule": if len(arg.Vals) != 1 { diff --git a/pkg/sql/opt/testutils/testcat/create_table.go b/pkg/sql/opt/testutils/testcat/create_table.go index a9fe2d527a36..40580d1f6281 100644 --- a/pkg/sql/opt/testutils/testcat/create_table.go +++ b/pkg/sql/opt/testutils/testcat/create_table.go @@ -173,7 +173,7 @@ func (tt *Table) addIndex(def *tree.IndexTableDef, typ indexType) { // Only add columns that aren't already part of index. found := false for _, colDef := range def.Columns { - if pkCol.Column.ColName() == opt.ColumnName(colDef.Column) { + if pkCol.Column.ColName() == colDef.Column { found = true } } @@ -206,7 +206,7 @@ func (tt *Table) addIndex(def *tree.IndexTableDef, typ indexType) { // key columns. found := false for _, pkCol := range pkCols { - if opt.ColumnName(name) == pkCol.Column.ColName() { + if name == pkCol.Column.ColName() { found = true } } diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index 3572e025ad02..a5b4a5018858 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -398,8 +398,8 @@ func (tc *Column) IsNullable() bool { } // ColName is part of the opt.Column interface. -func (tc *Column) ColName() opt.ColumnName { - return opt.ColumnName(tc.Name) +func (tc *Column) ColName() tree.Name { + return tree.Name(tc.Name) } // DatumType is part of the opt.Column interface. diff --git a/pkg/sql/opt/xform/optimizer_test.go b/pkg/sql/opt/xform/optimizer_test.go index 40265669ffa1..facb44668cd5 100644 --- a/pkg/sql/opt/xform/optimizer_test.go +++ b/pkg/sql/opt/xform/optimizer_test.go @@ -17,7 +17,7 @@ package xform_test import ( "testing" - "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat" "github.com/cockroachdb/cockroach/pkg/testutils/datadriven" @@ -30,7 +30,7 @@ import ( func TestCoster(t *testing.T) { runDataDrivenTest( t, "testdata/coster/", - opt.ExprFmtHideRuleProps|opt.ExprFmtHideQualifications|opt.ExprFmtHideScalars, + memo.ExprFmtHideRuleProps|memo.ExprFmtHideQualifications|memo.ExprFmtHideScalars, ) } @@ -39,7 +39,7 @@ func TestCoster(t *testing.T) { // make test PKG=./pkg/sql/opt/xform TESTS="TestPhysicalPropsFactory/presentation" // ... func TestPhysicalPropsFactory(t *testing.T) { - runDataDrivenTest(t, "testdata/physprops/", opt.ExprFmtHideAll) + runDataDrivenTest(t, "testdata/physprops/", memo.ExprFmtHideAll) } // TestRuleProps files can be run separately like this: @@ -50,8 +50,8 @@ func TestRuleProps(t *testing.T) { catalog := testcat.New() datadriven.RunTest(t, path, func(d *datadriven.TestData) string { tester := testutils.NewOptTester(catalog, d.Input) - tester.Flags.ExprFormat = opt.ExprFmtHideStats | opt.ExprFmtHideCost | - opt.ExprFmtHideQualifications | opt.ExprFmtHideScalars + tester.Flags.ExprFormat = memo.ExprFmtHideStats | memo.ExprFmtHideCost | + memo.ExprFmtHideQualifications | memo.ExprFmtHideScalars return tester.RunCommand(t, d) }) }) @@ -65,8 +65,8 @@ func TestRules(t *testing.T) { runDataDrivenTest( t, "testdata/rules/", - opt.ExprFmtHideStats|opt.ExprFmtHideCost|opt.ExprFmtHideRuleProps| - opt.ExprFmtHideQualifications|opt.ExprFmtHideScalars, + memo.ExprFmtHideStats|memo.ExprFmtHideCost|memo.ExprFmtHideRuleProps| + memo.ExprFmtHideQualifications|memo.ExprFmtHideScalars, ) } @@ -81,8 +81,8 @@ func TestExternal(t *testing.T) { runDataDrivenTest( t, "testdata/external/", - opt.ExprFmtHideStats|opt.ExprFmtHideCost|opt.ExprFmtHideRuleProps| - opt.ExprFmtHideQualifications|opt.ExprFmtHideScalars, + memo.ExprFmtHideStats|memo.ExprFmtHideCost|memo.ExprFmtHideRuleProps| + memo.ExprFmtHideQualifications|memo.ExprFmtHideScalars, ) } @@ -93,7 +93,7 @@ func TestExternal(t *testing.T) { // // // See OptTester.Handle for supported commands. -func runDataDrivenTest(t *testing.T, path string, fmtFlags opt.ExprFmtFlags) { +func runDataDrivenTest(t *testing.T, path string, fmtFlags memo.ExprFmtFlags) { datadriven.Walk(t, path, func(t *testing.T, path string) { catalog := testcat.New() datadriven.RunTest(t, path, func(d *datadriven.TestData) string { diff --git a/pkg/sql/opt/xform/testdata/coster/groupby b/pkg/sql/opt/xform/testdata/coster/groupby index 5d04f03f8dbe..1f7ede35469f 100644 --- a/pkg/sql/opt/xform/testdata/coster/groupby +++ b/pkg/sql/opt/xform/testdata/coster/groupby @@ -27,6 +27,6 @@ group-by │ └── fd: (1)-->(2,3) └── aggregations [outer=(1)] ├── max [type=int, outer=(1)] - │ └── variable: a.k [type=int, outer=(1)] + │ └── variable: k [type=int, outer=(1)] └── min [type=int, outer=(1)] - └── variable: a.k [type=int, outer=(1)] + └── variable: k [type=int, outer=(1)] diff --git a/pkg/sql/opt/xform/testdata/coster/join b/pkg/sql/opt/xform/testdata/coster/join index 6aa5c582eb05..980ae4fbcf9e 100644 --- a/pkg/sql/opt/xform/testdata/coster/join +++ b/pkg/sql/opt/xform/testdata/coster/join @@ -49,9 +49,9 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(4) │ └── filters [type=bool, outer=(4), constraints=(/4: [/1.0 - /1.0]; tight), fd=()-->(4)] - │ └── a.d = 1.0 [type=bool, outer=(4), constraints=(/4: [/1.0 - /1.0]; tight)] + │ └── d = 1.0 [type=bool, outer=(4), constraints=(/4: [/1.0 - /1.0]; tight)] └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── a.k = b.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── k = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] exec-ddl CREATE TABLE abc (a INT PRIMARY KEY, b INT, c INT, INDEX c_idx (c)) diff --git a/pkg/sql/opt/xform/testdata/coster/project b/pkg/sql/opt/xform/testdata/coster/project index 24b1dbf4d93d..fd99fc2905df 100644 --- a/pkg/sql/opt/xform/testdata/coster/project +++ b/pkg/sql/opt/xform/testdata/coster/project @@ -25,7 +25,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2,3) └── projections [outer=(1-3)] - └── a.s || 'foo' [type=string, outer=(3)] + └── s || 'foo' [type=string, outer=(3)] opt SELECT k, k+2, i*d FROM a @@ -43,5 +43,5 @@ project │ ├── key: (1) │ └── fd: (1)-->(2,4) └── projections [outer=(1,2,4)] - ├── a.k + 2 [type=int, outer=(1)] - └── a.i * a.d [type=decimal, outer=(2,4)] + ├── k + 2 [type=int, outer=(1)] + └── i * d [type=decimal, outer=(2,4)] diff --git a/pkg/sql/opt/xform/testdata/coster/select b/pkg/sql/opt/xform/testdata/coster/select index 038c448f9e98..e6e8686b1cf2 100644 --- a/pkg/sql/opt/xform/testdata/coster/select +++ b/pkg/sql/opt/xform/testdata/coster/select @@ -25,4 +25,4 @@ select │ ├── key: (1) │ └── fd: (1)-->(3) └── filters [type=bool, outer=(3), constraints=(/3: [/'foo' - ]; tight)] - └── a.s >= 'foo' [type=bool, outer=(3), constraints=(/3: [/'foo' - ]; tight)] + └── s >= 'foo' [type=bool, outer=(3), constraints=(/3: [/'foo' - ]; tight)] diff --git a/pkg/sql/opt/xform/testdata/coster/virtual-scan b/pkg/sql/opt/xform/testdata/coster/virtual-scan index 9bb55f6f4c66..b25356634ced 100644 --- a/pkg/sql/opt/xform/testdata/coster/virtual-scan +++ b/pkg/sql/opt/xform/testdata/coster/virtual-scan @@ -25,4 +25,4 @@ select │ ├── stats: [rows=1000, distinct(2)=700] │ └── cost: 10 └── filters [type=bool, outer=(2), constraints=(/2: [/'public' - /'public']; tight), fd=()-->(2)] - └── system.information_schema.schemata.schema_name = 'public' [type=bool, outer=(2), constraints=(/2: [/'public' - /'public']; tight)] + └── schema_name = 'public' [type=bool, outer=(2), constraints=(/2: [/'public' - /'public']; tight)] diff --git a/pkg/sql/opt/xform/testdata/external/activerecord b/pkg/sql/opt/xform/testdata/external/activerecord index 73b7a5763576..9b5caff02dc0 100644 --- a/pkg/sql/opt/xform/testdata/external/activerecord +++ b/pkg/sql/opt/xform/testdata/external/activerecord @@ -224,7 +224,7 @@ sort │ │ │ ├── key: (36) │ │ │ └── fd: (36)-->(63) │ │ └── filters [type=bool, outer=(29,63), constraints=(/29: (/NULL - ]; /63: (/NULL - ])] - │ │ └── pg_collation.oid != pg_type.typcollation [type=bool, outer=(29,63), constraints=(/29: (/NULL - ]; /63: (/NULL - ])] + │ │ └── pg_collation.oid != typcollation [type=bool, outer=(29,63), constraints=(/29: (/NULL - ]; /63: (/NULL - ])] │ ├── left-join (lookup pg_attrdef) │ │ ├── columns: attrelid:1(oid!null) attname:2(string!null) atttypid:3(oid!null) attnum:6(int!null) atttypmod:9(int!null) attnotnull:13(bool!null) attisdropped:15(bool!null) attcollation:18(oid!null) adrelid:23(oid) adnum:24(int) adbin:25(string) │ │ ├── key columns: [22] = [22] @@ -244,20 +244,20 @@ sort │ │ │ │ │ ├── key: (1,6) │ │ │ │ │ └── fd: (1,6)-->(2,3,9,13,15,18), (1,2)-->(3,6,9,13,15,18) │ │ │ │ └── filters [type=bool, outer=(1,6,15), constraints=(/1: (/NULL - ]; /6: [/1 - ]; /15: [/false - /false]), fd=()-->(15)] - │ │ │ │ ├── pg_attribute.attrelid = '"numbers"'::REGCLASS [type=bool, outer=(1), constraints=(/1: (/NULL - ])] - │ │ │ │ ├── pg_attribute.attnum > 0 [type=bool, outer=(6), constraints=(/6: [/1 - ]; tight)] - │ │ │ │ └── NOT pg_attribute.attisdropped [type=bool, outer=(15), constraints=(/15: [/false - /false]; tight)] + │ │ │ │ ├── attrelid = '"numbers"'::REGCLASS [type=bool, outer=(1), constraints=(/1: (/NULL - ])] + │ │ │ │ ├── attnum > 0 [type=bool, outer=(6), constraints=(/6: [/1 - ]; tight)] + │ │ │ │ └── NOT attisdropped [type=bool, outer=(15), constraints=(/15: [/false - /false]; tight)] │ │ │ └── filters [type=bool, outer=(1,6,23,24), constraints=(/1: (/NULL - ]; /6: (/NULL - ]; /23: (/NULL - ]; /24: [/1 - ]), fd=(1)==(23), (23)==(1), (6)==(24), (24)==(6)] - │ │ │ ├── pg_attribute.attrelid = pg_attrdef.adrelid [type=bool, outer=(1,23), constraints=(/1: (/NULL - ]; /23: (/NULL - ])] - │ │ │ ├── pg_attribute.attnum = pg_attrdef.adnum [type=bool, outer=(6,24), constraints=(/6: (/NULL - ]; /24: (/NULL - ])] - │ │ │ ├── pg_attrdef.adrelid = '"numbers"'::REGCLASS [type=bool, outer=(23), constraints=(/23: (/NULL - ])] - │ │ │ └── pg_attrdef.adnum > 0 [type=bool, outer=(24), constraints=(/24: [/1 - ]; tight)] + │ │ │ ├── attrelid = adrelid [type=bool, outer=(1,23), constraints=(/1: (/NULL - ]; /23: (/NULL - ])] + │ │ │ ├── attnum = adnum [type=bool, outer=(6,24), constraints=(/6: (/NULL - ]; /24: (/NULL - ])] + │ │ │ ├── adrelid = '"numbers"'::REGCLASS [type=bool, outer=(23), constraints=(/23: (/NULL - ])] + │ │ │ └── adnum > 0 [type=bool, outer=(24), constraints=(/24: [/1 - ]; tight)] │ │ └── true [type=bool] │ └── filters [type=bool, outer=(3,18,29,36), constraints=(/3: (/NULL - ]; /18: (/NULL - ]; /29: (/NULL - ]; /36: (/NULL - ]), fd=(18)==(29), (29)==(18), (3)==(36), (36)==(3)] - │ ├── pg_collation.oid = pg_attribute.attcollation [type=bool, outer=(18,29), constraints=(/18: (/NULL - ]; /29: (/NULL - ])] - │ └── pg_type.oid = pg_attribute.atttypid [type=bool, outer=(3,36), constraints=(/3: (/NULL - ]; /36: (/NULL - ])] + │ ├── pg_collation.oid = attcollation [type=bool, outer=(18,29), constraints=(/18: (/NULL - ]; /29: (/NULL - ])] + │ └── pg_type.oid = atttypid [type=bool, outer=(3,36), constraints=(/3: (/NULL - ]; /36: (/NULL - ])] └── projections [outer=(1-3,6,9,13,23,25,30)] - ├── format_type(pg_attribute.atttypid, pg_attribute.atttypmod) [type=string, outer=(3,9)] - ├── pg_get_expr(pg_attrdef.adbin, pg_attrdef.adrelid) [type=string, outer=(23,25)] + ├── format_type(atttypid, atttypmod) [type=string, outer=(3,9)] + ├── pg_get_expr(adbin, adrelid) [type=string, outer=(23,25)] ├── variable: pg_collation.collname [type=string, outer=(30)] - └── col_description(pg_attribute.attrelid, pg_attribute.attnum) [type=string, outer=(1,6)] + └── col_description(attrelid, attnum) [type=string, outer=(1,6)] diff --git a/pkg/sql/opt/xform/testdata/external/hibernate b/pkg/sql/opt/xform/testdata/external/hibernate index c97acc8e13b6..832d3deb207a 100644 --- a/pkg/sql/opt/xform/testdata/external/hibernate +++ b/pkg/sql/opt/xform/testdata/external/hibernate @@ -92,9 +92,9 @@ project │ │ │ ├── key: (2) │ │ │ └── fd: ()-->(1) │ │ └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] - │ │ └── phone_register.person_id = phone.id [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + │ │ └── person_id = phone.id [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ └── phone.id = phone_register.person_id [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ └── phone.id = person_id [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] └── projections [outer=(1-6,9)] └── variable: phone.since [type=timestamp, outer=(9)] @@ -255,7 +255,7 @@ inner-join │ │ │ ├── key: (15) │ │ │ └── fd: (15)-->(18) │ │ └── filters [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ]), fd=(1)==(18), (18)==(1)] - │ │ └── phone.id = phone_call.phone_id [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ])] + │ │ └── phone.id = phone_id [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ])] │ ├── scan person │ │ ├── columns: person.id:6(int!null) address:7(string) createdon:8(timestamp) name:9(string) nickname:10(string) version:11(int!null) │ │ ├── key: (6) @@ -316,7 +316,7 @@ project │ │ │ ├── key: (15) │ │ │ └── fd: (15)-->(18) │ │ └── filters [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ]), fd=(1)==(18), (18)==(1)] - │ │ └── phone.id = phone_call.phone_id [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ])] + │ │ └── phone.id = phone_id [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ])] │ ├── scan person │ │ ├── columns: person.id:6(int!null) │ │ └── key: (6) @@ -376,13 +376,13 @@ inner-join │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-6) │ │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ │ └── person.address = $1 [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ │ └── address = $1 [type=bool, outer=(2), constraints=(/2: (/NULL - ])] │ ├── scan phone │ │ ├── columns: phone.id:10(int!null) person_id:13(int) │ │ ├── key: (10) │ │ └── fd: (10)-->(13) │ └── filters [type=bool, outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] - │ └── person.id = phone.person_id [type=bool, outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ])] + │ └── person.id = person_id [type=bool, outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ])] ├── select │ ├── columns: partner.id:7(int!null) partner.name:8(string!null) partner.version:9(int!null) │ ├── key: (7) @@ -800,18 +800,18 @@ distinct-on │ │ ├── key: (10) │ │ └── fd: (10)-->(11-15) │ └── filters [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] - │ └── phone.person_id = person.id [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ])] + │ └── person_id = person.id [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ])] └── aggregations [outer=(11-15)] ├── const-agg [type=string, outer=(11)] - │ └── variable: person.address [type=string, outer=(11)] + │ └── variable: address [type=string, outer=(11)] ├── const-agg [type=timestamp, outer=(12)] - │ └── variable: person.createdon [type=timestamp, outer=(12)] + │ └── variable: createdon [type=timestamp, outer=(12)] ├── const-agg [type=string, outer=(13)] - │ └── variable: person.name [type=string, outer=(13)] + │ └── variable: name [type=string, outer=(13)] ├── const-agg [type=string, outer=(14)] - │ └── variable: person.nickname [type=string, outer=(14)] + │ └── variable: nickname [type=string, outer=(14)] └── const-agg [type=int, outer=(15)] - └── variable: person.version [type=int, outer=(15)] + └── variable: version [type=int, outer=(15)] opt select @@ -857,16 +857,16 @@ project │ │ │ ├── key: (6) │ │ │ └── fd: (6)-->(9) │ │ └── filters [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] - │ │ └── phone.id = phone_call.phone_id [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] + │ │ └── phone.id = phone_id [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] │ └── aggregations [outer=(2-4,6)] │ ├── max [type=int, outer=(6)] │ │ └── variable: phone_call.id [type=int, outer=(6)] │ ├── const-agg [type=string, outer=(2)] - │ │ └── variable: phone.phone_number [type=string, outer=(2)] + │ │ └── variable: phone_number [type=string, outer=(2)] │ ├── const-agg [type=string, outer=(3)] - │ │ └── variable: phone.phone_type [type=string, outer=(3)] + │ │ └── variable: phone_type [type=string, outer=(3)] │ └── const-agg [type=int, outer=(4)] - │ └── variable: phone.person_id [type=int, outer=(4)] + │ └── variable: person_id [type=int, outer=(4)] └── filters [type=bool, outer=(10), constraints=(/10: (/NULL - ])] └── max = $1 [type=bool, outer=(10), constraints=(/10: (/NULL - ])] @@ -913,20 +913,20 @@ project │ │ ├── scan phone │ │ │ └── columns: person_id:10(int) │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ └── person.id = phone.person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ └── person.id = person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ └── aggregations [outer=(2-6,10)] │ ├── count [type=int, outer=(10)] - │ │ └── variable: phone.person_id [type=int, outer=(10)] + │ │ └── variable: person_id [type=int, outer=(10)] │ ├── const-agg [type=string, outer=(2)] - │ │ └── variable: person.address [type=string, outer=(2)] + │ │ └── variable: address [type=string, outer=(2)] │ ├── const-agg [type=timestamp, outer=(3)] - │ │ └── variable: person.createdon [type=timestamp, outer=(3)] + │ │ └── variable: createdon [type=timestamp, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: person.name [type=string, outer=(4)] + │ │ └── variable: name [type=string, outer=(4)] │ ├── const-agg [type=string, outer=(5)] - │ │ └── variable: person.nickname [type=string, outer=(5)] + │ │ └── variable: nickname [type=string, outer=(5)] │ └── const-agg [type=int, outer=(6)] - │ └── variable: person.version [type=int, outer=(6)] + │ └── variable: version [type=int, outer=(6)] └── filters [type=bool, outer=(12), constraints=(/12: [/2 - /2]; tight), fd=()-->(12)] └── count = 2 [type=bool, outer=(12), constraints=(/12: [/2 - /2]; tight)] @@ -974,16 +974,16 @@ project │ │ │ ├── key: (6) │ │ │ └── fd: (6)-->(9) │ │ └── filters [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] - │ │ └── phone.id = phone_call.phone_id [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] + │ │ └── phone.id = phone_id [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] │ └── aggregations [outer=(2-4,6)] │ ├── min [type=int, outer=(6)] │ │ └── variable: phone_call.id [type=int, outer=(6)] │ ├── const-agg [type=string, outer=(2)] - │ │ └── variable: phone.phone_number [type=string, outer=(2)] + │ │ └── variable: phone_number [type=string, outer=(2)] │ ├── const-agg [type=string, outer=(3)] - │ │ └── variable: phone.phone_type [type=string, outer=(3)] + │ │ └── variable: phone_type [type=string, outer=(3)] │ └── const-agg [type=int, outer=(4)] - │ └── variable: phone.person_id [type=int, outer=(4)] + │ └── variable: person_id [type=int, outer=(4)] └── filters [type=bool, outer=(10), constraints=(/10: (/NULL - ])] └── min = $1 [type=bool, outer=(10), constraints=(/10: (/NULL - ])] @@ -1032,22 +1032,22 @@ project │ │ │ ├── scan phone │ │ │ │ └── columns: person_id:10(int) order_id:11(int) │ │ │ └── filters [type=bool, outer=(11), constraints=(/11: (/NULL - ]; tight)] - │ │ │ └── phone.order_id IS NOT NULL [type=bool, outer=(11), constraints=(/11: (/NULL - ]; tight)] + │ │ │ └── order_id IS NOT NULL [type=bool, outer=(11), constraints=(/11: (/NULL - ]; tight)] │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ └── person.id = phone.person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ └── person.id = person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ └── aggregations [outer=(2-6,11)] │ ├── max [type=int, outer=(11)] - │ │ └── variable: phone.order_id [type=int, outer=(11)] + │ │ └── variable: order_id [type=int, outer=(11)] │ ├── const-agg [type=string, outer=(2)] - │ │ └── variable: person.address [type=string, outer=(2)] + │ │ └── variable: address [type=string, outer=(2)] │ ├── const-agg [type=timestamp, outer=(3)] - │ │ └── variable: person.createdon [type=timestamp, outer=(3)] + │ │ └── variable: createdon [type=timestamp, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: person.name [type=string, outer=(4)] + │ │ └── variable: name [type=string, outer=(4)] │ ├── const-agg [type=string, outer=(5)] - │ │ └── variable: person.nickname [type=string, outer=(5)] + │ │ └── variable: nickname [type=string, outer=(5)] │ └── const-agg [type=int, outer=(6)] - │ └── variable: person.version [type=int, outer=(6)] + │ └── variable: version [type=int, outer=(6)] └── filters [type=bool, outer=(12), constraints=(/12: [/0 - /0]; tight), fd=()-->(12)] └── max = 0 [type=bool, outer=(12), constraints=(/12: [/0 - /0]; tight)] @@ -1090,7 +1090,7 @@ semi-join │ └── filters [type=bool, outer=(7), constraints=(/7: (/NULL - ])] │ └── phone.id = $1::INT [type=bool, outer=(7), constraints=(/7: (/NULL - ])] └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - └── person.id = phone.person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + └── person.id = person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] opt select @@ -1131,7 +1131,7 @@ semi-join │ └── filters [type=bool, outer=(7), constraints=(/7: (/NULL - ])] │ └── phone.id = $1::INT [type=bool, outer=(7), constraints=(/7: (/NULL - ])] └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - └── person.id = phone.person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + └── person.id = person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] opt select @@ -1166,7 +1166,7 @@ semi-join │ ├── key: (7) │ └── fd: (7)-->(10) └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - └── person.id = phone.person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + └── person.id = person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] opt select @@ -1199,9 +1199,9 @@ anti-join │ ├── scan phone_repairtimestamps │ │ └── columns: phone_id:6(int!null) repairtimestamps:7(timestamp) │ └── filters [type=bool, outer=(7)] - │ └── (phone_repairtimestamps.repairtimestamps >= $1::DATE) IS NOT false [type=bool, outer=(7)] + │ └── (repairtimestamps >= $1::DATE) IS NOT false [type=bool, outer=(7)] └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── phone.id = phone_repairtimestamps.phone_id [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── id = phone_id [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] opt select @@ -1242,12 +1242,12 @@ semi-join (merge) │ ├── scan phone │ │ └── columns: person_id:10(int) order_id:11(int) │ └── filters [type=bool, outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] - │ └── phone.order_id = 1 [type=bool, outer=(11), constraints=(/11: [/1 - /1]; tight)] + │ └── order_id = 1 [type=bool, outer=(11), constraints=(/11: [/1 - /1]; tight)] └── merge-on ├── left ordering: +1 ├── right ordering: +10 └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - └── person.id = phone.person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + └── person.id = person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] opt select @@ -1319,15 +1319,15 @@ project │ ├── const-agg [type=int, outer=(11)] │ │ └── variable: phone.order_id [type=int, outer=(11)] │ ├── const-agg [type=string, outer=(2)] - │ │ └── variable: person.address [type=string, outer=(2)] + │ │ └── variable: address [type=string, outer=(2)] │ ├── const-agg [type=timestamp, outer=(3)] - │ │ └── variable: person.createdon [type=timestamp, outer=(3)] + │ │ └── variable: createdon [type=timestamp, outer=(3)] │ ├── const-agg [type=string, outer=(4)] - │ │ └── variable: person.name [type=string, outer=(4)] + │ │ └── variable: name [type=string, outer=(4)] │ ├── const-agg [type=string, outer=(5)] - │ │ └── variable: person.nickname [type=string, outer=(5)] + │ │ └── variable: nickname [type=string, outer=(5)] │ ├── const-agg [type=int, outer=(6)] - │ │ └── variable: person.version [type=int, outer=(6)] + │ │ └── variable: version [type=int, outer=(6)] │ └── const-agg [type=int, outer=(1)] │ └── variable: person.id [type=int, outer=(1)] └── filters [type=bool, outer=(11,17), constraints=(/11: (/NULL - ]; /17: (/NULL - ]), fd=(11)==(17), (17)==(11)] @@ -1364,7 +1364,7 @@ anti-join │ ├── key: (7) │ └── fd: (7)-->(10) └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - └── person.id = phone.person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + └── person.id = person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] opt select @@ -1399,7 +1399,7 @@ semi-join │ ├── key: (7) │ └── fd: (7)-->(10) └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - └── person.id = phone.person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + └── person.id = person_id [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] opt select @@ -1430,7 +1430,7 @@ anti-join │ ├── key: (6) │ └── fd: (6)-->(9) └── filters [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] - └── phone.id = phone_call.phone_id [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] + └── phone.id = phone_id [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] opt select @@ -1469,12 +1469,12 @@ semi-join (merge) │ │ ├── columns: person_id:7(int!null) addresses:8(string) │ │ └── ordering: +7 opt(8) │ └── filters [type=bool, outer=(8), constraints=(/8: [/'Home address' - /'Home address']; tight), fd=()-->(8)] - │ └── person_addresses.addresses = 'Home address' [type=bool, outer=(8), constraints=(/8: [/'Home address' - /'Home address']; tight)] + │ └── addresses = 'Home address' [type=bool, outer=(8), constraints=(/8: [/'Home address' - /'Home address']; tight)] └── merge-on ├── left ordering: +1 ├── right ordering: +7 └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── person.id = person_addresses.person_id [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + └── id = person_id [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] opt select @@ -1512,12 +1512,12 @@ anti-join (merge) │ │ ├── columns: person_id:7(int!null) addresses:8(string) │ │ └── ordering: +7 │ └── filters [type=bool, outer=(8)] - │ └── (person_addresses.addresses = 'Home address') IS NOT false [type=bool, outer=(8)] + │ └── (addresses = 'Home address') IS NOT false [type=bool, outer=(8)] └── merge-on ├── left ordering: +1 ├── right ordering: +7 └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── person.id = person_addresses.person_id [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + └── id = person_id [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] exec-ddl drop table Phone, phone_call, Person, Phone_repairTimestamps, Person_addresses; @@ -1594,14 +1594,14 @@ project │ │ ├── scan employee_phones │ │ │ └── columns: employee_id:4(int!null) │ │ └── filters [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ └── employee.id = employee_phones.employee_id [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] + │ │ └── id = employee_id [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] │ └── aggregations [outer=(2-4)] │ ├── count [type=int, outer=(4)] - │ │ └── variable: employee_phones.employee_id [type=int, outer=(4)] + │ │ └── variable: employee_id [type=int, outer=(4)] │ ├── const-agg [type=string, outer=(2)] - │ │ └── variable: employee.email [type=string, outer=(2)] + │ │ └── variable: email [type=string, outer=(2)] │ └── const-agg [type=int, outer=(3)] - │ └── variable: employee.currentproject_id [type=int, outer=(3)] + │ └── variable: currentproject_id [type=int, outer=(3)] └── filters [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] └── count = 1 [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight)] @@ -1749,15 +1749,15 @@ left-join │ │ │ ├── columns: company_id:6(int!null) employees_id:7(int!null) │ │ │ └── key: (6,7) │ │ └── filters [type=bool, outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ]), fd=(7)==(12), (12)==(7)] - │ │ └── company_employee.employees_id = id [type=bool, outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ])] + │ │ └── employees_id = id [type=bool, outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ])] │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ └── company.id = company_employee.company_id [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── company.id = company_id [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ├── scan location │ ├── columns: location.id:3(int!null) address:4(string) zip:5(int!null) │ ├── key: (3) │ └── fd: (3)-->(4,5) └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] - └── company.location_id = location.id [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + └── location_id = location.id [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] exec-ddl drop table Company, Company_Employee, Employee, Manager, Location; @@ -1850,9 +1850,9 @@ project │ │ │ ├── key: (2) │ │ │ └── fd: ()-->(1) │ │ └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] - │ │ └── newspaper_news.news_news_id = news.news_id [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + │ │ └── news_news_id = news.news_id [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] │ └── filters [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] - │ └── news.news_id = newspaper_news.news_news_id [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + │ └── news.news_id = news_news_id [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] └── projections [outer=(1-5,8)] └── variable: news.title [type=string, outer=(8)] @@ -1960,13 +1960,13 @@ project │ │ │ │ │ ├── key: (2) │ │ │ │ │ └── fd: ()-->(1) │ │ │ │ └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] - │ │ │ │ └── generationuser_generationgroup.ref_id = generationgroup.id [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + │ │ │ │ └── ref_id = generationgroup.id [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ │ └── generationgroup.id = generationuser_generationgroup.ref_id [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + │ │ │ └── generationgroup.id = ref_id [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] │ │ └── filters [type=bool, outer=(2,12), constraints=(/2: (/NULL - ]; /12: (/NULL - ]), fd=(2)==(12), (12)==(2)] - │ │ └── generationgroup.id = generationuser_generationgroup.ref_id [type=bool, outer=(2,12), constraints=(/2: (/NULL - ]; /12: (/NULL - ])] + │ │ └── generationgroup.id = ref_id [type=bool, outer=(2,12), constraints=(/2: (/NULL - ]; /12: (/NULL - ])] │ └── filters [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] - │ └── generationgroup.id = generationuser_generationgroup.ref_id [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ])] + │ └── generationgroup.id = ref_id [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ])] └── projections [outer=(1-6,8,14,20)] ├── variable: generationgroup.age [type=string, outer=(8)] ├── variable: generationgroup.culture [type=string, outer=(14)] @@ -2061,18 +2061,18 @@ project │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2-4) │ │ │ └── filters [type=bool, outer=(4), constraints=(/4: (/NULL - ])] - │ │ │ └── tbid2.auctionid = $1 [type=bool, outer=(4), constraints=(/4: (/NULL - ])] + │ │ │ └── auctionid = $1 [type=bool, outer=(4), constraints=(/4: (/NULL - ])] │ │ └── filters [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] - │ │ └── tauction2.successfulbid = tbid2.id [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] + │ │ └── successfulbid = tbid2.id [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ])] │ └── aggregations [outer=(2-4,10)] │ ├── const-not-null-agg [type=bool, outer=(10)] │ │ └── variable: true [type=bool, outer=(10)] │ ├── const-agg [type=decimal, outer=(2)] - │ │ └── variable: tbid2.amount [type=decimal, outer=(2)] + │ │ └── variable: amount [type=decimal, outer=(2)] │ ├── const-agg [type=timestamp, outer=(3)] - │ │ └── variable: tbid2.createddatetime [type=timestamp, outer=(3)] + │ │ └── variable: createddatetime [type=timestamp, outer=(3)] │ └── const-agg [type=int, outer=(4)] - │ └── variable: tbid2.auctionid [type=int, outer=(4)] + │ └── variable: auctionid [type=int, outer=(4)] └── projections [outer=(1-4,11)] └── true_agg IS NOT NULL [type=bool, outer=(11)] @@ -2212,7 +2212,7 @@ project │ │ │ │ └── filters [type=bool, outer=(10,12), constraints=(/10: (/NULL - ]; /12: (/NULL - ]), fd=(10)==(12), (12)==(10)] │ │ │ │ └── lineitem.productid = product.productid [type=bool, outer=(10,12), constraints=(/10: (/NULL - ]; /12: (/NULL - ])] │ │ │ └── projections [outer=(8,9,11,14)] - │ │ │ └── lineitem.quantity * product.cost [type=decimal, outer=(11,14)] + │ │ │ └── lineitem.quantity * cost [type=decimal, outer=(11,14)] │ │ ├── left-join (merge) │ │ │ ├── columns: customerorder.customerid:1(string!null) customerorder.ordernumber:2(int!null) orderdate:3(date!null) lineitem.customerid:4(string) lineitem.ordernumber:5(int) lineitem.productid:6(string) lineitem.quantity:7(int) │ │ │ ├── key: (6) @@ -2245,7 +2245,7 @@ project │ ├── const-agg [type=int, outer=(2)] │ │ └── variable: customerorder.ordernumber [type=int, outer=(2)] │ ├── const-agg [type=date, outer=(3)] - │ │ └── variable: customerorder.orderdate [type=date, outer=(3)] + │ │ └── variable: orderdate [type=date, outer=(3)] │ ├── const-agg [type=string, outer=(4)] │ │ └── variable: lineitem.customerid [type=string, outer=(4)] │ ├── const-agg [type=int, outer=(5)] @@ -2392,11 +2392,11 @@ project │ │ │ ├── sum [type=decimal, outer=(23)] │ │ │ │ └── variable: column23 [type=decimal, outer=(23)] │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ └── variable: customer.name [type=string, outer=(2)] + │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ ├── const-agg [type=string, outer=(3)] - │ │ │ │ └── variable: customer.address [type=string, outer=(3)] + │ │ │ │ └── variable: address [type=string, outer=(3)] │ │ │ ├── const-agg [type=date, outer=(6)] - │ │ │ │ └── variable: customerorder.orderdate [type=date, outer=(6)] + │ │ │ │ └── variable: orderdate [type=date, outer=(6)] │ │ │ ├── const-agg [type=int, outer=(10)] │ │ │ │ └── variable: lineitem.quantity [type=int, outer=(10)] │ │ │ ├── const-agg [type=string, outer=(12)] @@ -2413,11 +2413,11 @@ project │ ├── sum [type=decimal, outer=(29)] │ │ └── variable: lineitem.quantity [type=int, outer=(29)] │ ├── const-agg [type=string, outer=(2)] - │ │ └── variable: customer.name [type=string, outer=(2)] + │ │ └── variable: name [type=string, outer=(2)] │ ├── const-agg [type=string, outer=(3)] - │ │ └── variable: customer.address [type=string, outer=(3)] + │ │ └── variable: address [type=string, outer=(3)] │ ├── const-agg [type=date, outer=(6)] - │ │ └── variable: customerorder.orderdate [type=date, outer=(6)] + │ │ └── variable: orderdate [type=date, outer=(6)] │ ├── const-agg [type=int, outer=(10)] │ │ └── variable: lineitem.quantity [type=int, outer=(10)] │ ├── const-agg [type=string, outer=(12)] @@ -2493,7 +2493,7 @@ project │ │ │ │ └── filters [type=bool, outer=(10,12), constraints=(/10: (/NULL - ]; /12: (/NULL - ]), fd=(10)==(12), (12)==(10)] │ │ │ │ └── lineitem.productid = product.productid [type=bool, outer=(10,12), constraints=(/10: (/NULL - ]; /12: (/NULL - ])] │ │ │ └── projections [outer=(8,9,11,14)] - │ │ │ └── lineitem.quantity * product.cost [type=decimal, outer=(11,14)] + │ │ │ └── lineitem.quantity * cost [type=decimal, outer=(11,14)] │ │ ├── left-join (merge) │ │ │ ├── columns: customerorder.customerid:1(string!null) customerorder.ordernumber:2(int!null) orderdate:3(date!null) lineitem.customerid:4(string) lineitem.ordernumber:5(int) lineitem.productid:6(string) lineitem.quantity:7(int) │ │ │ ├── key: (6) @@ -2526,7 +2526,7 @@ project │ ├── const-agg [type=int, outer=(2)] │ │ └── variable: customerorder.ordernumber [type=int, outer=(2)] │ ├── const-agg [type=date, outer=(3)] - │ │ └── variable: customerorder.orderdate [type=date, outer=(3)] + │ │ └── variable: orderdate [type=date, outer=(3)] │ ├── const-agg [type=string, outer=(4)] │ │ └── variable: lineitem.customerid [type=string, outer=(4)] │ ├── const-agg [type=int, outer=(5)] @@ -2588,7 +2588,7 @@ project │ │ │ │ └── filters [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] │ │ │ │ └── lineitem.productid = product.productid [type=bool, outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ])] │ │ │ └── projections [outer=(4,5,7,10)] - │ │ │ └── lineitem.quantity * product.cost [type=decimal, outer=(7,10)] + │ │ │ └── quantity * cost [type=decimal, outer=(7,10)] │ │ └── filters [type=bool, outer=(1,2,4,5), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /4: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(4), (4)==(1), (2)==(5), (5)==(2)] │ │ ├── lineitem.customerid = customerorder.customerid [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ])] │ │ └── lineitem.ordernumber = customerorder.ordernumber [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] @@ -2596,7 +2596,7 @@ project │ ├── sum [type=decimal, outer=(12)] │ │ └── variable: column12 [type=decimal, outer=(12)] │ └── const-agg [type=date, outer=(3)] - │ └── variable: customerorder.orderdate [type=date, outer=(3)] + │ └── variable: orderdate [type=date, outer=(3)] └── projections [outer=(1-3,13)] └── variable: sum [type=decimal, outer=(13)] @@ -2704,31 +2704,31 @@ group-by │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2-5) │ │ │ └── filters [type=bool, outer=(5,11), constraints=(/5: (/NULL - ]; /11: (/NULL - ]), fd=(5)==(11), (11)==(5)] - │ │ │ └── student.preferredcoursecode = enrolment.coursecode [type=bool, outer=(5,11), constraints=(/5: (/NULL - ]; /11: (/NULL - ])] + │ │ │ └── preferredcoursecode = enrolment.coursecode [type=bool, outer=(5,11), constraints=(/5: (/NULL - ]; /11: (/NULL - ])] │ │ └── aggregations [outer=(2-5,9,13)] │ │ ├── max [type=int, outer=(13)] │ │ │ └── variable: enrolment.year [type=int, outer=(13)] │ │ ├── const-agg [type=int, outer=(9)] │ │ │ └── variable: enrolment.year [type=int, outer=(9)] │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ └── variable: student.name [type=string, outer=(2)] + │ │ │ └── variable: name [type=string, outer=(2)] │ │ ├── const-agg [type=string, outer=(3)] - │ │ │ └── variable: student.address_city [type=string, outer=(3)] + │ │ │ └── variable: address_city [type=string, outer=(3)] │ │ ├── const-agg [type=string, outer=(4)] - │ │ │ └── variable: student.address_state [type=string, outer=(4)] + │ │ │ └── variable: address_state [type=string, outer=(4)] │ │ └── const-agg [type=string, outer=(5)] - │ │ └── variable: student.preferredcoursecode [type=string, outer=(5)] + │ │ └── variable: preferredcoursecode [type=string, outer=(5)] │ └── filters [type=bool, outer=(9,14), constraints=(/9: (/NULL - ]; /14: (/NULL - ]), fd=(9)==(14), (14)==(9)] │ └── enrolment.year = y0_ [type=bool, outer=(9,14), constraints=(/9: (/NULL - ]; /14: (/NULL - ])] └── aggregations [outer=(2-5)] ├── const-agg [type=string, outer=(2)] - │ └── variable: student.name [type=string, outer=(2)] + │ └── variable: name [type=string, outer=(2)] ├── const-agg [type=string, outer=(3)] - │ └── variable: student.address_city [type=string, outer=(3)] + │ └── variable: address_city [type=string, outer=(3)] ├── const-agg [type=string, outer=(4)] - │ └── variable: student.address_state [type=string, outer=(4)] + │ └── variable: address_state [type=string, outer=(4)] └── const-agg [type=string, outer=(5)] - └── variable: student.preferredcoursecode [type=string, outer=(5)] + └── variable: preferredcoursecode [type=string, outer=(5)] exec-ddl drop table student, enrolment diff --git a/pkg/sql/opt/xform/testdata/external/liquibase b/pkg/sql/opt/xform/testdata/external/liquibase index 12f0eb2a20f6..775299212c7d 100644 --- a/pkg/sql/opt/xform/testdata/external/liquibase +++ b/pkg/sql/opt/xform/testdata/external/liquibase @@ -316,7 +316,7 @@ project │ │ │ │ │ │ │ │ ├── key: (72) │ │ │ │ │ │ │ │ └── fd: (72)-->(73,79) │ │ │ │ │ │ │ └── filters [type=bool, outer=(79), constraints=(/79: [/true - /true]; tight), fd=()-->(79)] - │ │ │ │ │ │ │ └── pg_index.indisclustered = true [type=bool, outer=(79), constraints=(/79: [/true - /true]; tight)] + │ │ │ │ │ │ │ └── indisclustered = true [type=bool, outer=(79), constraints=(/79: [/true - /true]; tight)] │ │ │ │ │ │ ├── right-join │ │ │ │ │ │ │ ├── columns: pg_class.oid:1(oid!null) pg_class.relname:2(string!null) pg_class.relnamespace:3(oid!null) pg_class.relowner:5(oid!null) pg_class.reltablespace:8(oid!null) pg_class.reltuples:10(float!null) pg_class.relhasindex:13(bool!null) pg_class.relpersistence:15(string!null) pg_class.relkind:17(string!null) pg_class.relhasoids:20(bool!null) pg_class.relhasrules:22(bool!null) pg_class.relhastriggers:23(bool!null) pg_class.relacl:26(string[]) pg_class.reloptions:27(string[]) pg_namespace.oid:28(oid!null) pg_namespace.nspname:29(string!null) pg_tablespace.oid:32(oid) spcname:33(string) pg_inherits.inhrelid:38(oid) pg_inherits.inhparent:39(oid) pg_class.oid:41(oid) pg_class.relname:42(string) pg_class.relnamespace:43(oid) pg_namespace.oid:68(oid) pg_namespace.nspname:69(string) │ │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68) @@ -372,13 +372,13 @@ project │ │ │ │ │ │ │ └── filters [type=bool, outer=(1,38), constraints=(/1: (/NULL - ]; /38: (/NULL - ]), fd=(1)==(38), (38)==(1)] │ │ │ │ │ │ │ └── pg_inherits.inhrelid = pg_class.oid [type=bool, outer=(1,38), constraints=(/1: (/NULL - ]; /38: (/NULL - ])] │ │ │ │ │ │ └── filters [type=bool, outer=(1,73), constraints=(/1: (/NULL - ]; /73: (/NULL - ]), fd=(1)==(73), (73)==(1)] - │ │ │ │ │ │ └── pg_index.indrelid = pg_class.oid [type=bool, outer=(1,73), constraints=(/1: (/NULL - ]; /73: (/NULL - ])] + │ │ │ │ │ │ └── indrelid = pg_class.oid [type=bool, outer=(1,73), constraints=(/1: (/NULL - ]; /73: (/NULL - ])] │ │ │ │ │ └── filters [type=bool, outer=(72,91), constraints=(/72: (/NULL - ]; /91: (/NULL - ]), fd=(72)==(91), (91)==(72)] - │ │ │ │ │ └── pg_class.oid = pg_index.indexrelid [type=bool, outer=(72,91), constraints=(/72: (/NULL - ]; /91: (/NULL - ])] + │ │ │ │ │ └── pg_class.oid = indexrelid [type=bool, outer=(72,91), constraints=(/72: (/NULL - ]; /91: (/NULL - ])] │ │ │ │ └── filters [type=bool, outer=(1,118), constraints=(/1: (/NULL - ]; /118: (/NULL - ]), fd=(1)==(118), (118)==(1)] - │ │ │ │ └── pg_foreign_table.ftrelid = pg_class.oid [type=bool, outer=(1,118), constraints=(/1: (/NULL - ]; /118: (/NULL - ])] + │ │ │ │ └── ftrelid = pg_class.oid [type=bool, outer=(1,118), constraints=(/1: (/NULL - ]; /118: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(119,121), constraints=(/119: (/NULL - ]; /121: (/NULL - ]), fd=(119)==(121), (121)==(119)] - │ │ │ └── pg_foreign_table.ftserver = pg_foreign_server.oid [type=bool, outer=(119,121), constraints=(/119: (/NULL - ]; /121: (/NULL - ])] + │ │ │ └── ftserver = pg_foreign_server.oid [type=bool, outer=(119,121), constraints=(/119: (/NULL - ]; /121: (/NULL - ])] │ │ └── filters [type=bool, outer=(1,132), constraints=(/1: (/NULL - ]; /132: (/NULL - ]), fd=(1)==(132), (132)==(1)] │ │ └── pg_inherits.inhparent = pg_class.oid [type=bool, outer=(1,132), constraints=(/1: (/NULL - ]; /132: (/NULL - ])] │ └── aggregations [outer=(1,2,5,10,13,15,17,20,22,23,26,27,29,33,42,69,92,120,122,132)] @@ -411,7 +411,7 @@ project │ ├── const-agg [type=string, outer=(29)] │ │ └── variable: pg_namespace.nspname [type=string, outer=(29)] │ ├── const-agg [type=string, outer=(33)] - │ │ └── variable: pg_tablespace.spcname [type=string, outer=(33)] + │ │ └── variable: spcname [type=string, outer=(33)] │ ├── const-agg [type=string, outer=(42)] │ │ └── variable: pg_class.relname [type=string, outer=(42)] │ ├── const-agg [type=string, outer=(69)] @@ -419,9 +419,9 @@ project │ ├── const-agg [type=string, outer=(92)] │ │ └── variable: pg_class.relname [type=string, outer=(92)] │ ├── const-agg [type=string[], outer=(120)] - │ │ └── variable: pg_foreign_table.ftoptions [type=string[], outer=(120)] + │ │ └── variable: ftoptions [type=string[], outer=(120)] │ └── const-agg [type=string, outer=(122)] - │ └── variable: pg_foreign_server.srvname [type=string, outer=(122)] + │ └── variable: srvname [type=string, outer=(122)] └── projections [outer=(1,2,5,10,13,15,17,20,22,23,26,27,29,33,42,69,92,120,122,134)] ├── pg_get_userbyid(pg_class.relowner) [type=string, outer=(5)] ├── obj_description(pg_class.oid) [type=string, outer=(1)] diff --git a/pkg/sql/opt/xform/testdata/external/navicat b/pkg/sql/opt/xform/testdata/external/navicat index b7b6f4c7c529..a1c97ff03b6e 100644 --- a/pkg/sql/opt/xform/testdata/external/navicat +++ b/pkg/sql/opt/xform/testdata/external/navicat @@ -319,7 +319,7 @@ sort │ │ │ │ │ │ │ │ ├── key: (72) │ │ │ │ │ │ │ │ └── fd: (72)-->(73,79) │ │ │ │ │ │ │ └── filters [type=bool, outer=(79), constraints=(/79: [/true - /true]; tight), fd=()-->(79)] - │ │ │ │ │ │ │ └── pg_index.indisclustered = true [type=bool, outer=(79), constraints=(/79: [/true - /true]; tight)] + │ │ │ │ │ │ │ └── indisclustered = true [type=bool, outer=(79), constraints=(/79: [/true - /true]; tight)] │ │ │ │ │ │ ├── right-join │ │ │ │ │ │ │ ├── columns: pg_class.oid:1(oid!null) pg_class.relname:2(string!null) pg_class.relnamespace:3(oid!null) pg_class.relowner:5(oid!null) pg_class.reltablespace:8(oid!null) pg_class.reltuples:10(float!null) pg_class.relhasindex:13(bool!null) pg_class.relpersistence:15(string!null) pg_class.relkind:17(string!null) pg_class.relhasoids:20(bool!null) pg_class.relhasrules:22(bool!null) pg_class.relhastriggers:23(bool!null) pg_class.relacl:26(string[]) pg_class.reloptions:27(string[]) pg_namespace.oid:28(oid!null) pg_namespace.nspname:29(string!null) pg_tablespace.oid:32(oid) spcname:33(string) pg_inherits.inhrelid:38(oid) pg_inherits.inhparent:39(oid) pg_class.oid:41(oid) pg_class.relname:42(string) pg_class.relnamespace:43(oid) pg_namespace.oid:68(oid) pg_namespace.nspname:69(string) │ │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68) @@ -375,13 +375,13 @@ sort │ │ │ │ │ │ │ └── filters [type=bool, outer=(1,38), constraints=(/1: (/NULL - ]; /38: (/NULL - ]), fd=(1)==(38), (38)==(1)] │ │ │ │ │ │ │ └── pg_inherits.inhrelid = pg_class.oid [type=bool, outer=(1,38), constraints=(/1: (/NULL - ]; /38: (/NULL - ])] │ │ │ │ │ │ └── filters [type=bool, outer=(1,73), constraints=(/1: (/NULL - ]; /73: (/NULL - ]), fd=(1)==(73), (73)==(1)] - │ │ │ │ │ │ └── pg_index.indrelid = pg_class.oid [type=bool, outer=(1,73), constraints=(/1: (/NULL - ]; /73: (/NULL - ])] + │ │ │ │ │ │ └── indrelid = pg_class.oid [type=bool, outer=(1,73), constraints=(/1: (/NULL - ]; /73: (/NULL - ])] │ │ │ │ │ └── filters [type=bool, outer=(72,91), constraints=(/72: (/NULL - ]; /91: (/NULL - ]), fd=(72)==(91), (91)==(72)] - │ │ │ │ │ └── pg_class.oid = pg_index.indexrelid [type=bool, outer=(72,91), constraints=(/72: (/NULL - ]; /91: (/NULL - ])] + │ │ │ │ │ └── pg_class.oid = indexrelid [type=bool, outer=(72,91), constraints=(/72: (/NULL - ]; /91: (/NULL - ])] │ │ │ │ └── filters [type=bool, outer=(1,118), constraints=(/1: (/NULL - ]; /118: (/NULL - ]), fd=(1)==(118), (118)==(1)] - │ │ │ │ └── pg_foreign_table.ftrelid = pg_class.oid [type=bool, outer=(1,118), constraints=(/1: (/NULL - ]; /118: (/NULL - ])] + │ │ │ │ └── ftrelid = pg_class.oid [type=bool, outer=(1,118), constraints=(/1: (/NULL - ]; /118: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(119,121), constraints=(/119: (/NULL - ]; /121: (/NULL - ]), fd=(119)==(121), (121)==(119)] - │ │ │ └── pg_foreign_table.ftserver = pg_foreign_server.oid [type=bool, outer=(119,121), constraints=(/119: (/NULL - ]; /121: (/NULL - ])] + │ │ │ └── ftserver = pg_foreign_server.oid [type=bool, outer=(119,121), constraints=(/119: (/NULL - ]; /121: (/NULL - ])] │ │ └── filters [type=bool, outer=(1,132), constraints=(/1: (/NULL - ]; /132: (/NULL - ]), fd=(1)==(132), (132)==(1)] │ │ └── pg_inherits.inhparent = pg_class.oid [type=bool, outer=(1,132), constraints=(/1: (/NULL - ]; /132: (/NULL - ])] │ └── aggregations [outer=(1,2,5,10,13,15,17,20,22,23,26,27,29,33,42,69,92,120,122,132)] @@ -414,7 +414,7 @@ sort │ ├── const-agg [type=string, outer=(29)] │ │ └── variable: pg_namespace.nspname [type=string, outer=(29)] │ ├── const-agg [type=string, outer=(33)] - │ │ └── variable: pg_tablespace.spcname [type=string, outer=(33)] + │ │ └── variable: spcname [type=string, outer=(33)] │ ├── const-agg [type=string, outer=(42)] │ │ └── variable: pg_class.relname [type=string, outer=(42)] │ ├── const-agg [type=string, outer=(69)] @@ -422,9 +422,9 @@ sort │ ├── const-agg [type=string, outer=(92)] │ │ └── variable: pg_class.relname [type=string, outer=(92)] │ ├── const-agg [type=string[], outer=(120)] - │ │ └── variable: pg_foreign_table.ftoptions [type=string[], outer=(120)] + │ │ └── variable: ftoptions [type=string[], outer=(120)] │ └── const-agg [type=string, outer=(122)] - │ └── variable: pg_foreign_server.srvname [type=string, outer=(122)] + │ └── variable: srvname [type=string, outer=(122)] └── projections [outer=(1,2,5,10,13,15,17,20,22,23,26,27,29,33,42,69,92,120,122,134)] ├── pg_get_userbyid(pg_class.relowner) [type=string, outer=(5)] ├── obj_description(pg_class.oid) [type=string, outer=(1)] diff --git a/pkg/sql/opt/xform/testdata/external/nova b/pkg/sql/opt/xform/testdata/external/nova index 24df2b51631e..3cca08ca55be 100644 --- a/pkg/sql/opt/xform/testdata/external/nova +++ b/pkg/sql/opt/xform/testdata/external/nova @@ -324,7 +324,7 @@ sort │ │ │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ │ └── filters [type=bool, outer=(7), constraints=(/7: (/NULL - ])] - │ │ │ │ │ │ └── flavors.flavorid = $2 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] + │ │ │ │ │ │ └── flavorid = $2 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] │ │ │ │ │ ├── project │ │ │ │ │ │ ├── columns: true:22(bool!null) flavor_projects.flavor_id:17(int!null) │ │ │ │ │ │ ├── fd: ()-->(22) @@ -338,7 +338,7 @@ sort │ │ │ │ │ │ │ │ ├── key: (17,18) │ │ │ │ │ │ │ │ └── ordering: +17 │ │ │ │ │ │ │ └── filters [type=bool, outer=(18), constraints=(/18: (/NULL - ])] - │ │ │ │ │ │ │ └── flavor_projects.project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] │ │ │ │ │ │ └── projections [outer=(17)] │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ └── merge-on @@ -350,33 +350,33 @@ sort │ │ │ │ ├── const-not-null-agg [type=bool, outer=(22)] │ │ │ │ │ └── variable: true [type=bool, outer=(22)] │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ │ │ └── const-agg [type=timestamp, outer=(15)] │ │ │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ │ │ └── filters [type=bool, outer=(12,23)] - │ │ │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] │ │ └── placeholder: $3 [type=int] │ └── placeholder: $4 [type=int] └── filters [type=bool, outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] @@ -523,7 +523,7 @@ left-join (lookup flavor_extra_specs) │ │ │ │ │ │ │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ │ │ │ │ │ │ └── ordering: +1 opt(11) │ │ │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(11), constraints=(/11: [/false - /false]; tight), fd=()-->(11)] - │ │ │ │ │ │ │ │ │ │ └── flavors.disabled = false [type=bool, outer=(11), constraints=(/11: [/false - /false]; tight)] + │ │ │ │ │ │ │ │ │ │ └── disabled = false [type=bool, outer=(11), constraints=(/11: [/false - /false]; tight)] │ │ │ │ │ │ │ │ │ ├── project │ │ │ │ │ │ │ │ │ │ ├── columns: true:28(bool!null) flavor_projects.flavor_id:17(int!null) │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(28) @@ -549,66 +549,66 @@ left-join (lookup flavor_extra_specs) │ │ │ │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(28)] │ │ │ │ │ │ │ │ │ └── variable: true [type=bool, outer=(28)] │ │ │ │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ │ │ │ │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ │ │ │ │ │ │ └── const-agg [type=timestamp, outer=(15)] │ │ │ │ │ │ │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ │ │ │ │ │ │ └── filters [type=bool, outer=(12,29)] - │ │ │ │ │ │ │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,29)] + │ │ │ │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,29)] │ │ │ │ │ │ └── filters [type=bool, outer=(1,23), constraints=(/1: (/NULL - ]; /23: (/NULL - ]), fd=(1)==(23), (23)==(1)] │ │ │ │ │ │ └── flavor_projects.flavor_id = flavors.id [type=bool, outer=(1,23), constraints=(/1: (/NULL - ]; /23: (/NULL - ])] │ │ │ │ │ └── aggregations [outer=(2-12,14,15,31)] │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(31)] │ │ │ │ │ │ └── variable: true [type=bool, outer=(31)] │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ │ │ │ └── const-agg [type=timestamp, outer=(15)] │ │ │ │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ │ │ │ └── filters [type=bool, outer=(12,32)] - │ │ │ │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,32)] + │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,32)] │ │ │ └── placeholder: $3 [type=int] │ │ └── placeholder: $4 [type=int] │ └── filters [type=bool, outer=(1,37), constraints=(/1: (/NULL - ]; /37: (/NULL - ]), fd=(1)==(37), (37)==(1)] @@ -746,7 +746,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(19,20), constraints=(/19: (/NULL - ]; /20: (/NULL - ])] │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $3 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── instance_type_projects.project_id = $4 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ │ └── project_id = $4 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ │ └── projections [outer=(18)] │ │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ │ └── merge-on @@ -758,27 +758,27 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] │ │ │ │ │ │ └── variable: true [type=bool, outer=(25)] │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -788,7 +788,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ │ │ │ └── filters [type=bool, outer=(12,26)] - │ │ │ │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] │ │ │ └── placeholder: $5 [type=int] │ │ └── placeholder: $6 [type=int] │ └── filters [type=bool, outer=(1,31,32), constraints=(/1: (/NULL - ]; /31: (/NULL - ]; /32: (/NULL - ]), fd=(1)==(31), (31)==(1)] @@ -893,7 +893,7 @@ sort │ │ │ │ │ │ └── ordering: +26 │ │ │ │ │ └── filters [type=bool, outer=(27,28), constraints=(/27: (/NULL - ]; /28: (/NULL - ])] │ │ │ │ │ ├── instance_type_projects.deleted = $3 [type=bool, outer=(28), constraints=(/28: (/NULL - ])] - │ │ │ │ │ └── instance_type_projects.project_id = $4 [type=bool, outer=(27), constraints=(/27: (/NULL - ])] + │ │ │ │ │ └── project_id = $4 [type=bool, outer=(27), constraints=(/27: (/NULL - ])] │ │ │ │ └── projections [outer=(26)] │ │ │ │ └── true [type=bool] │ │ │ └── merge-on @@ -905,27 +905,27 @@ sort │ │ ├── const-not-null-agg [type=bool, outer=(33)] │ │ │ └── variable: true [type=bool, outer=(33)] │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ └── variable: name [type=string, outer=(2)] │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ └── variable: swap [type=int, outer=(8)] │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -935,7 +935,7 @@ sort │ │ └── const-agg [type=timestamp, outer=(16)] │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ └── filters [type=bool, outer=(12,34)] - │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,34)] + │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,34)] └── filters [type=bool, outer=(1,20), constraints=(/1: (/NULL - ]; /20: (/NULL - ]), fd=(1)==(20), (20)==(1)] └── instance_type_extra_specs.instance_type_id = instance_types.id [type=bool, outer=(1,20), constraints=(/1: (/NULL - ]; /20: (/NULL - ])] @@ -1045,7 +1045,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ └── fd: (1)-->(2,13), (2,13)~~>(1) │ │ │ │ │ │ │ └── filters [type=bool, outer=(2,13), constraints=(/2: (/NULL - ]; /13: (/NULL - ])] │ │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ │ └── instance_types.name = $4 [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ │ │ │ │ │ │ └── name = $4 [type=bool, outer=(2), constraints=(/2: (/NULL - ])] │ │ │ │ │ │ ├── project │ │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) │ │ │ │ │ │ │ ├── fd: ()-->(25) @@ -1055,7 +1055,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ │ └── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(19,20), constraints=(/19: (/NULL - ]; /20: (/NULL - ])] │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── instance_type_projects.project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ │ └── projections [outer=(18)] │ │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ │ └── filters [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ]), fd=(1)==(18), (18)==(1)] @@ -1064,27 +1064,27 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] │ │ │ │ │ │ └── variable: true [type=bool, outer=(25)] │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -1094,7 +1094,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ │ │ │ └── filters [type=bool, outer=(12,26)] - │ │ │ │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] │ │ │ └── placeholder: $5 [type=int] │ │ └── placeholder: $6 [type=int] │ └── filters [type=bool, outer=(1,31,32), constraints=(/1: (/NULL - ]; /31: (/NULL - ]; /32: (/NULL - ]), fd=(1)==(31), (31)==(1)] @@ -1218,7 +1218,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(18-20), constraints=(/18: (/NULL - ]; /19: (/NULL - ]; /20: (/NULL - ])] │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ ├── instance_type_projects.project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ │ ├── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ │ │ └── instance_type_projects.instance_type_id = $4 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] │ │ │ │ │ │ │ └── projections [outer=(18)] │ │ │ │ │ │ │ └── true [type=bool] @@ -1231,27 +1231,27 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] │ │ │ │ │ │ └── variable: true [type=bool, outer=(25)] │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -1261,7 +1261,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ │ │ │ └── filters [type=bool, outer=(12,26)] - │ │ │ │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] │ │ │ └── placeholder: $5 [type=int] │ │ └── placeholder: $6 [type=int] │ └── filters [type=bool, outer=(1,31,32), constraints=(/1: (/NULL - ]; /31: (/NULL - ]; /32: (/NULL - ]), fd=(1)==(31), (31)==(1)] @@ -1360,7 +1360,7 @@ right-join │ │ │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ │ └── filters [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ │ │ │ │ │ └── flavors.name = $2 [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ │ │ │ │ │ └── name = $2 [type=bool, outer=(2), constraints=(/2: (/NULL - ])] │ │ │ │ │ ├── project │ │ │ │ │ │ ├── columns: true:22(bool!null) flavor_projects.flavor_id:17(int!null) │ │ │ │ │ │ ├── fd: ()-->(22) @@ -1374,7 +1374,7 @@ right-join │ │ │ │ │ │ │ │ ├── key: (17,18) │ │ │ │ │ │ │ │ └── ordering: +17 │ │ │ │ │ │ │ └── filters [type=bool, outer=(18), constraints=(/18: (/NULL - ])] - │ │ │ │ │ │ │ └── flavor_projects.project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] │ │ │ │ │ │ └── projections [outer=(17)] │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ └── merge-on @@ -1386,33 +1386,33 @@ right-join │ │ │ │ ├── const-not-null-agg [type=bool, outer=(22)] │ │ │ │ │ └── variable: true [type=bool, outer=(22)] │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ │ │ └── const-agg [type=timestamp, outer=(15)] │ │ │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ │ │ └── filters [type=bool, outer=(12,23)] - │ │ │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] │ │ └── placeholder: $3 [type=int] │ └── placeholder: $4 [type=int] └── filters [type=bool, outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] @@ -1509,7 +1509,7 @@ right-join │ │ │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ │ └── filters [type=bool, outer=(7), constraints=(/7: (/NULL - ])] - │ │ │ │ │ │ └── flavors.flavorid = $2 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] + │ │ │ │ │ │ └── flavorid = $2 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] │ │ │ │ │ ├── project │ │ │ │ │ │ ├── columns: true:22(bool!null) flavor_projects.flavor_id:17(int!null) │ │ │ │ │ │ ├── fd: ()-->(22) @@ -1523,7 +1523,7 @@ right-join │ │ │ │ │ │ │ │ ├── key: (17,18) │ │ │ │ │ │ │ │ └── ordering: +17 │ │ │ │ │ │ │ └── filters [type=bool, outer=(18), constraints=(/18: (/NULL - ])] - │ │ │ │ │ │ │ └── flavor_projects.project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] │ │ │ │ │ │ └── projections [outer=(17)] │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ └── merge-on @@ -1535,33 +1535,33 @@ right-join │ │ │ │ ├── const-not-null-agg [type=bool, outer=(22)] │ │ │ │ │ └── variable: true [type=bool, outer=(22)] │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ │ │ └── const-agg [type=timestamp, outer=(15)] │ │ │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ │ │ └── filters [type=bool, outer=(12,23)] - │ │ │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] │ │ └── placeholder: $3 [type=int] │ └── placeholder: $4 [type=int] └── filters [type=bool, outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] @@ -1675,7 +1675,7 @@ sort │ │ │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ │ └── filters [type=bool, outer=(1,7)] - │ │ │ │ │ │ └── (flavors.flavorid > $2) OR ((flavors.flavorid = $3) AND (flavors.id > $4)) [type=bool, outer=(1,7)] + │ │ │ │ │ │ └── (flavorid > $2) OR ((flavorid = $3) AND (flavors.id > $4)) [type=bool, outer=(1,7)] │ │ │ │ │ ├── project │ │ │ │ │ │ ├── columns: true:22(bool!null) flavor_projects.flavor_id:17(int!null) │ │ │ │ │ │ ├── fd: ()-->(22) @@ -1689,7 +1689,7 @@ sort │ │ │ │ │ │ │ │ ├── key: (17,18) │ │ │ │ │ │ │ │ └── ordering: +17 │ │ │ │ │ │ │ └── filters [type=bool, outer=(18), constraints=(/18: (/NULL - ])] - │ │ │ │ │ │ │ └── flavor_projects.project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] │ │ │ │ │ │ └── projections [outer=(17)] │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ └── merge-on @@ -1701,33 +1701,33 @@ sort │ │ │ │ ├── const-not-null-agg [type=bool, outer=(22)] │ │ │ │ │ └── variable: true [type=bool, outer=(22)] │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ │ │ └── const-agg [type=timestamp, outer=(15)] │ │ │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ │ │ └── filters [type=bool, outer=(12,23)] - │ │ │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] │ │ └── placeholder: $5 [type=int] │ └── placeholder: $6 [type=int] └── filters [type=bool, outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] @@ -1862,7 +1862,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(19,20), constraints=(/19: (/NULL - ]; /20: (/NULL - ])] │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── instance_type_projects.project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ │ └── projections [outer=(18)] │ │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ │ └── merge-on @@ -1874,27 +1874,27 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] │ │ │ │ │ │ └── variable: true [type=bool, outer=(25)] │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -1904,7 +1904,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ │ │ │ └── filters [type=bool, outer=(12,26)] - │ │ │ │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] │ │ │ └── placeholder: $4 [type=int] │ │ └── placeholder: $5 [type=int] │ └── filters [type=bool, outer=(1,31,32), constraints=(/1: (/NULL - ]; /31: (/NULL - ]; /32: (/NULL - ]), fd=(1)==(31), (31)==(1)] @@ -2018,7 +2018,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ └── fd: (1)-->(7,13), (7,13)~~>(1) │ │ │ │ │ │ │ └── filters [type=bool, outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ])] │ │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ │ └── instance_types.flavorid = $4 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] + │ │ │ │ │ │ │ └── flavorid = $4 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] │ │ │ │ │ │ ├── project │ │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) │ │ │ │ │ │ │ ├── fd: ()-->(25) @@ -2028,7 +2028,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ │ └── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(19,20), constraints=(/19: (/NULL - ]; /20: (/NULL - ])] │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── instance_type_projects.project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ │ └── projections [outer=(18)] │ │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ │ └── filters [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ]), fd=(1)==(18), (18)==(1)] @@ -2037,27 +2037,27 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] │ │ │ │ │ │ └── variable: true [type=bool, outer=(25)] │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -2067,7 +2067,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ │ │ │ └── filters [type=bool, outer=(12,26)] - │ │ │ │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] │ │ │ └── placeholder: $5 [type=int] │ │ └── placeholder: $6 [type=int] │ └── filters [type=bool, outer=(1,31,32), constraints=(/1: (/NULL - ]; /31: (/NULL - ]; /32: (/NULL - ]), fd=(1)==(31), (31)==(1)] @@ -2153,7 +2153,7 @@ sort │ │ │ │ │ │ ├── key: (23,24) │ │ │ │ │ │ └── ordering: +23 │ │ │ │ │ └── filters [type=bool, outer=(24), constraints=(/24: (/NULL - ])] - │ │ │ │ │ └── flavor_projects.project_id = $1 [type=bool, outer=(24), constraints=(/24: (/NULL - ])] + │ │ │ │ │ └── project_id = $1 [type=bool, outer=(24), constraints=(/24: (/NULL - ])] │ │ │ │ └── projections [outer=(23)] │ │ │ │ └── true [type=bool] │ │ │ └── merge-on @@ -2165,33 +2165,33 @@ sort │ │ ├── const-not-null-agg [type=bool, outer=(28)] │ │ │ └── variable: true [type=bool, outer=(28)] │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ └── variable: name [type=string, outer=(2)] │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ └── variable: swap [type=int, outer=(8)] │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ └── const-agg [type=timestamp, outer=(15)] │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ └── filters [type=bool, outer=(12,29)] - │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,29)] + │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,29)] └── filters [type=bool, outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ]), fd=(1)==(19), (19)==(1)] └── flavor_extra_specs.flavor_id = flavors.id [type=bool, outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ])] @@ -2318,7 +2318,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ └── fd: (1)-->(7,13), (7,13)~~>(1) │ │ │ │ │ │ │ └── filters [type=bool, outer=(1,7,13), constraints=(/13: (/NULL - ])] │ │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ │ └── (instance_types.flavorid > $4) OR ((instance_types.flavorid = $5) AND (instance_types.id > $6)) [type=bool, outer=(1,7)] + │ │ │ │ │ │ │ └── (flavorid > $4) OR ((flavorid = $5) AND (instance_types.id > $6)) [type=bool, outer=(1,7)] │ │ │ │ │ │ ├── project │ │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) │ │ │ │ │ │ │ ├── fd: ()-->(25) @@ -2328,7 +2328,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ │ └── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(19,20), constraints=(/19: (/NULL - ]; /20: (/NULL - ])] │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── instance_type_projects.project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ │ └── projections [outer=(18)] │ │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ │ └── filters [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ]), fd=(1)==(18), (18)==(1)] @@ -2337,27 +2337,27 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] │ │ │ │ │ │ └── variable: true [type=bool, outer=(25)] │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -2367,7 +2367,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ │ │ │ └── filters [type=bool, outer=(12,26)] - │ │ │ │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] │ │ │ └── placeholder: $7 [type=int] │ │ └── placeholder: $8 [type=int] │ └── filters [type=bool, outer=(1,31,32), constraints=(/1: (/NULL - ]; /31: (/NULL - ]; /32: (/NULL - ]), fd=(1)==(31), (31)==(1)] @@ -2487,7 +2487,7 @@ sort │ │ │ │ │ │ │ │ ├── key: (17,18) │ │ │ │ │ │ │ │ └── ordering: +17 │ │ │ │ │ │ │ └── filters [type=bool, outer=(18), constraints=(/18: (/NULL - ])] - │ │ │ │ │ │ │ └── flavor_projects.project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] │ │ │ │ │ │ └── projections [outer=(17)] │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ └── merge-on @@ -2499,33 +2499,33 @@ sort │ │ │ │ ├── const-not-null-agg [type=bool, outer=(22)] │ │ │ │ │ └── variable: true [type=bool, outer=(22)] │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ │ │ └── const-agg [type=timestamp, outer=(15)] │ │ │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ │ │ └── filters [type=bool, outer=(12,23)] - │ │ │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] │ │ └── placeholder: $2 [type=int] │ └── placeholder: $3 [type=int] └── filters [type=bool, outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] @@ -2672,7 +2672,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ │ │ │ └── ordering: +1 opt(11) │ │ │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(11,13), constraints=(/11: [/false - /false]; /13: (/NULL - ]), fd=()-->(11)] │ │ │ │ │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ │ │ │ │ └── instance_types.disabled = false [type=bool, outer=(11), constraints=(/11: [/false - /false]; tight)] + │ │ │ │ │ │ │ │ │ │ └── disabled = false [type=bool, outer=(11), constraints=(/11: [/false - /false]; tight)] │ │ │ │ │ │ │ │ │ ├── project │ │ │ │ │ │ │ │ │ │ ├── columns: true:33(bool!null) instance_type_projects.instance_type_id:18(int!null) │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(33) @@ -2697,27 +2697,27 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(33)] │ │ │ │ │ │ │ │ │ └── variable: true [type=bool, outer=(33)] │ │ │ │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ │ │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -2727,7 +2727,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] │ │ │ │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ │ │ │ │ │ │ └── filters [type=bool, outer=(12,34)] - │ │ │ │ │ │ │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,34)] + │ │ │ │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,34)] │ │ │ │ │ │ ├── project │ │ │ │ │ │ │ ├── columns: true:36(bool!null) instance_type_projects.instance_type_id:26(int!null) │ │ │ │ │ │ │ ├── fd: ()-->(36) @@ -2747,27 +2747,27 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(36)] │ │ │ │ │ │ └── variable: true [type=bool, outer=(36)] │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -2777,7 +2777,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ │ │ │ └── filters [type=bool, outer=(12,37)] - │ │ │ │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,37)] + │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,37)] │ │ │ └── placeholder: $7 [type=int] │ │ └── placeholder: $8 [type=int] │ └── filters [type=bool, outer=(1,42,43), constraints=(/1: (/NULL - ]; /42: (/NULL - ]; /43: (/NULL - ]), fd=(1)==(42), (42)==(1)] @@ -2906,7 +2906,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ └── fd: (1)-->(7,13), (7,13)~~>(1) │ │ │ │ │ │ │ └── filters [type=bool, outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ])] │ │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ │ └── instance_types.flavorid = $4 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] + │ │ │ │ │ │ │ └── flavorid = $4 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] │ │ │ │ │ │ ├── project │ │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) │ │ │ │ │ │ │ ├── fd: ()-->(25) @@ -2916,7 +2916,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ │ │ │ │ └── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(19,20), constraints=(/19: (/NULL - ]; /20: (/NULL - ])] │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── instance_type_projects.project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ │ └── projections [outer=(18)] │ │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ │ └── filters [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ]), fd=(1)==(18), (18)==(1)] @@ -2925,27 +2925,27 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] │ │ │ │ │ │ └── variable: true [type=bool, outer=(25)] │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: instance_types.name [type=string, outer=(2)] + │ │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: instance_types.memory_mb [type=int, outer=(3)] + │ │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: instance_types.vcpus [type=int, outer=(4)] + │ │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: instance_types.root_gb [type=int, outer=(5)] + │ │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: instance_types.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: instance_types.flavorid [type=string, outer=(7)] + │ │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: instance_types.swap [type=int, outer=(8)] + │ │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: instance_types.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: instance_types.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: instance_types.disabled [type=bool, outer=(11)] + │ │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: instance_types.is_public [type=bool, outer=(12)] + │ │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool, outer=(13)] │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] @@ -2955,7 +2955,7 @@ left-join (lookup instance_type_extra_specs) │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp, outer=(16)] │ │ │ │ └── filters [type=bool, outer=(12,26)] - │ │ │ │ └── (instance_types.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] │ │ │ └── placeholder: $5 [type=int] │ │ └── placeholder: $6 [type=int] │ └── filters [type=bool, outer=(1,31,32), constraints=(/1: (/NULL - ]; /31: (/NULL - ]; /32: (/NULL - ]), fd=(1)==(31), (31)==(1)] @@ -3068,7 +3068,7 @@ right-join │ │ │ │ │ │ │ │ ├── key: (17,18) │ │ │ │ │ │ │ │ └── ordering: +17 │ │ │ │ │ │ │ └── filters [type=bool, outer=(17,18), constraints=(/17: (/NULL - ]; /18: (/NULL - ])] - │ │ │ │ │ │ │ ├── flavor_projects.project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ ├── project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] │ │ │ │ │ │ │ └── flavor_projects.flavor_id = $2 [type=bool, outer=(17), constraints=(/17: (/NULL - ])] │ │ │ │ │ │ └── projections [outer=(17)] │ │ │ │ │ │ └── true [type=bool] @@ -3081,33 +3081,33 @@ right-join │ │ │ │ ├── const-not-null-agg [type=bool, outer=(22)] │ │ │ │ │ └── variable: true [type=bool, outer=(22)] │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ │ │ └── const-agg [type=timestamp, outer=(15)] │ │ │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ │ │ └── filters [type=bool, outer=(12,23)] - │ │ │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] │ │ └── placeholder: $3 [type=int] │ └── placeholder: $4 [type=int] └── filters [type=bool, outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] @@ -3220,7 +3220,7 @@ sort │ │ │ │ │ │ │ ├── key: (17,18) │ │ │ │ │ │ │ └── ordering: +17 │ │ │ │ │ │ └── filters [type=bool, outer=(18), constraints=(/18: (/NULL - ])] - │ │ │ │ │ │ └── flavor_projects.project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ └── project_id = $1 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] │ │ │ │ │ └── projections [outer=(17)] │ │ │ │ │ └── true [type=bool] │ │ │ │ └── merge-on @@ -3232,35 +3232,35 @@ sort │ │ │ ├── const-not-null-agg [type=bool, outer=(22)] │ │ │ │ └── variable: true [type=bool, outer=(22)] │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ └── variable: flavors.name [type=string, outer=(2)] + │ │ │ │ └── variable: name [type=string, outer=(2)] │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ └── variable: flavors.memory_mb [type=int, outer=(3)] + │ │ │ │ └── variable: memory_mb [type=int, outer=(3)] │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ └── variable: flavors.vcpus [type=int, outer=(4)] + │ │ │ │ └── variable: vcpus [type=int, outer=(4)] │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ └── variable: flavors.root_gb [type=int, outer=(5)] + │ │ │ │ └── variable: root_gb [type=int, outer=(5)] │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ └── variable: flavors.ephemeral_gb [type=int, outer=(6)] + │ │ │ │ └── variable: ephemeral_gb [type=int, outer=(6)] │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ └── variable: flavors.flavorid [type=string, outer=(7)] + │ │ │ │ └── variable: flavorid [type=string, outer=(7)] │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ └── variable: flavors.swap [type=int, outer=(8)] + │ │ │ │ └── variable: swap [type=int, outer=(8)] │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ └── variable: flavors.rxtx_factor [type=float, outer=(9)] + │ │ │ │ └── variable: rxtx_factor [type=float, outer=(9)] │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ └── variable: flavors.vcpu_weight [type=int, outer=(10)] + │ │ │ │ └── variable: vcpu_weight [type=int, outer=(10)] │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ └── variable: flavors.disabled [type=bool, outer=(11)] + │ │ │ │ └── variable: disabled [type=bool, outer=(11)] │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ └── variable: flavors.is_public [type=bool, outer=(12)] + │ │ │ │ └── variable: is_public [type=bool, outer=(12)] │ │ │ ├── const-agg [type=string, outer=(13)] - │ │ │ │ └── variable: flavors.description [type=string, outer=(13)] + │ │ │ │ └── variable: description [type=string, outer=(13)] │ │ │ ├── const-agg [type=timestamp, outer=(14)] │ │ │ │ └── variable: flavors.created_at [type=timestamp, outer=(14)] │ │ │ └── const-agg [type=timestamp, outer=(15)] │ │ │ └── variable: flavors.updated_at [type=timestamp, outer=(15)] │ │ └── filters [type=bool, outer=(12,23)] - │ │ └── (flavors.is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] + │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,23)] │ └── placeholder: $2 [type=int] └── filters [type=bool, outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] └── flavor_extra_specs.flavor_id = flavors.id [type=bool, outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ])] diff --git a/pkg/sql/opt/xform/testdata/external/pgadmin b/pkg/sql/opt/xform/testdata/external/pgadmin index 46436f28252c..75156f3c2c9b 100644 --- a/pkg/sql/opt/xform/testdata/external/pgadmin +++ b/pkg/sql/opt/xform/testdata/external/pgadmin @@ -146,11 +146,11 @@ project │ │ │ │ ├── scan pg_roles │ │ │ │ │ └── columns: oid:21(oid) rolname:22(name) rolsuper:23(bool) │ │ │ │ └── filters [type=bool, outer=(22), constraints=(/22: (/NULL - ])] - │ │ │ │ └── pg_roles.rolname = current_user() [type=bool, outer=(22), constraints=(/22: (/NULL - ])] + │ │ │ │ └── rolname = current_user() [type=bool, outer=(22), constraints=(/22: (/NULL - ])] │ │ │ └── projections [outer=(4,21,23)] - │ │ │ └── pg_roles.rolsuper OR (pg_roles.oid = pg_stat_activity.usesysid) [type=bool, outer=(4,21,23)] + │ │ │ └── rolsuper OR (oid = usesysid) [type=bool, outer=(4,21,23)] │ │ └── filters [type=bool, outer=(36), constraints=(/36: [/true - /true]; tight), fd=()-->(36)] │ │ └── variable: ?column? [type=bool, outer=(36), constraints=(/36: [/true - /true]; tight)] │ └── true [type=bool] └── projections [outer=(2,3,5-12,19)] - └── CASE WHEN (pg_stat_activity.client_hostname IS NOT NULL) AND (pg_stat_activity.client_hostname != '') THEN (pg_stat_activity.client_hostname || ':') || pg_stat_activity.client_port::STRING WHEN (pg_stat_activity.client_addr IS NOT NULL) AND (pg_stat_activity.client_addr::STRING != '') THEN (pg_stat_activity.client_addr::STRING || ':') || pg_stat_activity.client_port::STRING WHEN pg_stat_activity.client_port = -1 THEN 'local pipe' ELSE 'localhost:' || pg_stat_activity.client_port::STRING END [type=string, outer=(7-9)] + └── CASE WHEN (client_hostname IS NOT NULL) AND (client_hostname != '') THEN (client_hostname || ':') || client_port::STRING WHEN (client_addr IS NOT NULL) AND (client_addr::STRING != '') THEN (client_addr::STRING || ':') || client_port::STRING WHEN client_port = -1 THEN 'local pipe' ELSE 'localhost:' || client_port::STRING END [type=string, outer=(7-9)] diff --git a/pkg/sql/opt/xform/testdata/external/pgjdbc b/pkg/sql/opt/xform/testdata/external/pgjdbc index f6c13bf992ff..26d0a95117a4 100644 --- a/pkg/sql/opt/xform/testdata/external/pgjdbc +++ b/pkg/sql/opt/xform/testdata/external/pgjdbc @@ -139,7 +139,7 @@ project │ │ │ ├── scan pg_namespace │ │ │ │ └── columns: pg_namespace.oid:33(oid) nspname:34(name!null) │ │ │ └── filters [type=bool, outer=(34), constraints=(/34: (/NULL - /'pg_catalog') [/e'pg_catalog\x00' - ]; tight)] - │ │ │ └── pg_namespace.nspname != 'pg_catalog' [type=bool, outer=(34), constraints=(/34: (/NULL - /'pg_catalog') [/e'pg_catalog\x00' - ]; tight)] + │ │ │ └── nspname != 'pg_catalog' [type=bool, outer=(34), constraints=(/34: (/NULL - /'pg_catalog') [/e'pg_catalog\x00' - ]; tight)] │ │ └── filters [type=bool, outer=(3,33), constraints=(/3: (/NULL - ]; /33: (/NULL - ]), fd=(3)==(33), (33)==(3)] │ │ └── pg_type.typnamespace = pg_namespace.oid [type=bool, outer=(3,33), constraints=(/3: (/NULL - ]; /33: (/NULL - ])] │ ├── max1-row diff --git a/pkg/sql/opt/xform/testdata/external/tpcc b/pkg/sql/opt/xform/testdata/external/tpcc index 171427cd38e5..aa3ce88a246f 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc +++ b/pkg/sql/opt/xform/testdata/external/tpcc @@ -539,7 +539,7 @@ ALTER TABLE order_line INJECT STATISTICS '[ # reflect on-line database activity as typically found in production # environments. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT w_tax FROM warehouse WHERE w_id = 10 ---- project @@ -551,7 +551,7 @@ project ├── fd: ()-->(8) ├── prune: (8) └── scan warehouse - ├── columns: warehouse.w_id:1(int!null) warehouse.w_tax:8(decimal) + ├── columns: w_id:1(int!null) w_tax:8(decimal) ├── constraint: /1: [/10 - /10] ├── cardinality: [0 - 1] ├── stats: [rows=1, distinct(1)=1] @@ -561,7 +561,7 @@ project ├── prune: (1,8) └── interesting orderings: (+1) -opt format=show-all +opt format=hide-qual SELECT c_discount, c_last, c_credit FROM customer WHERE c_w_id = 10 AND c_d_id = 100 AND c_id = 50 @@ -575,7 +575,7 @@ project ├── fd: ()-->(6,14,16) ├── prune: (6,14,16) └── scan customer - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_last:6(string) customer.c_credit:14(string) customer.c_discount:16(decimal) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_last:6(string) c_credit:14(string) c_discount:16(decimal) ├── constraint: /3/2/1: [/10/100/50 - /10/100/50] ├── cardinality: [0 - 1] ├── stats: [rows=6.80272109e-07, distinct(1)=6.80272109e-07, distinct(2)=6.80272109e-07, distinct(3)=6.80272109e-07] @@ -585,7 +585,7 @@ project ├── prune: (1-3,6,14,16) └── interesting orderings: (+3,+2,+1) (+3,+2,+6) -opt format=show-all +opt format=hide-qual SELECT i_price, i_name, i_data FROM item WHERE i_id IN (125, 150, 175, 200, 25, 50, 75, 100, 225, 250, 275, 300) @@ -602,7 +602,7 @@ scan item ├── prune: (3-5) └── interesting orderings: (+1) -opt format=show-all +opt format=hide-qual SELECT s_quantity, s_ytd, s_order_cnt, s_remote_cnt, s_data, s_dist_05 FROM stock WHERE (s_i_id, s_w_id) IN ((1000, 4), (900, 4), (1100, 4), (1500, 4), (1400, 4)) @@ -618,7 +618,7 @@ project ├── prune: (1,3,8,14-17) ├── interesting orderings: (+1) └── scan stock - ├── columns: stock.s_i_id:1(int!null) stock.s_w_id:2(int!null) stock.s_quantity:3(int) stock.s_dist_05:8(string) stock.s_ytd:14(int) stock.s_order_cnt:15(int) stock.s_remote_cnt:16(int) stock.s_data:17(string) + ├── columns: s_i_id:1(int!null) s_w_id:2(int!null) s_quantity:3(int) s_dist_05:8(string) s_ytd:14(int) s_order_cnt:15(int) s_remote_cnt:16(int) s_data:17(string) ├── constraint: /2/1: [/4/900 - /4/900] [/4/1000 - /4/1000] [/4/1100 - /4/1100] [/4/1400 - /4/1400] [/4/1500 - /4/1500] ├── stats: [rows=5, distinct(1)=5, distinct(2)=1] ├── cost: 6.25 @@ -637,7 +637,7 @@ project # stringent response time requirements to satisfy on-line users. In addition, # this transaction includes non-primary key access to the CUSTOMER table. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT c_id FROM customer WHERE c_w_id = 10 AND c_d_id = 100 AND c_last = 'Smith' @@ -652,7 +652,7 @@ project ├── ordering: +4 ├── prune: (1,4) └── scan customer@customer_idx - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_first:4(string) customer.c_last:6(string!null) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_first:4(string) c_last:6(string!null) ├── constraint: /3/2/6/4/1: [/10/100/'Smith' - /10/100/'Smith'] ├── stats: [rows=6.80272109e-07, distinct(2)=6.80272109e-07, distinct(3)=6.80272109e-07, distinct(6)=6.80272109e-07] ├── cost: 7.4829932e-07 @@ -671,7 +671,7 @@ project # users. In addition, this table includes non-primary key access to the # CUSTOMER table. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = 10 AND c_d_id = 100 AND c_id = 50 @@ -685,7 +685,7 @@ project ├── fd: ()-->(4-6,17) ├── prune: (4-6,17) └── scan customer - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_first:4(string) customer.c_middle:5(string) customer.c_last:6(string) customer.c_balance:17(decimal) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_first:4(string) c_middle:5(string) c_last:6(string) c_balance:17(decimal) ├── constraint: /3/2/1: [/10/100/50 - /10/100/50] ├── cardinality: [0 - 1] ├── stats: [rows=6.80272109e-07, distinct(1)=6.80272109e-07, distinct(2)=6.80272109e-07, distinct(3)=6.80272109e-07] @@ -695,7 +695,7 @@ project ├── prune: (1-6,17) └── interesting orderings: (+3,+2,+1) (+3,+2,+6,+4,+1) -opt format=show-all +opt format=hide-qual SELECT c_id, c_balance, c_first, c_middle FROM customer WHERE c_w_id = 10 AND c_d_id = 100 AND c_last = 'Smith' @@ -710,7 +710,7 @@ project ├── ordering: +4 ├── prune: (1,4,5,17) └── index-join customer - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_first:4(string) customer.c_middle:5(string) customer.c_last:6(string!null) customer.c_balance:17(decimal) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_first:4(string) c_middle:5(string) c_last:6(string!null) c_balance:17(decimal) ├── stats: [rows=6.80272109e-07, distinct(2)=6.80272109e-07, distinct(3)=6.80272109e-07, distinct(6)=6.80272109e-07] ├── cost: 3.66666667e-06 ├── key: (1) @@ -718,7 +718,7 @@ project ├── ordering: +4 opt(2,3,6) ├── interesting orderings: (+3,+2,+1) (+3,+2,+6,+4,+1) └── scan customer@customer_idx - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_first:4(string) customer.c_last:6(string!null) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_first:4(string) c_last:6(string!null) ├── constraint: /3/2/6/4/1: [/10/100/'Smith' - /10/100/'Smith'] ├── stats: [rows=6.80272109e-07, distinct(2)=6.80272109e-07, distinct(3)=6.80272109e-07, distinct(6)=6.80272109e-07] ├── cost: 7.4829932e-07 @@ -728,7 +728,7 @@ project ├── prune: (1-4,6) └── interesting orderings: (+3,+2,+1) (+3,+2,+6,+4,+1) -opt format=show-all +opt format=hide-qual SELECT o_id, o_entry_d, o_carrier_id FROM "order" WHERE o_w_id = 10 AND o_d_id = 100 AND o_c_id = 50 @@ -744,7 +744,7 @@ project ├── fd: ()-->(1,5,6) ├── prune: (1,5,6) └── index-join order - ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_c_id:4(int!null) "order".o_entry_d:5(timestamp) "order".o_carrier_id:6(int) + ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_c_id:4(int!null) o_entry_d:5(timestamp) o_carrier_id:6(int) ├── cardinality: [0 - 1] ├── stats: [rows=6.80272109e-07] ├── cost: 3.55782313e-06 @@ -752,7 +752,7 @@ project ├── fd: ()-->(1-6) ├── interesting orderings: (+3,+2,-1) (+3,+2,+4,+1) └── scan order@secondary,rev - ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_c_id:4(int!null) + ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_c_id:4(int!null) ├── constraint: /3/2/4/1: [/10/100/50 - /10/100/50] ├── limit: 1(rev) ├── stats: [rows=6.80272109e-07] @@ -762,7 +762,7 @@ project ├── prune: (1-4) └── interesting orderings: (+3,+2,-1) (+3,+2,+4,+1) -opt format=show-all +opt format=hide-qual SELECT ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_delivery_d FROM order_line WHERE ol_w_id = 10 AND ol_d_id = 100 AND ol_o_id = 1000 @@ -774,7 +774,7 @@ project ├── prune: (5-9) ├── interesting orderings: (+6) └── scan order_line - ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) order_line.ol_i_id:5(int!null) order_line.ol_supply_w_id:6(int) order_line.ol_delivery_d:7(timestamp) order_line.ol_quantity:8(int) order_line.ol_amount:9(decimal) + ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_i_id:5(int!null) ol_supply_w_id:6(int) ol_delivery_d:7(timestamp) ol_quantity:8(int) ol_amount:9(decimal) ├── constraint: /3/2/-1/4: [/10/100/1000 - /10/100/1000] ├── stats: [rows=2.04081633e-07, distinct(1)=2.04081633e-07, distinct(2)=2.04081633e-07, distinct(3)=2.04081633e-07] ├── cost: 2.40816327e-07 @@ -798,7 +798,7 @@ project # indicating transaction completion. The result of the deferred execution is # recorded into a result file. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT no_o_id FROM new_order WHERE no_w_id = 10 AND no_d_id = 100 @@ -814,7 +814,7 @@ project ├── fd: ()-->(1) ├── prune: (1) └── scan new_order,rev - ├── columns: new_order.no_o_id:1(int!null) new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + ├── columns: no_o_id:1(int!null) no_d_id:2(int!null) no_w_id:3(int!null) ├── constraint: /3/2/-1: [/10/100 - /10/100] ├── limit: 1(rev) ├── stats: [rows=0.142857143] @@ -824,7 +824,7 @@ project ├── prune: (1-3) └── interesting orderings: (+3,+2,-1) -opt format=show-all +opt format=hide-qual SELECT sum(ol_amount) FROM order_line WHERE ol_w_id = 10 AND ol_d_id = 100 AND ol_o_id = 1000 @@ -838,7 +838,7 @@ scalar-group-by ├── fd: ()-->(11) ├── prune: (11) ├── scan order_line - │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) order_line.ol_amount:9(decimal) + │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_amount:9(decimal) │ ├── constraint: /3/2/-1/4: [/10/100/1000 - /10/100/1000] │ ├── stats: [rows=2.04081633e-07, distinct(1)=2.04081633e-07, distinct(2)=2.04081633e-07, distinct(3)=2.04081633e-07] │ ├── cost: 2.32653061e-07 @@ -847,7 +847,7 @@ scalar-group-by │ └── interesting orderings: (+3,+2,-1) └── aggregations [outer=(9)] └── sum [type=decimal, outer=(9)] - └── variable: order_line.ol_amount [type=decimal, outer=(9)] + └── variable: ol_amount [type=decimal, outer=(9)] # -------------------------------------------------- # 2.8 The Stock-Level Transaction @@ -857,7 +857,7 @@ scalar-group-by # heavy read-only database transaction with a low frequency of execution, a # relaxed response time requirement, and relaxed consistency requirements. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT d_next_o_id FROM district WHERE d_w_id = 10 AND d_id = 100 @@ -871,7 +871,7 @@ project ├── fd: ()-->(11) ├── prune: (11) └── scan district - ├── columns: district.d_id:1(int!null) district.d_w_id:2(int!null) district.d_next_o_id:11(int) + ├── columns: d_id:1(int!null) d_w_id:2(int!null) d_next_o_id:11(int) ├── constraint: /2/1: [/10/100 - /10/100] ├── cardinality: [0 - 1] ├── stats: [rows=0.142857143, distinct(1)=0.142857143, distinct(2)=0.142857143] @@ -881,7 +881,7 @@ project ├── prune: (1,2,11) └── interesting orderings: (+2,+1) -opt format=show-all +opt format=hide-qual SELECT count(DISTINCT s_i_id) FROM order_line JOIN stock @@ -900,14 +900,14 @@ scalar-group-by ├── fd: ()-->(28) ├── prune: (28) ├── inner-join (lookup stock) - │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) order_line.ol_i_id:5(int!null) stock.s_i_id:11(int!null) stock.s_w_id:12(int!null) stock.s_quantity:13(int!null) + │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_i_id:5(int!null) s_i_id:11(int!null) s_w_id:12(int!null) s_quantity:13(int!null) │ ├── key columns: [3 5] = [12 11] │ ├── stats: [rows=4.73185689e-06, distinct(3)=4.08163265e-06, distinct(5)=4.0816376e-06, distinct(11)=4.0816376e-06, distinct(12)=4.08163265e-06] │ ├── cost: 2.66578201e-05 │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) │ ├── interesting orderings: (+3,+2,-1) │ ├── scan order_line - │ │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) order_line.ol_i_id:5(int!null) + │ │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_i_id:5(int!null) │ │ ├── constraint: /3/2/-1/4: [/10/100/999 - /10/100/980] │ │ ├── stats: [rows=4.08163265e-06, distinct(1)=4.08163265e-06, distinct(2)=4.08163265e-06, distinct(3)=4.08163265e-06, distinct(5)=4.0816376e-06] │ │ ├── cost: 4.65306122e-06 @@ -916,21 +916,21 @@ scalar-group-by │ │ └── interesting orderings: (+3,+2,-1) │ └── filters [type=bool, outer=(3,5,11-13), constraints=(/3: (/NULL - ]; /5: (/NULL - ]; /11: (/NULL - ]; /12: [/10 - /10]; /13: (/NULL - /14]), fd=()-->(3,12), (5)==(11), (11)==(5), (3)==(12), (12)==(3)] │ ├── eq [type=bool, outer=(5,11), constraints=(/5: (/NULL - ]; /11: (/NULL - ])] - │ │ ├── variable: stock.s_i_id [type=int, outer=(11)] - │ │ └── variable: order_line.ol_i_id [type=int, outer=(5)] + │ │ ├── variable: s_i_id [type=int, outer=(11)] + │ │ └── variable: ol_i_id [type=int, outer=(5)] │ ├── eq [type=bool, outer=(3,12), constraints=(/3: (/NULL - ]; /12: (/NULL - ])] - │ │ ├── variable: stock.s_w_id [type=int, outer=(12)] - │ │ └── variable: order_line.ol_w_id [type=int, outer=(3)] + │ │ ├── variable: s_w_id [type=int, outer=(12)] + │ │ └── variable: ol_w_id [type=int, outer=(3)] │ ├── eq [type=bool, outer=(12), constraints=(/12: [/10 - /10]; tight)] - │ │ ├── variable: stock.s_w_id [type=int, outer=(12)] + │ │ ├── variable: s_w_id [type=int, outer=(12)] │ │ └── const: 10 [type=int] │ └── lt [type=bool, outer=(13), constraints=(/13: (/NULL - /14]; tight)] - │ ├── variable: stock.s_quantity [type=int, outer=(13)] + │ ├── variable: s_quantity [type=int, outer=(13)] │ └── const: 15 [type=int] └── aggregations [outer=(11)] └── count [type=int, outer=(11)] └── agg-distinct [type=int, outer=(11)] - └── variable: stock.s_i_id [type=int, outer=(11)] + └── variable: s_i_id [type=int, outer=(11)] # -------------------------------------------------- # Consistency Queries @@ -938,7 +938,7 @@ scalar-group-by # These queries run after TPCC in order to check database consistency. # They are not part of the benchmark itself. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT count(*) FROM warehouse FULL OUTER JOIN @@ -959,14 +959,14 @@ scalar-group-by ├── fd: ()-->(22) ├── prune: (22) ├── inner-join - │ ├── columns: warehouse.w_id:1(int!null) warehouse.w_ytd:9(decimal!null) district.d_w_id:11(int!null) sum_d_ytd:21(decimal!null) + │ ├── columns: w_id:1(int!null) w_ytd:9(decimal!null) d_w_id:11(int!null) sum_d_ytd:21(decimal!null) │ ├── stats: [rows=1.11111111, distinct(1)=1.11111111, distinct(11)=1.11111111] │ ├── cost: 126.511111 │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) │ ├── interesting orderings: (+1) (+11) │ ├── scan warehouse - │ │ ├── columns: warehouse.w_id:1(int!null) warehouse.w_ytd:9(decimal) + │ │ ├── columns: w_id:1(int!null) w_ytd:9(decimal) │ │ ├── stats: [rows=10, distinct(1)=10] │ │ ├── cost: 11.1 │ │ ├── key: (1) @@ -974,8 +974,8 @@ scalar-group-by │ │ ├── prune: (1,9) │ │ └── interesting orderings: (+1) │ ├── group-by - │ │ ├── columns: district.d_w_id:11(int!null) sum_d_ytd:21(decimal) - │ │ ├── grouping columns: district.d_w_id:11(int!null) + │ │ ├── columns: d_w_id:11(int!null) sum_d_ytd:21(decimal) + │ │ ├── grouping columns: d_w_id:11(int!null) │ │ ├── stats: [rows=10, distinct(11)=10] │ │ ├── cost: 115.1 │ │ ├── key: (11) @@ -983,25 +983,25 @@ scalar-group-by │ │ ├── prune: (21) │ │ ├── interesting orderings: (+11) │ │ ├── scan district - │ │ │ ├── columns: district.d_w_id:11(int!null) district.d_ytd:19(decimal) + │ │ │ ├── columns: d_w_id:11(int!null) d_ytd:19(decimal) │ │ │ ├── stats: [rows=100, distinct(11)=10] │ │ │ ├── cost: 113 │ │ │ ├── prune: (11,19) │ │ │ └── interesting orderings: (+11) │ │ └── aggregations [outer=(19)] │ │ └── sum [type=decimal, outer=(19)] - │ │ └── variable: district.d_ytd [type=decimal, outer=(19)] + │ │ └── variable: d_ytd [type=decimal, outer=(19)] │ └── filters [type=bool, outer=(1,9,11,21), constraints=(/1: (/NULL - ]; /9: (/NULL - ]; /11: (/NULL - ]; /21: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ ├── eq [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ])] - │ │ ├── variable: warehouse.w_id [type=int, outer=(1)] - │ │ └── variable: district.d_w_id [type=int, outer=(11)] + │ │ ├── variable: w_id [type=int, outer=(1)] + │ │ └── variable: d_w_id [type=int, outer=(11)] │ └── ne [type=bool, outer=(9,21), constraints=(/9: (/NULL - ]; /21: (/NULL - ])] - │ ├── variable: warehouse.w_ytd [type=decimal, outer=(9)] + │ ├── variable: w_ytd [type=decimal, outer=(9)] │ └── variable: sum_d_ytd [type=decimal, outer=(21)] └── aggregations └── count-rows [type=int] -opt format=show-all +opt format=hide-qual SELECT d_next_o_id FROM district ORDER BY d_w_id, d_id @@ -1016,7 +1016,7 @@ scan district ├── prune: (1,2,11) └── interesting orderings: (+2,+1) -opt format=show-all +opt format=hide-qual SELECT max(no_o_id) FROM new_order GROUP BY no_d_id, no_w_id @@ -1031,15 +1031,15 @@ sort ├── ordering: +3,+2 ├── prune: (4) └── group-by - ├── columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) max:4(int) - ├── grouping columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) max:4(int) + ├── grouping columns: no_d_id:2(int!null) no_w_id:3(int!null) ├── stats: [rows=100, distinct(2,3)=100] ├── cost: 98101 ├── key: (2,3) ├── fd: (2,3)-->(4) ├── prune: (4) ├── scan new_order - │ ├── columns: new_order.no_o_id:1(int!null) new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + │ ├── columns: no_o_id:1(int!null) no_d_id:2(int!null) no_w_id:3(int!null) │ ├── stats: [rows=90000, distinct(2,3)=100] │ ├── cost: 95400 │ ├── key: (1-3) @@ -1047,9 +1047,9 @@ sort │ └── interesting orderings: (+3,+2,-1) └── aggregations [outer=(1)] └── max [type=int, outer=(1)] - └── variable: new_order.no_o_id [type=int, outer=(1)] + └── variable: no_o_id [type=int, outer=(1)] -opt format=show-all +opt format=hide-qual SELECT max(o_id) FROM "order" GROUP BY o_d_id, o_w_id @@ -1064,15 +1064,15 @@ sort ├── ordering: +3,+2 ├── prune: (9) └── group-by - ├── columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) max:9(int) - ├── grouping columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + ├── columns: o_d_id:2(int!null) o_w_id:3(int!null) max:9(int) + ├── grouping columns: o_d_id:2(int!null) o_w_id:3(int!null) ├── stats: [rows=100, distinct(2,3)=100] ├── cost: 330001 ├── key: (2,3) ├── fd: (2,3)-->(9) ├── prune: (9) ├── scan order@order_idx - │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) │ ├── stats: [rows=300000, distinct(2,3)=100] │ ├── cost: 321000 │ ├── key: (1-3) @@ -1080,9 +1080,9 @@ sort │ └── interesting orderings: (+3,+2,-1) └── aggregations [outer=(1)] └── max [type=int, outer=(1)] - └── variable: "order".o_id [type=int, outer=(1)] + └── variable: o_id [type=int, outer=(1)] -opt format=show-all +opt format=hide-qual SELECT count(*) FROM ( @@ -1101,15 +1101,15 @@ scalar-group-by ├── fd: ()-->(8) ├── prune: (8) ├── select - │ ├── columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) column4:4(int) column5:5(int) column6:6(int) + │ ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) column4:4(int) column5:5(int) column6:6(int) │ ├── stats: [rows=33.3333333] │ ├── cost: 99902 │ ├── key: (2,3) │ ├── fd: (2,3)-->(4-6) │ ├── interesting orderings: (+3,+2) │ ├── group-by - │ │ ├── columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) column4:4(int) column5:5(int) column6:6(int) - │ │ ├── grouping columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + │ │ ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) column4:4(int) column5:5(int) column6:6(int) + │ │ ├── grouping columns: no_d_id:2(int!null) no_w_id:3(int!null) │ │ ├── stats: [rows=100, distinct(2,3)=100] │ │ ├── cost: 99901 │ │ ├── key: (2,3) @@ -1117,7 +1117,7 @@ scalar-group-by │ │ ├── prune: (4-6) │ │ ├── interesting orderings: (+3,+2) │ │ ├── scan new_order - │ │ │ ├── columns: new_order.no_o_id:1(int!null) new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + │ │ │ ├── columns: no_o_id:1(int!null) no_d_id:2(int!null) no_w_id:3(int!null) │ │ │ ├── stats: [rows=90000, distinct(2,3)=100] │ │ │ ├── cost: 95400 │ │ │ ├── key: (1-3) @@ -1125,9 +1125,9 @@ scalar-group-by │ │ │ └── interesting orderings: (+3,+2,-1) │ │ └── aggregations [outer=(1)] │ │ ├── max [type=int, outer=(1)] - │ │ │ └── variable: new_order.no_o_id [type=int, outer=(1)] + │ │ │ └── variable: no_o_id [type=int, outer=(1)] │ │ ├── min [type=int, outer=(1)] - │ │ │ └── variable: new_order.no_o_id [type=int, outer=(1)] + │ │ │ └── variable: no_o_id [type=int, outer=(1)] │ │ └── count-rows [type=int] │ └── filters [type=bool, outer=(4-6)] │ └── ne [type=bool, outer=(4-6)] @@ -1140,7 +1140,7 @@ scalar-group-by └── aggregations └── count-rows [type=int] -opt format=show-all +opt format=hide-qual SELECT sum(o_ol_cnt) FROM "order" GROUP BY o_w_id, o_d_id @@ -1155,24 +1155,24 @@ sort ├── ordering: +3,+2 ├── prune: (9) └── group-by - ├── columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) sum:9(decimal) - ├── grouping columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + ├── columns: o_d_id:2(int!null) o_w_id:3(int!null) sum:9(decimal) + ├── grouping columns: o_d_id:2(int!null) o_w_id:3(int!null) ├── stats: [rows=100, distinct(2,3)=100] ├── cost: 342001 ├── key: (2,3) ├── fd: (2,3)-->(9) ├── prune: (9) ├── scan order - │ ├── columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_ol_cnt:7(int) + │ ├── columns: o_d_id:2(int!null) o_w_id:3(int!null) o_ol_cnt:7(int) │ ├── stats: [rows=300000, distinct(2,3)=100] │ ├── cost: 333000 │ ├── prune: (2,3,7) │ └── interesting orderings: (+3,+2) └── aggregations [outer=(7)] └── sum [type=decimal, outer=(7)] - └── variable: "order".o_ol_cnt [type=int, outer=(7)] + └── variable: o_ol_cnt [type=int, outer=(7)] -opt format=show-all +opt format=hide-qual SELECT count(*) FROM order_line GROUP BY ol_w_id, ol_d_id @@ -1187,15 +1187,15 @@ sort ├── ordering: +3,+2 ├── prune: (11) └── group-by - ├── columns: order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) count:11(int) - ├── grouping columns: order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) + ├── columns: ol_d_id:2(int!null) ol_w_id:3(int!null) count:11(int) + ├── grouping columns: ol_d_id:2(int!null) ol_w_id:3(int!null) ├── stats: [rows=100, distinct(2,3)=100] ├── cost: 1100001 ├── key: (2,3) ├── fd: (2,3)-->(11) ├── prune: (11) ├── scan order_line@order_line_fk - │ ├── columns: order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) + │ ├── columns: ol_d_id:2(int!null) ol_w_id:3(int!null) │ ├── stats: [rows=1000000, distinct(2,3)=100] │ ├── cost: 1070000 │ ├── prune: (2,3) @@ -1203,33 +1203,33 @@ sort └── aggregations └── count-rows [type=int] -opt format=show-all +opt format=hide-qual (SELECT no_w_id, no_d_id, no_o_id FROM new_order) EXCEPT ALL (SELECT o_w_id, o_d_id, o_id FROM "order" WHERE o_carrier_id IS NULL) ---- except-all ├── columns: no_w_id:3(int!null) no_d_id:2(int!null) no_o_id:1(int!null) - ├── left columns: new_order.no_w_id:3(int!null) new_order.no_d_id:2(int!null) new_order.no_o_id:1(int!null) - ├── right columns: "order".o_w_id:6(int) "order".o_d_id:5(int) "order".o_id:4(int) + ├── left columns: no_w_id:3(int!null) no_d_id:2(int!null) no_o_id:1(int!null) + ├── right columns: o_w_id:6(int) o_d_id:5(int) o_id:4(int) ├── stats: [rows=90000] ├── cost: 424200.029 ├── scan new_order - │ ├── columns: new_order.no_o_id:1(int!null) new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + │ ├── columns: no_o_id:1(int!null) no_d_id:2(int!null) no_w_id:3(int!null) │ ├── stats: [rows=90000] │ ├── cost: 95400 │ ├── key: (1-3) │ ├── prune: (1-3) │ └── interesting orderings: (+3,+2,-1) └── project - ├── columns: "order".o_id:4(int!null) "order".o_d_id:5(int!null) "order".o_w_id:6(int!null) + ├── columns: o_id:4(int!null) o_d_id:5(int!null) o_w_id:6(int!null) ├── stats: [rows=1.42857143] ├── cost: 327000.014 ├── key: (4-6) ├── prune: (4-6) ├── interesting orderings: (+6,+5,-4) └── select - ├── columns: "order".o_id:4(int!null) "order".o_d_id:5(int!null) "order".o_w_id:6(int!null) "order".o_carrier_id:9(int) + ├── columns: o_id:4(int!null) o_d_id:5(int!null) o_w_id:6(int!null) o_carrier_id:9(int) ├── stats: [rows=1.42857143, distinct(9)=1] ├── cost: 327000 ├── key: (4-6) @@ -1237,7 +1237,7 @@ except-all ├── prune: (4-6) ├── interesting orderings: (+6,+5,-4) (+6,+5,+9,+4) ├── scan order@order_idx - │ ├── columns: "order".o_id:4(int!null) "order".o_d_id:5(int!null) "order".o_w_id:6(int!null) "order".o_carrier_id:9(int) + │ ├── columns: o_id:4(int!null) o_d_id:5(int!null) o_w_id:6(int!null) o_carrier_id:9(int) │ ├── stats: [rows=300000, distinct(9)=210000] │ ├── cost: 324000 │ ├── key: (4-6) @@ -1246,29 +1246,29 @@ except-all │ └── interesting orderings: (+6,+5,-4) (+6,+5,+9,+4) └── filters [type=bool, outer=(9), constraints=(/9: [/NULL - /NULL]; tight), fd=()-->(9)] └── is [type=bool, outer=(9), constraints=(/9: [/NULL - /NULL]; tight)] - ├── variable: "order".o_carrier_id [type=int, outer=(9)] + ├── variable: o_carrier_id [type=int, outer=(9)] └── null [type=unknown] -opt format=show-all +opt format=hide-qual (SELECT o_w_id, o_d_id, o_id FROM "order" WHERE o_carrier_id IS NULL) EXCEPT ALL (SELECT no_w_id, no_d_id, no_o_id FROM new_order) ---- except-all ├── columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) - ├── left columns: "order".o_w_id:3(int!null) "order".o_d_id:2(int!null) "order".o_id:1(int!null) - ├── right columns: new_order.no_w_id:11(int) new_order.no_d_id:10(int) new_order.no_o_id:9(int) + ├── left columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) + ├── right columns: no_w_id:11(int) no_d_id:10(int) no_o_id:9(int) ├── stats: [rows=1.42857143] ├── cost: 423300.043 ├── project - │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) │ ├── stats: [rows=1.42857143] │ ├── cost: 327000.014 │ ├── key: (1-3) │ ├── prune: (1-3) │ ├── interesting orderings: (+3,+2,-1) │ └── select - │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_carrier_id:6(int) + │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_carrier_id:6(int) │ ├── stats: [rows=1.42857143, distinct(6)=1] │ ├── cost: 327000 │ ├── key: (1-3) @@ -1276,7 +1276,7 @@ except-all │ ├── prune: (1-3) │ ├── interesting orderings: (+3,+2,-1) (+3,+2,+6,+1) │ ├── scan order@order_idx - │ │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_carrier_id:6(int) + │ │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_carrier_id:6(int) │ │ ├── stats: [rows=300000, distinct(6)=210000] │ │ ├── cost: 324000 │ │ ├── key: (1-3) @@ -1285,17 +1285,17 @@ except-all │ │ └── interesting orderings: (+3,+2,-1) (+3,+2,+6,+1) │ └── filters [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] │ └── is [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight)] - │ ├── variable: "order".o_carrier_id [type=int, outer=(6)] + │ ├── variable: o_carrier_id [type=int, outer=(6)] │ └── null [type=unknown] └── scan new_order - ├── columns: new_order.no_o_id:9(int!null) new_order.no_d_id:10(int!null) new_order.no_w_id:11(int!null) + ├── columns: no_o_id:9(int!null) no_d_id:10(int!null) no_w_id:11(int!null) ├── stats: [rows=90000] ├── cost: 95400 ├── key: (9-11) ├── prune: (9-11) └── interesting orderings: (+11,+10,-9) -opt format=show-all +opt format=hide-qual ( SELECT o_w_id, o_d_id, o_id, o_ol_cnt FROM "order" @@ -1311,12 +1311,12 @@ EXCEPT ALL ---- except-all ├── columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) o_ol_cnt:7(int) - ├── left columns: "order".o_w_id:3(int!null) "order".o_d_id:2(int!null) "order".o_id:1(int!null) "order".o_ol_cnt:7(int) - ├── right columns: order_line.ol_w_id:11(int) order_line.ol_d_id:10(int) order_line.ol_o_id:9(int) count:19(int) + ├── left columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) o_ol_cnt:7(int) + ├── right columns: ol_w_id:11(int) ol_d_id:10(int) ol_o_id:9(int) count:19(int) ├── stats: [rows=300000] ├── cost: 1464000 ├── scan order - │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_ol_cnt:7(int) + │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_ol_cnt:7(int) │ ├── stats: [rows=300000] │ ├── cost: 336000 │ ├── key: (1-3) @@ -1324,8 +1324,8 @@ except-all │ ├── prune: (1-3,7) │ └── interesting orderings: (+3,+2,-1) └── group-by - ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) count:19(int) - ├── grouping columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) + ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) count:19(int) + ├── grouping columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) ├── stats: [rows=100000, distinct(9-11)=100000] ├── cost: 1121000 ├── key: (9-11) @@ -1333,7 +1333,7 @@ except-all ├── prune: (19) ├── interesting orderings: (+11,+10,-9) ├── scan order_line@order_line_fk - │ ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) + │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) │ ├── stats: [rows=1000000, distinct(9-11)=100000] │ ├── cost: 1080000 │ ├── prune: (9-11) @@ -1341,7 +1341,7 @@ except-all └── aggregations └── count-rows [type=int] -opt format=show-all +opt format=hide-qual ( SELECT ol_w_id, ol_d_id, ol_o_id, count(*) FROM order_line @@ -1357,13 +1357,13 @@ EXCEPT ALL ---- except-all ├── columns: ol_w_id:3(int!null) ol_d_id:2(int!null) ol_o_id:1(int!null) count:11(int) - ├── left columns: order_line.ol_w_id:3(int!null) order_line.ol_d_id:2(int!null) order_line.ol_o_id:1(int!null) count:11(int) - ├── right columns: "order".o_w_id:14(int) "order".o_d_id:13(int) "order".o_id:12(int) "order".o_ol_cnt:18(int) + ├── left columns: ol_w_id:3(int!null) ol_d_id:2(int!null) ol_o_id:1(int!null) count:11(int) + ├── right columns: o_w_id:14(int) o_d_id:13(int) o_id:12(int) o_ol_cnt:18(int) ├── stats: [rows=100000] ├── cost: 1462000 ├── group-by - │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) count:11(int) - │ ├── grouping columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) + │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) count:11(int) + │ ├── grouping columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) │ ├── stats: [rows=100000, distinct(1-3)=100000] │ ├── cost: 1121000 │ ├── key: (1-3) @@ -1371,7 +1371,7 @@ except-all │ ├── prune: (11) │ ├── interesting orderings: (+3,+2,-1) │ ├── scan order_line@order_line_fk - │ │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) + │ │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) │ │ ├── stats: [rows=1000000, distinct(1-3)=100000] │ │ ├── cost: 1080000 │ │ ├── prune: (1-3) @@ -1379,7 +1379,7 @@ except-all │ └── aggregations │ └── count-rows [type=int] └── scan order - ├── columns: "order".o_id:12(int!null) "order".o_d_id:13(int!null) "order".o_w_id:14(int!null) "order".o_ol_cnt:18(int) + ├── columns: o_id:12(int!null) o_d_id:13(int!null) o_w_id:14(int!null) o_ol_cnt:18(int) ├── stats: [rows=300000] ├── cost: 336000 ├── key: (12-14) @@ -1387,7 +1387,7 @@ except-all ├── prune: (12-14,18) └── interesting orderings: (+14,+13,-12) -opt format=show-all +opt format=hide-qual SELECT count(*) FROM ( @@ -1413,25 +1413,25 @@ scalar-group-by ├── fd: ()-->(19) ├── prune: (19) ├── select - │ ├── columns: "order".o_id:1(int) "order".o_d_id:2(int) "order".o_w_id:3(int) order_line.ol_o_id:9(int) order_line.ol_d_id:10(int) order_line.ol_w_id:11(int) + │ ├── columns: o_id:1(int) o_d_id:2(int) o_w_id:3(int) ol_o_id:9(int) ol_d_id:10(int) ol_w_id:11(int) │ ├── stats: [rows=0.701984663] │ ├── cost: 1477000.11 │ ├── interesting orderings: (+3,+2,-1) (+11,+10,-9) │ ├── full-join - │ │ ├── columns: "order".o_id:1(int) "order".o_d_id:2(int) "order".o_w_id:3(int) order_line.ol_o_id:9(int) order_line.ol_d_id:10(int) order_line.ol_w_id:11(int) + │ │ ├── columns: o_id:1(int) o_d_id:2(int) o_w_id:3(int) ol_o_id:9(int) ol_d_id:10(int) ol_w_id:11(int) │ │ ├── stats: [rows=2.10595399] │ │ ├── cost: 1477000.09 │ │ ├── reject-nulls: (1-3,9-11) │ │ ├── interesting orderings: (+3,+2,-1) (+11,+10,-9) │ │ ├── project - │ │ │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + │ │ │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) │ │ │ ├── stats: [rows=1.42857143, distinct(1)=1.42856997, distinct(2)=1.42856997, distinct(3)=1.33122395] │ │ │ ├── cost: 327000.014 │ │ │ ├── key: (1-3) │ │ │ ├── prune: (1-3) │ │ │ ├── interesting orderings: (+3,+2,-1) │ │ │ └── select - │ │ │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_carrier_id:6(int) + │ │ │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_carrier_id:6(int) │ │ │ ├── stats: [rows=1.42857143, distinct(1)=1.42856997, distinct(2)=1.42856997, distinct(3)=1.33122395, distinct(6)=1] │ │ │ ├── cost: 327000 │ │ │ ├── key: (1-3) @@ -1439,7 +1439,7 @@ scalar-group-by │ │ │ ├── prune: (1-3) │ │ │ ├── interesting orderings: (+3,+2,-1) (+3,+2,+6,+1) │ │ │ ├── scan order@order_idx - │ │ │ │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_carrier_id:6(int) + │ │ │ │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_carrier_id:6(int) │ │ │ │ ├── stats: [rows=300000, distinct(1)=210000, distinct(2)=210000, distinct(3)=10, distinct(6)=210000] │ │ │ │ ├── cost: 324000 │ │ │ │ ├── key: (1-3) @@ -1448,48 +1448,48 @@ scalar-group-by │ │ │ │ └── interesting orderings: (+3,+2,-1) (+3,+2,+6,+1) │ │ │ └── filters [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] │ │ │ └── is [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight)] - │ │ │ ├── variable: "order".o_carrier_id [type=int, outer=(6)] + │ │ │ ├── variable: o_carrier_id [type=int, outer=(6)] │ │ │ └── null [type=unknown] │ │ ├── project - │ │ │ ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) + │ │ │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) │ │ │ ├── stats: [rows=1.42857143, distinct(9)=1.42857099, distinct(10)=1.42857099, distinct(11)=1.33122189] │ │ │ ├── cost: 1150000.01 │ │ │ ├── prune: (9-11) │ │ │ ├── interesting orderings: (+11,+10,-9) │ │ │ └── select - │ │ │ ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) order_line.ol_delivery_d:15(timestamp) + │ │ │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) ol_delivery_d:15(timestamp) │ │ │ ├── stats: [rows=1.42857143, distinct(9)=1.42857099, distinct(10)=1.42857099, distinct(11)=1.33122189, distinct(15)=1] │ │ │ ├── cost: 1150000 │ │ │ ├── fd: ()-->(15) │ │ │ ├── prune: (9-11) │ │ │ ├── interesting orderings: (+11,+10,-9) │ │ │ ├── scan order_line - │ │ │ │ ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) order_line.ol_delivery_d:15(timestamp) + │ │ │ │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) ol_delivery_d:15(timestamp) │ │ │ │ ├── stats: [rows=1000000, distinct(9)=700000, distinct(10)=700000, distinct(11)=10, distinct(15)=700000] │ │ │ │ ├── cost: 1140000 │ │ │ │ ├── prune: (9-11,15) │ │ │ │ └── interesting orderings: (+11,+10,-9) │ │ │ └── filters [type=bool, outer=(15), constraints=(/15: [/NULL - /NULL]; tight), fd=()-->(15)] │ │ │ └── is [type=bool, outer=(15), constraints=(/15: [/NULL - /NULL]; tight)] - │ │ │ ├── variable: order_line.ol_delivery_d [type=timestamp, outer=(15)] + │ │ │ ├── variable: ol_delivery_d [type=timestamp, outer=(15)] │ │ │ └── null [type=unknown] │ │ └── filters [type=bool, outer=(1-3,9-11), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /3: (/NULL - ]; /9: (/NULL - ]; /10: (/NULL - ]; /11: (/NULL - ]), fd=(3)==(11), (11)==(3), (2)==(10), (10)==(2), (1)==(9), (9)==(1)] │ │ ├── eq [type=bool, outer=(3,11), constraints=(/3: (/NULL - ]; /11: (/NULL - ])] - │ │ │ ├── variable: order_line.ol_w_id [type=int, outer=(11)] - │ │ │ └── variable: "order".o_w_id [type=int, outer=(3)] + │ │ │ ├── variable: ol_w_id [type=int, outer=(11)] + │ │ │ └── variable: o_w_id [type=int, outer=(3)] │ │ ├── eq [type=bool, outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ])] - │ │ │ ├── variable: order_line.ol_d_id [type=int, outer=(10)] - │ │ │ └── variable: "order".o_d_id [type=int, outer=(2)] + │ │ │ ├── variable: ol_d_id [type=int, outer=(10)] + │ │ │ └── variable: o_d_id [type=int, outer=(2)] │ │ └── eq [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] - │ │ ├── variable: order_line.ol_o_id [type=int, outer=(9)] - │ │ └── variable: "order".o_id [type=int, outer=(1)] + │ │ ├── variable: ol_o_id [type=int, outer=(9)] + │ │ └── variable: o_id [type=int, outer=(1)] │ └── filters [type=bool, outer=(1,9)] │ └── or [type=bool, outer=(1,9)] │ ├── is [type=bool, outer=(9)] - │ │ ├── variable: order_line.ol_o_id [type=int, outer=(9)] + │ │ ├── variable: ol_o_id [type=int, outer=(9)] │ │ └── null [type=unknown] │ └── is [type=bool, outer=(1)] - │ ├── variable: "order".o_id [type=int, outer=(1)] + │ ├── variable: o_id [type=int, outer=(1)] │ └── null [type=unknown] └── aggregations └── count-rows [type=int] diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats index 5993b40be003..890bcb9bc0b2 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats @@ -344,7 +344,7 @@ TABLE order_line # reflect on-line database activity as typically found in production # environments. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT w_tax FROM warehouse WHERE w_id = 10 ---- project @@ -356,7 +356,7 @@ project ├── fd: ()-->(8) ├── prune: (8) └── scan warehouse - ├── columns: warehouse.w_id:1(int!null) warehouse.w_tax:8(decimal) + ├── columns: w_id:1(int!null) w_tax:8(decimal) ├── constraint: /1: [/10 - /10] ├── cardinality: [0 - 1] ├── stats: [rows=1, distinct(1)=1] @@ -366,7 +366,7 @@ project ├── prune: (1,8) └── interesting orderings: (+1) -opt format=show-all +opt format=hide-qual SELECT c_discount, c_last, c_credit FROM customer WHERE c_w_id = 10 AND c_d_id = 100 AND c_id = 50 @@ -380,7 +380,7 @@ project ├── fd: ()-->(6,14,16) ├── prune: (6,14,16) └── scan customer - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_last:6(string) customer.c_credit:14(string) customer.c_discount:16(decimal) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_last:6(string) c_credit:14(string) c_discount:16(decimal) ├── constraint: /3/2/1: [/10/100/50 - /10/100/50] ├── cardinality: [0 - 1] ├── stats: [rows=2.9154519e-06, distinct(1)=2.9154519e-06, distinct(2)=2.9154519e-06, distinct(3)=2.9154519e-06] @@ -390,7 +390,7 @@ project ├── prune: (1-3,6,14,16) └── interesting orderings: (+3,+2,+1) (+3,+2,+6) -opt format=show-all +opt format=hide-qual SELECT i_price, i_name, i_data FROM item WHERE i_id IN (125, 150, 175, 200, 25, 50, 75, 100, 225, 250, 275, 300) @@ -407,7 +407,7 @@ scan item ├── prune: (3-5) └── interesting orderings: (+1) -opt format=show-all +opt format=hide-qual SELECT s_quantity, s_ytd, s_order_cnt, s_remote_cnt, s_data, s_dist_05 FROM stock WHERE (s_i_id, s_w_id) IN ((1000, 4), (900, 4), (1100, 4), (1500, 4), (1400, 4)) @@ -423,7 +423,7 @@ project ├── prune: (1,3,8,14-17) ├── interesting orderings: (+1) └── scan stock - ├── columns: stock.s_i_id:1(int!null) stock.s_w_id:2(int!null) stock.s_quantity:3(int) stock.s_dist_05:8(string) stock.s_ytd:14(int) stock.s_order_cnt:15(int) stock.s_remote_cnt:16(int) stock.s_data:17(string) + ├── columns: s_i_id:1(int!null) s_w_id:2(int!null) s_quantity:3(int) s_dist_05:8(string) s_ytd:14(int) s_order_cnt:15(int) s_remote_cnt:16(int) s_data:17(string) ├── constraint: /2/1: [/4/900 - /4/900] [/4/1000 - /4/1000] [/4/1100 - /4/1100] [/4/1400 - /4/1400] [/4/1500 - /4/1500] ├── stats: [rows=0.0102040816, distinct(1)=0.0102040816, distinct(2)=0.0102040816] ├── cost: 0.012755102 @@ -442,7 +442,7 @@ project # stringent response time requirements to satisfy on-line users. In addition, # this transaction includes non-primary key access to the CUSTOMER table. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT c_id FROM customer WHERE c_w_id = 10 AND c_d_id = 100 AND c_last = 'Smith' @@ -457,7 +457,7 @@ project ├── ordering: +4 ├── prune: (1,4) └── scan customer@customer_idx - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_first:4(string) customer.c_last:6(string!null) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_first:4(string) c_last:6(string!null) ├── constraint: /3/2/6/4/1: [/10/100/'Smith' - /10/100/'Smith'] ├── stats: [rows=2.9154519e-06, distinct(2)=2.9154519e-06, distinct(3)=2.9154519e-06, distinct(6)=2.9154519e-06] ├── cost: 3.20699708e-06 @@ -476,7 +476,7 @@ project # users. In addition, this table includes non-primary key access to the # CUSTOMER table. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = 10 AND c_d_id = 100 AND c_id = 50 @@ -490,7 +490,7 @@ project ├── fd: ()-->(4-6,17) ├── prune: (4-6,17) └── scan customer - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_first:4(string) customer.c_middle:5(string) customer.c_last:6(string) customer.c_balance:17(decimal) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_first:4(string) c_middle:5(string) c_last:6(string) c_balance:17(decimal) ├── constraint: /3/2/1: [/10/100/50 - /10/100/50] ├── cardinality: [0 - 1] ├── stats: [rows=2.9154519e-06, distinct(1)=2.9154519e-06, distinct(2)=2.9154519e-06, distinct(3)=2.9154519e-06] @@ -500,7 +500,7 @@ project ├── prune: (1-6,17) └── interesting orderings: (+3,+2,+1) (+3,+2,+6,+4,+1) -opt format=show-all +opt format=hide-qual SELECT c_id, c_balance, c_first, c_middle FROM customer WHERE c_w_id = 10 AND c_d_id = 100 AND c_last = 'Smith' @@ -515,7 +515,7 @@ project ├── ordering: +4 ├── prune: (1,4,5,17) └── index-join customer - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_first:4(string) customer.c_middle:5(string) customer.c_last:6(string!null) customer.c_balance:17(decimal) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_first:4(string) c_middle:5(string) c_last:6(string!null) c_balance:17(decimal) ├── stats: [rows=2.9154519e-06, distinct(2)=2.9154519e-06, distinct(3)=2.9154519e-06, distinct(6)=2.9154519e-06] ├── cost: 1.57142857e-05 ├── key: (1) @@ -523,7 +523,7 @@ project ├── ordering: +4 opt(2,3,6) ├── interesting orderings: (+3,+2,+1) (+3,+2,+6,+4,+1) └── scan customer@customer_idx - ├── columns: customer.c_id:1(int!null) customer.c_d_id:2(int!null) customer.c_w_id:3(int!null) customer.c_first:4(string) customer.c_last:6(string!null) + ├── columns: c_id:1(int!null) c_d_id:2(int!null) c_w_id:3(int!null) c_first:4(string) c_last:6(string!null) ├── constraint: /3/2/6/4/1: [/10/100/'Smith' - /10/100/'Smith'] ├── stats: [rows=2.9154519e-06, distinct(2)=2.9154519e-06, distinct(3)=2.9154519e-06, distinct(6)=2.9154519e-06] ├── cost: 3.20699708e-06 @@ -533,7 +533,7 @@ project ├── prune: (1-4,6) └── interesting orderings: (+3,+2,+1) (+3,+2,+6,+4,+1) -opt format=show-all +opt format=hide-qual SELECT o_id, o_entry_d, o_carrier_id FROM "order" WHERE o_w_id = 10 AND o_d_id = 100 AND o_c_id = 50 @@ -549,7 +549,7 @@ project ├── fd: ()-->(1,5,6) ├── prune: (1,5,6) └── index-join order - ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_c_id:4(int!null) "order".o_entry_d:5(timestamp) "order".o_carrier_id:6(int) + ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_c_id:4(int!null) o_entry_d:5(timestamp) o_carrier_id:6(int) ├── cardinality: [0 - 1] ├── stats: [rows=2.9154519e-06] ├── cost: 1.52478134e-05 @@ -557,7 +557,7 @@ project ├── fd: ()-->(1-6) ├── interesting orderings: (+3,+2,-1) (+3,+2,+4,+1) └── scan order@secondary,rev - ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_c_id:4(int!null) + ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_c_id:4(int!null) ├── constraint: /3/2/4/1: [/10/100/50 - /10/100/50] ├── limit: 1(rev) ├── stats: [rows=2.9154519e-06] @@ -567,7 +567,7 @@ project ├── prune: (1-4) └── interesting orderings: (+3,+2,-1) (+3,+2,+4,+1) -opt format=show-all +opt format=hide-qual SELECT ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_delivery_d FROM order_line WHERE ol_w_id = 10 AND ol_d_id = 100 AND ol_o_id = 1000 @@ -579,7 +579,7 @@ project ├── prune: (5-9) ├── interesting orderings: (+6) └── scan order_line - ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) order_line.ol_i_id:5(int!null) order_line.ol_supply_w_id:6(int) order_line.ol_delivery_d:7(timestamp) order_line.ol_quantity:8(int) order_line.ol_amount:9(decimal) + ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_i_id:5(int!null) ol_supply_w_id:6(int) ol_delivery_d:7(timestamp) ol_quantity:8(int) ol_amount:9(decimal) ├── constraint: /3/2/-1/4: [/10/100/1000 - /10/100/1000] ├── stats: [rows=2.9154519e-06, distinct(1)=2.9154519e-06, distinct(2)=2.9154519e-06, distinct(3)=2.9154519e-06] ├── cost: 3.44023324e-06 @@ -603,7 +603,7 @@ project # indicating transaction completion. The result of the deferred execution is # recorded into a result file. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT no_o_id FROM new_order WHERE no_w_id = 10 AND no_d_id = 100 @@ -619,7 +619,7 @@ project ├── fd: ()-->(1) ├── prune: (1) └── scan new_order,rev - ├── columns: new_order.no_o_id:1(int!null) new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + ├── columns: no_o_id:1(int!null) no_d_id:2(int!null) no_w_id:3(int!null) ├── constraint: /3/2/-1: [/10/100 - /10/100] ├── limit: 1(rev) ├── stats: [rows=0.00204081633] @@ -629,7 +629,7 @@ project ├── prune: (1-3) └── interesting orderings: (+3,+2,-1) -opt format=show-all +opt format=hide-qual SELECT sum(ol_amount) FROM order_line WHERE ol_w_id = 10 AND ol_d_id = 100 AND ol_o_id = 1000 @@ -643,7 +643,7 @@ scalar-group-by ├── fd: ()-->(11) ├── prune: (11) ├── scan order_line - │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) order_line.ol_amount:9(decimal) + │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_amount:9(decimal) │ ├── constraint: /3/2/-1/4: [/10/100/1000 - /10/100/1000] │ ├── stats: [rows=2.9154519e-06, distinct(1)=2.9154519e-06, distinct(2)=2.9154519e-06, distinct(3)=2.9154519e-06] │ ├── cost: 3.32361516e-06 @@ -652,7 +652,7 @@ scalar-group-by │ └── interesting orderings: (+3,+2,-1) └── aggregations [outer=(9)] └── sum [type=decimal, outer=(9)] - └── variable: order_line.ol_amount [type=decimal, outer=(9)] + └── variable: ol_amount [type=decimal, outer=(9)] # -------------------------------------------------- # 2.8 The Stock-Level Transaction @@ -662,7 +662,7 @@ scalar-group-by # heavy read-only database transaction with a low frequency of execution, a # relaxed response time requirement, and relaxed consistency requirements. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT d_next_o_id FROM district WHERE d_w_id = 10 AND d_id = 100 @@ -676,7 +676,7 @@ project ├── fd: ()-->(11) ├── prune: (11) └── scan district - ├── columns: district.d_id:1(int!null) district.d_w_id:2(int!null) district.d_next_o_id:11(int) + ├── columns: d_id:1(int!null) d_w_id:2(int!null) d_next_o_id:11(int) ├── constraint: /2/1: [/10/100 - /10/100] ├── cardinality: [0 - 1] ├── stats: [rows=0.00204081633, distinct(1)=0.00204081633, distinct(2)=0.00204081633] @@ -686,7 +686,7 @@ project ├── prune: (1,2,11) └── interesting orderings: (+2,+1) -opt format=show-all +opt format=hide-qual SELECT count(DISTINCT s_i_id) FROM order_line JOIN stock @@ -705,14 +705,14 @@ scalar-group-by ├── fd: ()-->(28) ├── prune: (28) ├── inner-join (lookup stock) - │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) order_line.ol_i_id:5(int!null) stock.s_i_id:11(int!null) stock.s_w_id:12(int!null) stock.s_quantity:13(int!null) + │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_i_id:5(int!null) s_i_id:11(int!null) s_w_id:12(int!null) s_quantity:13(int!null) │ ├── key columns: [3 5] = [12 11] │ ├── stats: [rows=2.77662085e-05, distinct(3)=2.77662085e-05, distinct(5)=2.77662085e-05, distinct(11)=2.77662085e-05, distinct(12)=2.77662085e-05] │ ├── cost: 0.000333027905 │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) │ ├── interesting orderings: (+3,+2,-1) │ ├── scan order_line - │ │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) order_line.ol_i_id:5(int!null) + │ │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_i_id:5(int!null) │ │ ├── constraint: /3/2/-1/4: [/10/100/999 - /10/100/980] │ │ ├── stats: [rows=5.83090379e-05, distinct(1)=5.83090379e-05, distinct(2)=5.83090379e-05, distinct(3)=5.83090379e-05, distinct(5)=5.83090372e-05] │ │ ├── cost: 6.64723032e-05 @@ -721,21 +721,21 @@ scalar-group-by │ │ └── interesting orderings: (+3,+2,-1) │ └── filters [type=bool, outer=(3,5,11-13), constraints=(/3: (/NULL - ]; /5: (/NULL - ]; /11: (/NULL - ]; /12: [/10 - /10]; /13: (/NULL - /14]), fd=()-->(3,12), (5)==(11), (11)==(5), (3)==(12), (12)==(3)] │ ├── eq [type=bool, outer=(5,11), constraints=(/5: (/NULL - ]; /11: (/NULL - ])] - │ │ ├── variable: stock.s_i_id [type=int, outer=(11)] - │ │ └── variable: order_line.ol_i_id [type=int, outer=(5)] + │ │ ├── variable: s_i_id [type=int, outer=(11)] + │ │ └── variable: ol_i_id [type=int, outer=(5)] │ ├── eq [type=bool, outer=(3,12), constraints=(/3: (/NULL - ]; /12: (/NULL - ])] - │ │ ├── variable: stock.s_w_id [type=int, outer=(12)] - │ │ └── variable: order_line.ol_w_id [type=int, outer=(3)] + │ │ ├── variable: s_w_id [type=int, outer=(12)] + │ │ └── variable: ol_w_id [type=int, outer=(3)] │ ├── eq [type=bool, outer=(12), constraints=(/12: [/10 - /10]; tight)] - │ │ ├── variable: stock.s_w_id [type=int, outer=(12)] + │ │ ├── variable: s_w_id [type=int, outer=(12)] │ │ └── const: 10 [type=int] │ └── lt [type=bool, outer=(13), constraints=(/13: (/NULL - /14]; tight)] - │ ├── variable: stock.s_quantity [type=int, outer=(13)] + │ ├── variable: s_quantity [type=int, outer=(13)] │ └── const: 15 [type=int] └── aggregations [outer=(11)] └── count [type=int, outer=(11)] └── agg-distinct [type=int, outer=(11)] - └── variable: stock.s_i_id [type=int, outer=(11)] + └── variable: s_i_id [type=int, outer=(11)] # -------------------------------------------------- # Consistency Queries @@ -743,7 +743,7 @@ scalar-group-by # These queries run after TPCC in order to check database consistency. # They are not part of the benchmark itself. # -------------------------------------------------- -opt format=show-all +opt format=hide-qual SELECT count(*) FROM warehouse FULL OUTER JOIN @@ -764,14 +764,14 @@ scalar-group-by ├── fd: ()-->(22) ├── prune: (22) ├── inner-join - │ ├── columns: warehouse.w_id:1(int!null) warehouse.w_ytd:9(decimal!null) district.d_w_id:11(int!null) sum_d_ytd:21(decimal!null) + │ ├── columns: w_id:1(int!null) w_ytd:9(decimal!null) d_w_id:11(int!null) sum_d_ytd:21(decimal!null) │ ├── stats: [rows=77.7777778, distinct(1)=77.7777778, distinct(11)=77.7777778] │ ├── cost: 2292.52778 │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) │ ├── interesting orderings: (+1) (+11) │ ├── scan warehouse - │ │ ├── columns: warehouse.w_id:1(int!null) warehouse.w_ytd:9(decimal) + │ │ ├── columns: w_id:1(int!null) w_ytd:9(decimal) │ │ ├── stats: [rows=1000, distinct(1)=1000] │ │ ├── cost: 1110 │ │ ├── key: (1) @@ -779,8 +779,8 @@ scalar-group-by │ │ ├── prune: (1,9) │ │ └── interesting orderings: (+1) │ ├── group-by - │ │ ├── columns: district.d_w_id:11(int!null) sum_d_ytd:21(decimal) - │ │ ├── grouping columns: district.d_w_id:11(int!null) + │ │ ├── columns: d_w_id:11(int!null) sum_d_ytd:21(decimal) + │ │ ├── grouping columns: d_w_id:11(int!null) │ │ ├── stats: [rows=700, distinct(11)=700] │ │ ├── cost: 1157 │ │ ├── key: (11) @@ -788,25 +788,25 @@ scalar-group-by │ │ ├── prune: (21) │ │ ├── interesting orderings: (+11) │ │ ├── scan district - │ │ │ ├── columns: district.d_w_id:11(int!null) district.d_ytd:19(decimal) + │ │ │ ├── columns: d_w_id:11(int!null) d_ytd:19(decimal) │ │ │ ├── stats: [rows=1000, distinct(11)=700] │ │ │ ├── cost: 1130 │ │ │ ├── prune: (11,19) │ │ │ └── interesting orderings: (+11) │ │ └── aggregations [outer=(19)] │ │ └── sum [type=decimal, outer=(19)] - │ │ └── variable: district.d_ytd [type=decimal, outer=(19)] + │ │ └── variable: d_ytd [type=decimal, outer=(19)] │ └── filters [type=bool, outer=(1,9,11,21), constraints=(/1: (/NULL - ]; /9: (/NULL - ]; /11: (/NULL - ]; /21: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ ├── eq [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ])] - │ │ ├── variable: warehouse.w_id [type=int, outer=(1)] - │ │ └── variable: district.d_w_id [type=int, outer=(11)] + │ │ ├── variable: w_id [type=int, outer=(1)] + │ │ └── variable: d_w_id [type=int, outer=(11)] │ └── ne [type=bool, outer=(9,21), constraints=(/9: (/NULL - ]; /21: (/NULL - ])] - │ ├── variable: warehouse.w_ytd [type=decimal, outer=(9)] + │ ├── variable: w_ytd [type=decimal, outer=(9)] │ └── variable: sum_d_ytd [type=decimal, outer=(21)] └── aggregations └── count-rows [type=int] -opt format=show-all +opt format=hide-qual SELECT d_next_o_id FROM district ORDER BY d_w_id, d_id @@ -821,7 +821,7 @@ scan district ├── prune: (1,2,11) └── interesting orderings: (+2,+1) -opt format=show-all +opt format=hide-qual SELECT max(no_o_id) FROM new_order GROUP BY no_d_id, no_w_id @@ -836,15 +836,15 @@ sort ├── ordering: +3,+2 ├── prune: (4) └── group-by - ├── columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) max:4(int) - ├── grouping columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) max:4(int) + ├── grouping columns: no_d_id:2(int!null) no_w_id:3(int!null) ├── stats: [rows=1000, distinct(2,3)=1000] ├── cost: 1100 ├── key: (2,3) ├── fd: (2,3)-->(4) ├── prune: (4) ├── scan new_order - │ ├── columns: new_order.no_o_id:1(int!null) new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + │ ├── columns: no_o_id:1(int!null) no_d_id:2(int!null) no_w_id:3(int!null) │ ├── stats: [rows=1000, distinct(2,3)=1000] │ ├── cost: 1060 │ ├── key: (1-3) @@ -852,9 +852,9 @@ sort │ └── interesting orderings: (+3,+2,-1) └── aggregations [outer=(1)] └── max [type=int, outer=(1)] - └── variable: new_order.no_o_id [type=int, outer=(1)] + └── variable: no_o_id [type=int, outer=(1)] -opt format=show-all +opt format=hide-qual SELECT max(o_id) FROM "order" GROUP BY o_d_id, o_w_id @@ -869,15 +869,15 @@ sort ├── ordering: +3,+2 ├── prune: (9) └── group-by - ├── columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) max:9(int) - ├── grouping columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + ├── columns: o_d_id:2(int!null) o_w_id:3(int!null) max:9(int) + ├── grouping columns: o_d_id:2(int!null) o_w_id:3(int!null) ├── stats: [rows=1000, distinct(2,3)=1000] ├── cost: 1110 ├── key: (2,3) ├── fd: (2,3)-->(9) ├── prune: (9) ├── scan order@order_idx - │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) │ ├── stats: [rows=1000, distinct(2,3)=1000] │ ├── cost: 1070 │ ├── key: (1-3) @@ -885,9 +885,9 @@ sort │ └── interesting orderings: (+3,+2,-1) └── aggregations [outer=(1)] └── max [type=int, outer=(1)] - └── variable: "order".o_id [type=int, outer=(1)] + └── variable: o_id [type=int, outer=(1)] -opt format=show-all +opt format=hide-qual SELECT count(*) FROM ( @@ -906,15 +906,15 @@ scalar-group-by ├── fd: ()-->(8) ├── prune: (8) ├── select - │ ├── columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) column4:4(int) column5:5(int) column6:6(int) + │ ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) column4:4(int) column5:5(int) column6:6(int) │ ├── stats: [rows=333.333333] │ ├── cost: 1130 │ ├── key: (2,3) │ ├── fd: (2,3)-->(4-6) │ ├── interesting orderings: (+3,+2) │ ├── group-by - │ │ ├── columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) column4:4(int) column5:5(int) column6:6(int) - │ │ ├── grouping columns: new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + │ │ ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) column4:4(int) column5:5(int) column6:6(int) + │ │ ├── grouping columns: no_d_id:2(int!null) no_w_id:3(int!null) │ │ ├── stats: [rows=1000, distinct(2,3)=1000] │ │ ├── cost: 1120 │ │ ├── key: (2,3) @@ -922,7 +922,7 @@ scalar-group-by │ │ ├── prune: (4-6) │ │ ├── interesting orderings: (+3,+2) │ │ ├── scan new_order - │ │ │ ├── columns: new_order.no_o_id:1(int!null) new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + │ │ │ ├── columns: no_o_id:1(int!null) no_d_id:2(int!null) no_w_id:3(int!null) │ │ │ ├── stats: [rows=1000, distinct(2,3)=1000] │ │ │ ├── cost: 1060 │ │ │ ├── key: (1-3) @@ -930,9 +930,9 @@ scalar-group-by │ │ │ └── interesting orderings: (+3,+2,-1) │ │ └── aggregations [outer=(1)] │ │ ├── max [type=int, outer=(1)] - │ │ │ └── variable: new_order.no_o_id [type=int, outer=(1)] + │ │ │ └── variable: no_o_id [type=int, outer=(1)] │ │ ├── min [type=int, outer=(1)] - │ │ │ └── variable: new_order.no_o_id [type=int, outer=(1)] + │ │ │ └── variable: no_o_id [type=int, outer=(1)] │ │ └── count-rows [type=int] │ └── filters [type=bool, outer=(4-6)] │ └── ne [type=bool, outer=(4-6)] @@ -945,7 +945,7 @@ scalar-group-by └── aggregations └── count-rows [type=int] -opt format=show-all +opt format=hide-qual SELECT sum(o_ol_cnt) FROM "order" GROUP BY o_w_id, o_d_id @@ -960,24 +960,24 @@ sort ├── ordering: +3,+2 ├── prune: (9) └── group-by - ├── columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) sum:9(decimal) - ├── grouping columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + ├── columns: o_d_id:2(int!null) o_w_id:3(int!null) sum:9(decimal) + ├── grouping columns: o_d_id:2(int!null) o_w_id:3(int!null) ├── stats: [rows=1000, distinct(2,3)=1000] ├── cost: 1150 ├── key: (2,3) ├── fd: (2,3)-->(9) ├── prune: (9) ├── scan order - │ ├── columns: "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_ol_cnt:7(int) + │ ├── columns: o_d_id:2(int!null) o_w_id:3(int!null) o_ol_cnt:7(int) │ ├── stats: [rows=1000, distinct(2,3)=1000] │ ├── cost: 1110 │ ├── prune: (2,3,7) │ └── interesting orderings: (+3,+2) └── aggregations [outer=(7)] └── sum [type=decimal, outer=(7)] - └── variable: "order".o_ol_cnt [type=int, outer=(7)] + └── variable: o_ol_cnt [type=int, outer=(7)] -opt format=show-all +opt format=hide-qual SELECT count(*) FROM order_line GROUP BY ol_w_id, ol_d_id @@ -992,15 +992,15 @@ sort ├── ordering: +3,+2 ├── prune: (11) └── group-by - ├── columns: order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) count:11(int) - ├── grouping columns: order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) + ├── columns: ol_d_id:2(int!null) ol_w_id:3(int!null) count:11(int) + ├── grouping columns: ol_d_id:2(int!null) ol_w_id:3(int!null) ├── stats: [rows=1000, distinct(2,3)=1000] ├── cost: 1110 ├── key: (2,3) ├── fd: (2,3)-->(11) ├── prune: (11) ├── scan order_line@order_line_fk - │ ├── columns: order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) + │ ├── columns: ol_d_id:2(int!null) ol_w_id:3(int!null) │ ├── stats: [rows=1000, distinct(2,3)=1000] │ ├── cost: 1070 │ ├── prune: (2,3) @@ -1008,33 +1008,33 @@ sort └── aggregations └── count-rows [type=int] -opt format=show-all +opt format=hide-qual (SELECT no_w_id, no_d_id, no_o_id FROM new_order) EXCEPT ALL (SELECT o_w_id, o_d_id, o_id FROM "order" WHERE o_carrier_id IS NULL) ---- except-all ├── columns: no_w_id:3(int!null) no_d_id:2(int!null) no_o_id:1(int!null) - ├── left columns: new_order.no_w_id:3(int!null) new_order.no_d_id:2(int!null) new_order.no_o_id:1(int!null) - ├── right columns: "order".o_w_id:6(int) "order".o_d_id:5(int) "order".o_id:4(int) + ├── left columns: no_w_id:3(int!null) no_d_id:2(int!null) no_o_id:1(int!null) + ├── right columns: o_w_id:6(int) o_d_id:5(int) o_id:4(int) ├── stats: [rows=1000] ├── cost: 2170.02857 ├── scan new_order - │ ├── columns: new_order.no_o_id:1(int!null) new_order.no_d_id:2(int!null) new_order.no_w_id:3(int!null) + │ ├── columns: no_o_id:1(int!null) no_d_id:2(int!null) no_w_id:3(int!null) │ ├── stats: [rows=1000] │ ├── cost: 1060 │ ├── key: (1-3) │ ├── prune: (1-3) │ └── interesting orderings: (+3,+2,-1) └── project - ├── columns: "order".o_id:4(int!null) "order".o_d_id:5(int!null) "order".o_w_id:6(int!null) + ├── columns: o_id:4(int!null) o_d_id:5(int!null) o_w_id:6(int!null) ├── stats: [rows=1.42857143] ├── cost: 1090.01429 ├── key: (4-6) ├── prune: (4-6) ├── interesting orderings: (+6,+5,-4) └── select - ├── columns: "order".o_id:4(int!null) "order".o_d_id:5(int!null) "order".o_w_id:6(int!null) "order".o_carrier_id:9(int) + ├── columns: o_id:4(int!null) o_d_id:5(int!null) o_w_id:6(int!null) o_carrier_id:9(int) ├── stats: [rows=1.42857143, distinct(9)=1] ├── cost: 1090 ├── key: (4-6) @@ -1042,7 +1042,7 @@ except-all ├── prune: (4-6) ├── interesting orderings: (+6,+5,-4) (+6,+5,+9,+4) ├── scan order@order_idx - │ ├── columns: "order".o_id:4(int!null) "order".o_d_id:5(int!null) "order".o_w_id:6(int!null) "order".o_carrier_id:9(int) + │ ├── columns: o_id:4(int!null) o_d_id:5(int!null) o_w_id:6(int!null) o_carrier_id:9(int) │ ├── stats: [rows=1000, distinct(9)=700] │ ├── cost: 1080 │ ├── key: (4-6) @@ -1051,29 +1051,29 @@ except-all │ └── interesting orderings: (+6,+5,-4) (+6,+5,+9,+4) └── filters [type=bool, outer=(9), constraints=(/9: [/NULL - /NULL]; tight), fd=()-->(9)] └── is [type=bool, outer=(9), constraints=(/9: [/NULL - /NULL]; tight)] - ├── variable: "order".o_carrier_id [type=int, outer=(9)] + ├── variable: o_carrier_id [type=int, outer=(9)] └── null [type=unknown] -opt format=show-all +opt format=hide-qual (SELECT o_w_id, o_d_id, o_id FROM "order" WHERE o_carrier_id IS NULL) EXCEPT ALL (SELECT no_w_id, no_d_id, no_o_id FROM new_order) ---- except-all ├── columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) - ├── left columns: "order".o_w_id:3(int!null) "order".o_d_id:2(int!null) "order".o_id:1(int!null) - ├── right columns: new_order.no_w_id:11(int) new_order.no_d_id:10(int) new_order.no_o_id:9(int) + ├── left columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) + ├── right columns: no_w_id:11(int) no_d_id:10(int) no_o_id:9(int) ├── stats: [rows=1.42857143] ├── cost: 2160.04286 ├── project - │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) │ ├── stats: [rows=1.42857143] │ ├── cost: 1090.01429 │ ├── key: (1-3) │ ├── prune: (1-3) │ ├── interesting orderings: (+3,+2,-1) │ └── select - │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_carrier_id:6(int) + │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_carrier_id:6(int) │ ├── stats: [rows=1.42857143, distinct(6)=1] │ ├── cost: 1090 │ ├── key: (1-3) @@ -1081,7 +1081,7 @@ except-all │ ├── prune: (1-3) │ ├── interesting orderings: (+3,+2,-1) (+3,+2,+6,+1) │ ├── scan order@order_idx - │ │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_carrier_id:6(int) + │ │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_carrier_id:6(int) │ │ ├── stats: [rows=1000, distinct(6)=700] │ │ ├── cost: 1080 │ │ ├── key: (1-3) @@ -1090,17 +1090,17 @@ except-all │ │ └── interesting orderings: (+3,+2,-1) (+3,+2,+6,+1) │ └── filters [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] │ └── is [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight)] - │ ├── variable: "order".o_carrier_id [type=int, outer=(6)] + │ ├── variable: o_carrier_id [type=int, outer=(6)] │ └── null [type=unknown] └── scan new_order - ├── columns: new_order.no_o_id:9(int!null) new_order.no_d_id:10(int!null) new_order.no_w_id:11(int!null) + ├── columns: no_o_id:9(int!null) no_d_id:10(int!null) no_w_id:11(int!null) ├── stats: [rows=1000] ├── cost: 1060 ├── key: (9-11) ├── prune: (9-11) └── interesting orderings: (+11,+10,-9) -opt format=show-all +opt format=hide-qual ( SELECT o_w_id, o_d_id, o_id, o_ol_cnt FROM "order" @@ -1116,12 +1116,12 @@ EXCEPT ALL ---- except-all ├── columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) o_ol_cnt:7(int) - ├── left columns: "order".o_w_id:3(int!null) "order".o_d_id:2(int!null) "order".o_id:1(int!null) "order".o_ol_cnt:7(int) - ├── right columns: order_line.ol_w_id:11(int) order_line.ol_d_id:10(int) order_line.ol_o_id:9(int) count:19(int) + ├── left columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) o_ol_cnt:7(int) + ├── right columns: ol_w_id:11(int) ol_d_id:10(int) ol_o_id:9(int) count:19(int) ├── stats: [rows=1000] ├── cost: 2280 ├── scan order - │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_ol_cnt:7(int) + │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_ol_cnt:7(int) │ ├── stats: [rows=1000] │ ├── cost: 1120 │ ├── key: (1-3) @@ -1129,8 +1129,8 @@ except-all │ ├── prune: (1-3,7) │ └── interesting orderings: (+3,+2,-1) └── group-by - ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) count:19(int) - ├── grouping columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) + ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) count:19(int) + ├── grouping columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) ├── stats: [rows=1000, distinct(9-11)=1000] ├── cost: 1130 ├── key: (9-11) @@ -1138,7 +1138,7 @@ except-all ├── prune: (19) ├── interesting orderings: (+11,+10,-9) ├── scan order_line@order_line_fk - │ ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) + │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) │ ├── stats: [rows=1000, distinct(9-11)=1000] │ ├── cost: 1080 │ ├── prune: (9-11) @@ -1146,7 +1146,7 @@ except-all └── aggregations └── count-rows [type=int] -opt format=show-all +opt format=hide-qual ( SELECT ol_w_id, ol_d_id, ol_o_id, count(*) FROM order_line @@ -1162,13 +1162,13 @@ EXCEPT ALL ---- except-all ├── columns: ol_w_id:3(int!null) ol_d_id:2(int!null) ol_o_id:1(int!null) count:11(int) - ├── left columns: order_line.ol_w_id:3(int!null) order_line.ol_d_id:2(int!null) order_line.ol_o_id:1(int!null) count:11(int) - ├── right columns: "order".o_w_id:14(int) "order".o_d_id:13(int) "order".o_id:12(int) "order".o_ol_cnt:18(int) + ├── left columns: ol_w_id:3(int!null) ol_d_id:2(int!null) ol_o_id:1(int!null) count:11(int) + ├── right columns: o_w_id:14(int) o_d_id:13(int) o_id:12(int) o_ol_cnt:18(int) ├── stats: [rows=1000] ├── cost: 2280 ├── group-by - │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) count:11(int) - │ ├── grouping columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) + │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) count:11(int) + │ ├── grouping columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) │ ├── stats: [rows=1000, distinct(1-3)=1000] │ ├── cost: 1130 │ ├── key: (1-3) @@ -1176,7 +1176,7 @@ except-all │ ├── prune: (11) │ ├── interesting orderings: (+3,+2,-1) │ ├── scan order_line@order_line_fk - │ │ ├── columns: order_line.ol_o_id:1(int!null) order_line.ol_d_id:2(int!null) order_line.ol_w_id:3(int!null) + │ │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) │ │ ├── stats: [rows=1000, distinct(1-3)=1000] │ │ ├── cost: 1080 │ │ ├── prune: (1-3) @@ -1184,7 +1184,7 @@ except-all │ └── aggregations │ └── count-rows [type=int] └── scan order - ├── columns: "order".o_id:12(int!null) "order".o_d_id:13(int!null) "order".o_w_id:14(int!null) "order".o_ol_cnt:18(int) + ├── columns: o_id:12(int!null) o_d_id:13(int!null) o_w_id:14(int!null) o_ol_cnt:18(int) ├── stats: [rows=1000] ├── cost: 1120 ├── key: (12-14) @@ -1192,7 +1192,7 @@ except-all ├── prune: (12-14,18) └── interesting orderings: (+14,+13,-12) -opt format=show-all +opt format=hide-qual SELECT count(*) FROM ( @@ -1218,25 +1218,25 @@ scalar-group-by ├── fd: ()-->(19) ├── prune: (19) ├── select - │ ├── columns: "order".o_id:1(int) "order".o_d_id:2(int) "order".o_w_id:3(int) order_line.ol_o_id:9(int) order_line.ol_d_id:10(int) order_line.ol_w_id:11(int) + │ ├── columns: o_id:1(int) o_d_id:2(int) o_w_id:3(int) ol_o_id:9(int) ol_d_id:10(int) ol_w_id:11(int) │ ├── stats: [rows=0.718833144] │ ├── cost: 2240.11456 │ ├── interesting orderings: (+3,+2,-1) (+11,+10,-9) │ ├── full-join - │ │ ├── columns: "order".o_id:1(int) "order".o_d_id:2(int) "order".o_w_id:3(int) order_line.ol_o_id:9(int) order_line.ol_d_id:10(int) order_line.ol_w_id:11(int) + │ │ ├── columns: o_id:1(int) o_d_id:2(int) o_w_id:3(int) ol_o_id:9(int) ol_d_id:10(int) ol_w_id:11(int) │ │ ├── stats: [rows=2.15649943] │ │ ├── cost: 2240.09299 │ │ ├── reject-nulls: (1-3,9-11) │ │ ├── interesting orderings: (+3,+2,-1) (+11,+10,-9) │ │ ├── project - │ │ │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) + │ │ │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) │ │ │ ├── stats: [rows=1.42857143, distinct(1)=1.42813399, distinct(2)=1.42813399, distinct(3)=1.42813399] │ │ │ ├── cost: 1090.01429 │ │ │ ├── key: (1-3) │ │ │ ├── prune: (1-3) │ │ │ ├── interesting orderings: (+3,+2,-1) │ │ │ └── select - │ │ │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_carrier_id:6(int) + │ │ │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_carrier_id:6(int) │ │ │ ├── stats: [rows=1.42857143, distinct(1)=1.42813399, distinct(2)=1.42813399, distinct(3)=1.42813399, distinct(6)=1] │ │ │ ├── cost: 1090 │ │ │ ├── key: (1-3) @@ -1244,7 +1244,7 @@ scalar-group-by │ │ │ ├── prune: (1-3) │ │ │ ├── interesting orderings: (+3,+2,-1) (+3,+2,+6,+1) │ │ │ ├── scan order@order_idx - │ │ │ │ ├── columns: "order".o_id:1(int!null) "order".o_d_id:2(int!null) "order".o_w_id:3(int!null) "order".o_carrier_id:6(int) + │ │ │ │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) o_carrier_id:6(int) │ │ │ │ ├── stats: [rows=1000, distinct(1)=700, distinct(2)=700, distinct(3)=700, distinct(6)=700] │ │ │ │ ├── cost: 1080 │ │ │ │ ├── key: (1-3) @@ -1253,48 +1253,48 @@ scalar-group-by │ │ │ │ └── interesting orderings: (+3,+2,-1) (+3,+2,+6,+1) │ │ │ └── filters [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] │ │ │ └── is [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight)] - │ │ │ ├── variable: "order".o_carrier_id [type=int, outer=(6)] + │ │ │ ├── variable: o_carrier_id [type=int, outer=(6)] │ │ │ └── null [type=unknown] │ │ ├── project - │ │ │ ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) + │ │ │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) │ │ │ ├── stats: [rows=1.42857143, distinct(9)=1.42813399, distinct(10)=1.42813399, distinct(11)=1.42813399] │ │ │ ├── cost: 1150.01429 │ │ │ ├── prune: (9-11) │ │ │ ├── interesting orderings: (+11,+10,-9) │ │ │ └── select - │ │ │ ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) order_line.ol_delivery_d:15(timestamp) + │ │ │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) ol_delivery_d:15(timestamp) │ │ │ ├── stats: [rows=1.42857143, distinct(9)=1.42813399, distinct(10)=1.42813399, distinct(11)=1.42813399, distinct(15)=1] │ │ │ ├── cost: 1150 │ │ │ ├── fd: ()-->(15) │ │ │ ├── prune: (9-11) │ │ │ ├── interesting orderings: (+11,+10,-9) │ │ │ ├── scan order_line - │ │ │ │ ├── columns: order_line.ol_o_id:9(int!null) order_line.ol_d_id:10(int!null) order_line.ol_w_id:11(int!null) order_line.ol_delivery_d:15(timestamp) + │ │ │ │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) ol_delivery_d:15(timestamp) │ │ │ │ ├── stats: [rows=1000, distinct(9)=700, distinct(10)=700, distinct(11)=700, distinct(15)=700] │ │ │ │ ├── cost: 1140 │ │ │ │ ├── prune: (9-11,15) │ │ │ │ └── interesting orderings: (+11,+10,-9) │ │ │ └── filters [type=bool, outer=(15), constraints=(/15: [/NULL - /NULL]; tight), fd=()-->(15)] │ │ │ └── is [type=bool, outer=(15), constraints=(/15: [/NULL - /NULL]; tight)] - │ │ │ ├── variable: order_line.ol_delivery_d [type=timestamp, outer=(15)] + │ │ │ ├── variable: ol_delivery_d [type=timestamp, outer=(15)] │ │ │ └── null [type=unknown] │ │ └── filters [type=bool, outer=(1-3,9-11), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /3: (/NULL - ]; /9: (/NULL - ]; /10: (/NULL - ]; /11: (/NULL - ]), fd=(3)==(11), (11)==(3), (2)==(10), (10)==(2), (1)==(9), (9)==(1)] │ │ ├── eq [type=bool, outer=(3,11), constraints=(/3: (/NULL - ]; /11: (/NULL - ])] - │ │ │ ├── variable: order_line.ol_w_id [type=int, outer=(11)] - │ │ │ └── variable: "order".o_w_id [type=int, outer=(3)] + │ │ │ ├── variable: ol_w_id [type=int, outer=(11)] + │ │ │ └── variable: o_w_id [type=int, outer=(3)] │ │ ├── eq [type=bool, outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ])] - │ │ │ ├── variable: order_line.ol_d_id [type=int, outer=(10)] - │ │ │ └── variable: "order".o_d_id [type=int, outer=(2)] + │ │ │ ├── variable: ol_d_id [type=int, outer=(10)] + │ │ │ └── variable: o_d_id [type=int, outer=(2)] │ │ └── eq [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] - │ │ ├── variable: order_line.ol_o_id [type=int, outer=(9)] - │ │ └── variable: "order".o_id [type=int, outer=(1)] + │ │ ├── variable: ol_o_id [type=int, outer=(9)] + │ │ └── variable: o_id [type=int, outer=(1)] │ └── filters [type=bool, outer=(1,9)] │ └── or [type=bool, outer=(1,9)] │ ├── is [type=bool, outer=(9)] - │ │ ├── variable: order_line.ol_o_id [type=int, outer=(9)] + │ │ ├── variable: ol_o_id [type=int, outer=(9)] │ │ └── null [type=unknown] │ └── is [type=bool, outer=(1)] - │ ├── variable: "order".o_id [type=int, outer=(1)] + │ ├── variable: o_id [type=int, outer=(1)] │ └── null [type=unknown] └── aggregations └── count-rows [type=int] diff --git a/pkg/sql/opt/xform/testdata/external/tpch b/pkg/sql/opt/xform/testdata/external/tpch index c6ee55d74c1e..d0a03e6da293 100644 --- a/pkg/sql/opt/xform/testdata/external/tpch +++ b/pkg/sql/opt/xform/testdata/external/tpch @@ -259,25 +259,25 @@ sort │ │ ├── scan lineitem │ │ │ └── columns: l_quantity:5(float) l_extendedprice:6(float) l_discount:7(float) l_tax:8(float) l_returnflag:9(string) l_linestatus:10(string) l_shipdate:11(date) │ │ └── filters [type=bool, outer=(11), constraints=(/11: (/NULL - ])] - │ │ └── lineitem.l_shipdate <= ('1998-12-01' - '90d') [type=bool, outer=(11), constraints=(/11: (/NULL - ])] + │ │ └── l_shipdate <= ('1998-12-01' - '90d') [type=bool, outer=(11), constraints=(/11: (/NULL - ])] │ └── projections [outer=(5-10)] - │ ├── lineitem.l_extendedprice * (1.0 - lineitem.l_discount) [type=float, outer=(6,7)] - │ └── (lineitem.l_extendedprice * (1.0 - lineitem.l_discount)) * (lineitem.l_tax + 1.0) [type=float, outer=(6-8)] + │ ├── l_extendedprice * (1.0 - l_discount) [type=float, outer=(6,7)] + │ └── (l_extendedprice * (1.0 - l_discount)) * (l_tax + 1.0) [type=float, outer=(6-8)] └── aggregations [outer=(5-7,19,21)] ├── sum [type=float, outer=(5)] - │ └── variable: lineitem.l_quantity [type=float, outer=(5)] + │ └── variable: l_quantity [type=float, outer=(5)] ├── sum [type=float, outer=(6)] - │ └── variable: lineitem.l_extendedprice [type=float, outer=(6)] + │ └── variable: l_extendedprice [type=float, outer=(6)] ├── sum [type=float, outer=(19)] │ └── variable: column19 [type=float, outer=(19)] ├── sum [type=float, outer=(21)] │ └── variable: column21 [type=float, outer=(21)] ├── avg [type=float, outer=(5)] - │ └── variable: lineitem.l_quantity [type=float, outer=(5)] + │ └── variable: l_quantity [type=float, outer=(5)] ├── avg [type=float, outer=(6)] - │ └── variable: lineitem.l_extendedprice [type=float, outer=(6)] + │ └── variable: l_extendedprice [type=float, outer=(6)] ├── avg [type=float, outer=(7)] - │ └── variable: lineitem.l_discount [type=float, outer=(7)] + │ └── variable: l_discount [type=float, outer=(7)] └── count-rows [type=int] # -------------------------------------------------- @@ -456,21 +456,21 @@ project │ │ │ │ │ │ │ │ │ ├── key: (1) │ │ │ │ │ │ │ │ │ └── fd: (1)-->(3,5,6) │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(5,6), constraints=(/6: [/15 - /15]), fd=()-->(6)] - │ │ │ │ │ │ │ │ ├── part.p_size = 15 [type=bool, outer=(6), constraints=(/6: [/15 - /15]; tight)] - │ │ │ │ │ │ │ │ └── part.p_type LIKE '%BRASS' [type=bool, outer=(5)] + │ │ │ │ │ │ │ │ ├── p_size = 15 [type=bool, outer=(6), constraints=(/6: [/15 - /15]; tight)] + │ │ │ │ │ │ │ │ └── p_type LIKE '%BRASS' [type=bool, outer=(5)] │ │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ │ └── filters [type=bool, outer=(1,10,17,18), constraints=(/1: (/NULL - ]; /10: (/NULL - ]; /17: (/NULL - ]; /18: (/NULL - ]), fd=(1)==(17), (17)==(1), (10)==(18), (18)==(10)] - │ │ │ │ │ │ ├── part.p_partkey = partsupp.ps_partkey [type=bool, outer=(1,17), constraints=(/1: (/NULL - ]; /17: (/NULL - ])] + │ │ │ │ │ │ ├── p_partkey = partsupp.ps_partkey [type=bool, outer=(1,17), constraints=(/1: (/NULL - ]; /17: (/NULL - ])] │ │ │ │ │ │ └── supplier.s_suppkey = partsupp.ps_suppkey [type=bool, outer=(10,18), constraints=(/10: (/NULL - ]; /18: (/NULL - ])] │ │ │ │ │ └── filters [type=bool, outer=(1,29), constraints=(/1: (/NULL - ]; /29: (/NULL - ]), fd=(1)==(29), (29)==(1)] - │ │ │ │ │ └── part.p_partkey = partsupp.ps_partkey [type=bool, outer=(1,29), constraints=(/1: (/NULL - ]; /29: (/NULL - ])] + │ │ │ │ │ └── p_partkey = partsupp.ps_partkey [type=bool, outer=(1,29), constraints=(/1: (/NULL - ]; /29: (/NULL - ])] │ │ │ │ └── aggregations [outer=(1,3,11-16,20,32)] │ │ │ │ ├── min [type=float, outer=(32)] │ │ │ │ │ └── variable: partsupp.ps_supplycost [type=float, outer=(32)] │ │ │ │ ├── const-agg [type=float, outer=(20)] │ │ │ │ │ └── variable: partsupp.ps_supplycost [type=float, outer=(20)] │ │ │ │ ├── const-agg [type=string, outer=(3)] - │ │ │ │ │ └── variable: part.p_mfgr [type=string, outer=(3)] + │ │ │ │ │ └── variable: p_mfgr [type=string, outer=(3)] │ │ │ │ ├── const-agg [type=string, outer=(11)] │ │ │ │ │ └── variable: supplier.s_name [type=string, outer=(11)] │ │ │ │ ├── const-agg [type=string, outer=(12)] @@ -484,7 +484,7 @@ project │ │ │ │ ├── const-agg [type=string, outer=(16)] │ │ │ │ │ └── variable: supplier.s_comment [type=string, outer=(16)] │ │ │ │ └── const-agg [type=int, outer=(1)] - │ │ │ │ └── variable: part.p_partkey [type=int, outer=(1)] + │ │ │ │ └── variable: p_partkey [type=int, outer=(1)] │ │ │ └── filters [type=bool, outer=(20,48), constraints=(/20: (/NULL - ]; /48: (/NULL - ]), fd=(20)==(48), (48)==(20)] │ │ │ └── partsupp.ps_supplycost = min [type=bool, outer=(20,48), constraints=(/20: (/NULL - ]; /48: (/NULL - ])] │ │ └── filters [type=bool, outer=(13,22), constraints=(/13: (/NULL - ]; /22: (/NULL - ]), fd=(13)==(22), (22)==(13)] @@ -567,7 +567,7 @@ limit │ │ │ │ │ │ ├── key: (9) │ │ │ │ │ │ └── fd: (9)-->(10,13,16) │ │ │ │ │ └── filters [type=bool, outer=(13), constraints=(/13: (/NULL - /'1995-03-14']; tight)] - │ │ │ │ │ └── orders.o_orderdate < '1995-03-15' [type=bool, outer=(13), constraints=(/13: (/NULL - /'1995-03-14']; tight)] + │ │ │ │ │ └── o_orderdate < '1995-03-15' [type=bool, outer=(13), constraints=(/13: (/NULL - /'1995-03-14']; tight)] │ │ │ │ ├── select │ │ │ │ │ ├── columns: c_custkey:1(int!null) c_mktsegment:7(string!null) │ │ │ │ │ ├── key: (1) @@ -577,21 +577,21 @@ limit │ │ │ │ │ │ ├── key: (1) │ │ │ │ │ │ └── fd: (1)-->(7) │ │ │ │ │ └── filters [type=bool, outer=(7), constraints=(/7: [/'BUILDING' - /'BUILDING']; tight), fd=()-->(7)] - │ │ │ │ │ └── customer.c_mktsegment = 'BUILDING' [type=bool, outer=(7), constraints=(/7: [/'BUILDING' - /'BUILDING']; tight)] + │ │ │ │ │ └── c_mktsegment = 'BUILDING' [type=bool, outer=(7), constraints=(/7: [/'BUILDING' - /'BUILDING']; tight)] │ │ │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ │ │ └── customer.c_custkey = orders.o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(9,18,28), constraints=(/9: (/NULL - ]; /18: (/NULL - ]; /28: [/'1995-03-16' - ]), fd=(9)==(18), (18)==(9)] - │ │ │ ├── lineitem.l_orderkey = orders.o_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ])] - │ │ │ └── lineitem.l_shipdate > '1995-03-15' [type=bool, outer=(28), constraints=(/28: [/'1995-03-16' - ]; tight)] + │ │ │ ├── l_orderkey = o_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ])] + │ │ │ └── l_shipdate > '1995-03-15' [type=bool, outer=(28), constraints=(/28: [/'1995-03-16' - ]; tight)] │ │ └── projections [outer=(13,16,18,23,24)] - │ │ └── lineitem.l_extendedprice * (1.0 - lineitem.l_discount) [type=float, outer=(23,24)] + │ │ └── l_extendedprice * (1.0 - l_discount) [type=float, outer=(23,24)] │ └── aggregations [outer=(13,16,34)] │ ├── sum [type=float, outer=(34)] │ │ └── variable: column34 [type=float, outer=(34)] │ ├── const-agg [type=date, outer=(13)] - │ │ └── variable: orders.o_orderdate [type=date, outer=(13)] + │ │ └── variable: o_orderdate [type=date, outer=(13)] │ └── const-agg [type=int, outer=(16)] - │ └── variable: orders.o_shippriority [type=int, outer=(16)] + │ └── variable: o_shippriority [type=int, outer=(16)] └── const: 10 [type=int] # -------------------------------------------------- @@ -653,8 +653,8 @@ sort │ │ │ ├── fd: (1)-->(5,6) │ │ │ └── ordering: +1 │ │ └── filters [type=bool, outer=(5), constraints=(/5: [/'1993-07-01' - ])] - │ │ ├── orders.o_orderdate >= '1993-07-01' [type=bool, outer=(5), constraints=(/5: [/'1993-07-01' - ]; tight)] - │ │ └── orders.o_orderdate < ('1993-07-01' + '3mon') [type=bool, outer=(5), constraints=(/5: (/NULL - ])] + │ │ ├── o_orderdate >= '1993-07-01' [type=bool, outer=(5), constraints=(/5: [/'1993-07-01' - ]; tight)] + │ │ └── o_orderdate < ('1993-07-01' + '3mon') [type=bool, outer=(5), constraints=(/5: (/NULL - ])] │ ├── select │ │ ├── columns: l_orderkey:10(int!null) l_commitdate:21(date!null) l_receiptdate:22(date!null) │ │ ├── ordering: +10 @@ -662,12 +662,12 @@ sort │ │ │ ├── columns: l_orderkey:10(int!null) l_commitdate:21(date) l_receiptdate:22(date) │ │ │ └── ordering: +10 │ │ └── filters [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] - │ │ └── lineitem.l_commitdate < lineitem.l_receiptdate [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] + │ │ └── l_commitdate < l_receiptdate [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] │ └── merge-on │ ├── left ordering: +1 │ ├── right ordering: +10 │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ └── lineitem.l_orderkey = orders.o_orderkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ └── l_orderkey = o_orderkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] └── aggregations └── count-rows [type=int] @@ -756,22 +756,22 @@ sort │ │ │ │ │ │ │ │ ├── key: (9) │ │ │ │ │ │ │ │ └── fd: (9)-->(10,13) │ │ │ │ │ │ │ └── filters [type=bool, outer=(13), constraints=(/13: [/'1994-01-01' - ])] - │ │ │ │ │ │ │ ├── orders.o_orderdate >= '1994-01-01' [type=bool, outer=(13), constraints=(/13: [/'1994-01-01' - ]; tight)] - │ │ │ │ │ │ │ └── orders.o_orderdate < ('1994-01-01' + '1y') [type=bool, outer=(13), constraints=(/13: (/NULL - ])] + │ │ │ │ │ │ │ ├── o_orderdate >= '1994-01-01' [type=bool, outer=(13), constraints=(/13: [/'1994-01-01' - ]; tight)] + │ │ │ │ │ │ │ └── o_orderdate < ('1994-01-01' + '1y') [type=bool, outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ │ │ │ │ └── customer.c_custkey = orders.o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ │ │ │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ │ │ │ │ └── filters [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ]), fd=(9)==(18), (18)==(9)] - │ │ │ │ │ └── lineitem.l_orderkey = orders.o_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ])] + │ │ │ │ │ └── l_orderkey = o_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ])] │ │ │ │ └── filters [type=bool, outer=(4,20,34,37), constraints=(/4: (/NULL - ]; /20: (/NULL - ]; /34: (/NULL - ]; /37: (/NULL - ]), fd=(20)==(34), (34)==(20), (4)==(37), (37)==(4)] - │ │ │ │ ├── lineitem.l_suppkey = supplier.s_suppkey [type=bool, outer=(20,34), constraints=(/20: (/NULL - ]; /34: (/NULL - ])] - │ │ │ │ └── customer.c_nationkey = supplier.s_nationkey [type=bool, outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ])] + │ │ │ │ ├── l_suppkey = s_suppkey [type=bool, outer=(20,34), constraints=(/20: (/NULL - ]; /34: (/NULL - ])] + │ │ │ │ └── c_nationkey = s_nationkey [type=bool, outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(37,41), constraints=(/37: (/NULL - ]; /41: (/NULL - ]), fd=(37)==(41), (41)==(37)] - │ │ │ └── supplier.s_nationkey = nation.n_nationkey [type=bool, outer=(37,41), constraints=(/37: (/NULL - ]; /41: (/NULL - ])] + │ │ │ └── s_nationkey = n_nationkey [type=bool, outer=(37,41), constraints=(/37: (/NULL - ]; /41: (/NULL - ])] │ │ └── filters [type=bool, outer=(43,45,46), constraints=(/43: (/NULL - ]; /45: (/NULL - ]; /46: [/'ASIA' - /'ASIA']), fd=()-->(46), (43)==(45), (45)==(43)] - │ │ ├── nation.n_regionkey = region.r_regionkey [type=bool, outer=(43,45), constraints=(/43: (/NULL - ]; /45: (/NULL - ])] - │ │ └── region.r_name = 'ASIA' [type=bool, outer=(46), constraints=(/46: [/'ASIA' - /'ASIA']; tight)] + │ │ ├── n_regionkey = r_regionkey [type=bool, outer=(43,45), constraints=(/43: (/NULL - ]; /45: (/NULL - ])] + │ │ └── r_name = 'ASIA' [type=bool, outer=(46), constraints=(/46: [/'ASIA' - /'ASIA']; tight)] │ └── projections [outer=(23,24,42)] - │ └── lineitem.l_extendedprice * (1.0 - lineitem.l_discount) [type=float, outer=(23,24)] + │ └── l_extendedprice * (1.0 - l_discount) [type=float, outer=(23,24)] └── aggregations [outer=(48)] └── sum [type=float, outer=(48)] └── variable: column48 [type=float, outer=(48)] @@ -814,13 +814,13 @@ scalar-group-by │ │ ├── scan lineitem │ │ │ └── columns: l_quantity:5(float) l_extendedprice:6(float) l_discount:7(float) l_shipdate:11(date) │ │ └── filters [type=bool, outer=(5,7,11), constraints=(/5: (/NULL - /23.999999999999996]; /7: [/0.05 - /0.07]; /11: [/'1994-01-01' - ])] - │ │ ├── lineitem.l_shipdate >= '1994-01-01' [type=bool, outer=(11), constraints=(/11: [/'1994-01-01' - ]; tight)] - │ │ ├── lineitem.l_shipdate < ('1994-01-01' + '1y') [type=bool, outer=(11), constraints=(/11: (/NULL - ])] - │ │ ├── lineitem.l_discount >= 0.05 [type=bool, outer=(7), constraints=(/7: [/0.05 - ]; tight)] - │ │ ├── lineitem.l_discount <= 0.07 [type=bool, outer=(7), constraints=(/7: (/NULL - /0.07]; tight)] - │ │ └── lineitem.l_quantity < 24.0 [type=bool, outer=(5), constraints=(/5: (/NULL - /23.999999999999996]; tight)] + │ │ ├── l_shipdate >= '1994-01-01' [type=bool, outer=(11), constraints=(/11: [/'1994-01-01' - ]; tight)] + │ │ ├── l_shipdate < ('1994-01-01' + '1y') [type=bool, outer=(11), constraints=(/11: (/NULL - ])] + │ │ ├── l_discount >= 0.05 [type=bool, outer=(7), constraints=(/7: [/0.05 - ]; tight)] + │ │ ├── l_discount <= 0.07 [type=bool, outer=(7), constraints=(/7: (/NULL - /0.07]; tight)] + │ │ └── l_quantity < 24.0 [type=bool, outer=(5), constraints=(/5: (/NULL - /23.999999999999996]; tight)] │ └── projections [outer=(6,7)] - │ └── lineitem.l_extendedprice * lineitem.l_discount [type=float, outer=(6,7)] + │ └── l_extendedprice * l_discount [type=float, outer=(6,7)] └── aggregations [outer=(17)] └── sum [type=float, outer=(17)] └── variable: column17 [type=float, outer=(17)] @@ -932,22 +932,22 @@ sort │ │ │ │ │ │ │ ├── scan lineitem │ │ │ │ │ │ │ │ └── columns: l_orderkey:8(int!null) l_suppkey:10(int!null) l_extendedprice:13(float) l_discount:14(float) l_shipdate:18(date) │ │ │ │ │ │ │ └── filters [type=bool, outer=(18), constraints=(/18: [/'1995-01-01' - /'1996-12-31']; tight)] - │ │ │ │ │ │ │ ├── lineitem.l_shipdate >= '1995-01-01' [type=bool, outer=(18), constraints=(/18: [/'1995-01-01' - ]; tight)] - │ │ │ │ │ │ │ └── lineitem.l_shipdate <= '1996-12-31' [type=bool, outer=(18), constraints=(/18: (/NULL - /'1996-12-31']; tight)] + │ │ │ │ │ │ │ ├── l_shipdate >= '1995-01-01' [type=bool, outer=(18), constraints=(/18: [/'1995-01-01' - ]; tight)] + │ │ │ │ │ │ │ └── l_shipdate <= '1996-12-31' [type=bool, outer=(18), constraints=(/18: (/NULL - /'1996-12-31']; tight)] │ │ │ │ │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ │ │ │ │ └── supplier.s_suppkey = lineitem.l_suppkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ │ │ │ │ └── s_suppkey = l_suppkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ │ │ │ │ └── filters [type=bool, outer=(8,24), constraints=(/8: (/NULL - ]; /24: (/NULL - ]), fd=(8)==(24), (24)==(8)] - │ │ │ │ │ └── orders.o_orderkey = lineitem.l_orderkey [type=bool, outer=(8,24), constraints=(/8: (/NULL - ]; /24: (/NULL - ])] + │ │ │ │ │ └── o_orderkey = l_orderkey [type=bool, outer=(8,24), constraints=(/8: (/NULL - ]; /24: (/NULL - ])] │ │ │ │ └── filters [type=bool, outer=(25,33), constraints=(/25: (/NULL - ]; /33: (/NULL - ]), fd=(25)==(33), (33)==(25)] - │ │ │ │ └── customer.c_custkey = orders.o_custkey [type=bool, outer=(25,33), constraints=(/25: (/NULL - ]; /33: (/NULL - ])] + │ │ │ │ └── c_custkey = o_custkey [type=bool, outer=(25,33), constraints=(/25: (/NULL - ]; /33: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(4,41), constraints=(/4: (/NULL - ]; /41: (/NULL - ]), fd=(4)==(41), (41)==(4)] - │ │ │ └── supplier.s_nationkey = nation.n_nationkey [type=bool, outer=(4,41), constraints=(/4: (/NULL - ]; /41: (/NULL - ])] + │ │ │ └── s_nationkey = nation.n_nationkey [type=bool, outer=(4,41), constraints=(/4: (/NULL - ]; /41: (/NULL - ])] │ │ └── filters [type=bool, outer=(36,42,45,46), constraints=(/36: (/NULL - ]; /45: (/NULL - ]), fd=(36)==(45), (45)==(36)] - │ │ ├── customer.c_nationkey = nation.n_nationkey [type=bool, outer=(36,45), constraints=(/36: (/NULL - ]; /45: (/NULL - ])] + │ │ ├── c_nationkey = nation.n_nationkey [type=bool, outer=(36,45), constraints=(/36: (/NULL - ]; /45: (/NULL - ])] │ │ └── ((nation.n_name = 'FRANCE') AND (nation.n_name = 'GERMANY')) OR ((nation.n_name = 'GERMANY') AND (nation.n_name = 'FRANACE')) [type=bool, outer=(42,46)] │ └── projections [outer=(13,14,18,42,46)] - │ ├── extract('year', lineitem.l_shipdate) [type=int, outer=(18)] - │ └── lineitem.l_extendedprice * (1.0 - lineitem.l_discount) [type=float, outer=(13,14)] + │ ├── extract('year', l_shipdate) [type=int, outer=(18)] + │ └── l_extendedprice * (1.0 - l_discount) [type=float, outer=(13,14)] └── aggregations [outer=(50)] └── sum [type=float, outer=(50)] └── variable: volume [type=float, outer=(50)] @@ -1067,29 +1067,29 @@ sort │ │ │ │ │ │ │ │ │ │ │ │ ├── key: (1) │ │ │ │ │ │ │ │ │ │ │ │ └── fd: (1)-->(5) │ │ │ │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(5), constraints=(/5: [/'ECONOMY ANODIZED STEEL' - /'ECONOMY ANODIZED STEEL']; tight), fd=()-->(5)] - │ │ │ │ │ │ │ │ │ │ │ └── part.p_type = 'ECONOMY ANODIZED STEEL' [type=bool, outer=(5), constraints=(/5: [/'ECONOMY ANODIZED STEEL' - /'ECONOMY ANODIZED STEEL']; tight)] + │ │ │ │ │ │ │ │ │ │ │ └── p_type = 'ECONOMY ANODIZED STEEL' [type=bool, outer=(5), constraints=(/5: [/'ECONOMY ANODIZED STEEL' - /'ECONOMY ANODIZED STEEL']; tight)] │ │ │ │ │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ │ │ │ │ ├── scan lineitem │ │ │ │ │ │ │ │ │ │ └── columns: l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_extendedprice:22(float) l_discount:23(float) │ │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(1,10,18,19), constraints=(/1: (/NULL - ]; /10: (/NULL - ]; /18: (/NULL - ]; /19: (/NULL - ]), fd=(1)==(18), (18)==(1), (10)==(19), (19)==(10)] - │ │ │ │ │ │ │ │ │ ├── part.p_partkey = lineitem.l_partkey [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ])] - │ │ │ │ │ │ │ │ │ └── supplier.s_suppkey = lineitem.l_suppkey [type=bool, outer=(10,19), constraints=(/10: (/NULL - ]; /19: (/NULL - ])] + │ │ │ │ │ │ │ │ │ ├── p_partkey = l_partkey [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ])] + │ │ │ │ │ │ │ │ │ └── s_suppkey = l_suppkey [type=bool, outer=(10,19), constraints=(/10: (/NULL - ]; /19: (/NULL - ])] │ │ │ │ │ │ │ │ └── filters [type=bool, outer=(17,33,37), constraints=(/17: (/NULL - ]; /33: (/NULL - ]; /37: [/'1995-01-01' - /'1996-12-31']), fd=(17)==(33), (33)==(17)] - │ │ │ │ │ │ │ │ ├── lineitem.l_orderkey = orders.o_orderkey [type=bool, outer=(17,33), constraints=(/17: (/NULL - ]; /33: (/NULL - ])] - │ │ │ │ │ │ │ │ ├── orders.o_orderdate >= '1995-01-01' [type=bool, outer=(37), constraints=(/37: [/'1995-01-01' - ]; tight)] - │ │ │ │ │ │ │ │ └── orders.o_orderdate <= '1996-12-31' [type=bool, outer=(37), constraints=(/37: (/NULL - /'1996-12-31']; tight)] + │ │ │ │ │ │ │ │ ├── l_orderkey = o_orderkey [type=bool, outer=(17,33), constraints=(/17: (/NULL - ]; /33: (/NULL - ])] + │ │ │ │ │ │ │ │ ├── o_orderdate >= '1995-01-01' [type=bool, outer=(37), constraints=(/37: [/'1995-01-01' - ]; tight)] + │ │ │ │ │ │ │ │ └── o_orderdate <= '1996-12-31' [type=bool, outer=(37), constraints=(/37: (/NULL - /'1996-12-31']; tight)] │ │ │ │ │ │ │ └── filters [type=bool, outer=(34,42), constraints=(/34: (/NULL - ]; /42: (/NULL - ]), fd=(34)==(42), (42)==(34)] - │ │ │ │ │ │ │ └── orders.o_custkey = customer.c_custkey [type=bool, outer=(34,42), constraints=(/34: (/NULL - ]; /42: (/NULL - ])] + │ │ │ │ │ │ │ └── o_custkey = c_custkey [type=bool, outer=(34,42), constraints=(/34: (/NULL - ]; /42: (/NULL - ])] │ │ │ │ │ │ └── filters [type=bool, outer=(45,50), constraints=(/45: (/NULL - ]; /50: (/NULL - ]), fd=(45)==(50), (50)==(45)] - │ │ │ │ │ │ └── customer.c_nationkey = nation.n_nationkey [type=bool, outer=(45,50), constraints=(/45: (/NULL - ]; /50: (/NULL - ])] + │ │ │ │ │ │ └── c_nationkey = nation.n_nationkey [type=bool, outer=(45,50), constraints=(/45: (/NULL - ]; /50: (/NULL - ])] │ │ │ │ │ └── filters [type=bool, outer=(13,54), constraints=(/13: (/NULL - ]; /54: (/NULL - ]), fd=(13)==(54), (54)==(13)] - │ │ │ │ │ └── supplier.s_nationkey = nation.n_nationkey [type=bool, outer=(13,54), constraints=(/13: (/NULL - ]; /54: (/NULL - ])] + │ │ │ │ │ └── s_nationkey = nation.n_nationkey [type=bool, outer=(13,54), constraints=(/13: (/NULL - ]; /54: (/NULL - ])] │ │ │ │ └── filters [type=bool, outer=(52,58,59), constraints=(/52: (/NULL - ]; /58: (/NULL - ]; /59: [/'AMERICA' - /'AMERICA']), fd=()-->(59), (52)==(58), (58)==(52)] - │ │ │ │ ├── nation.n_regionkey = region.r_regionkey [type=bool, outer=(52,58), constraints=(/52: (/NULL - ]; /58: (/NULL - ])] - │ │ │ │ └── region.r_name = 'AMERICA' [type=bool, outer=(59), constraints=(/59: [/'AMERICA' - /'AMERICA']; tight)] + │ │ │ │ ├── nation.n_regionkey = r_regionkey [type=bool, outer=(52,58), constraints=(/52: (/NULL - ]; /58: (/NULL - ])] + │ │ │ │ └── r_name = 'AMERICA' [type=bool, outer=(59), constraints=(/59: [/'AMERICA' - /'AMERICA']; tight)] │ │ │ └── projections [outer=(22,23,37,55)] - │ │ │ ├── extract('year', orders.o_orderdate) [type=int, outer=(37)] - │ │ │ └── lineitem.l_extendedprice * (1.0 - lineitem.l_discount) [type=float, outer=(22,23)] + │ │ │ ├── extract('year', o_orderdate) [type=int, outer=(37)] + │ │ │ └── l_extendedprice * (1.0 - l_discount) [type=float, outer=(22,23)] │ │ └── projections [outer=(55,61,62)] │ │ └── CASE WHEN nation.n_name = 'BRAZIL' THEN volume ELSE 0.0 END [type=float, outer=(55,62)] │ └── aggregations [outer=(62,63)] @@ -1200,23 +1200,23 @@ sort │ │ │ │ │ │ │ │ ├── key: (1) │ │ │ │ │ │ │ │ └── fd: (1)-->(2) │ │ │ │ │ │ │ └── filters [type=bool, outer=(2)] - │ │ │ │ │ │ │ └── part.p_name LIKE '%green%' [type=bool, outer=(2)] + │ │ │ │ │ │ │ └── p_name LIKE '%green%' [type=bool, outer=(2)] │ │ │ │ │ │ └── true [type=bool] │ │ │ │ │ ├── scan lineitem │ │ │ │ │ │ └── columns: l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_quantity:21(float) l_extendedprice:22(float) l_discount:23(float) │ │ │ │ │ └── filters [type=bool, outer=(1,10,18,19), constraints=(/1: (/NULL - ]; /10: (/NULL - ]; /18: (/NULL - ]; /19: (/NULL - ]), fd=(10)==(19), (19)==(10), (1)==(18), (18)==(1)] - │ │ │ │ │ ├── supplier.s_suppkey = lineitem.l_suppkey [type=bool, outer=(10,19), constraints=(/10: (/NULL - ]; /19: (/NULL - ])] - │ │ │ │ │ └── part.p_partkey = lineitem.l_partkey [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ])] + │ │ │ │ │ ├── s_suppkey = l_suppkey [type=bool, outer=(10,19), constraints=(/10: (/NULL - ]; /19: (/NULL - ])] + │ │ │ │ │ └── p_partkey = l_partkey [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ])] │ │ │ │ └── filters [type=bool, outer=(18,19,33,34), constraints=(/18: (/NULL - ]; /19: (/NULL - ]; /33: (/NULL - ]; /34: (/NULL - ]), fd=(19)==(34), (34)==(19), (18)==(33), (33)==(18)] - │ │ │ │ ├── partsupp.ps_suppkey = lineitem.l_suppkey [type=bool, outer=(19,34), constraints=(/19: (/NULL - ]; /34: (/NULL - ])] - │ │ │ │ └── partsupp.ps_partkey = lineitem.l_partkey [type=bool, outer=(18,33), constraints=(/18: (/NULL - ]; /33: (/NULL - ])] + │ │ │ │ ├── ps_suppkey = l_suppkey [type=bool, outer=(19,34), constraints=(/19: (/NULL - ]; /34: (/NULL - ])] + │ │ │ │ └── ps_partkey = l_partkey [type=bool, outer=(18,33), constraints=(/18: (/NULL - ]; /33: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(17,38), constraints=(/17: (/NULL - ]; /38: (/NULL - ]), fd=(17)==(38), (38)==(17)] - │ │ │ └── orders.o_orderkey = lineitem.l_orderkey [type=bool, outer=(17,38), constraints=(/17: (/NULL - ]; /38: (/NULL - ])] + │ │ │ └── o_orderkey = l_orderkey [type=bool, outer=(17,38), constraints=(/17: (/NULL - ]; /38: (/NULL - ])] │ │ └── filters [type=bool, outer=(13,47), constraints=(/13: (/NULL - ]; /47: (/NULL - ]), fd=(13)==(47), (47)==(13)] - │ │ └── supplier.s_nationkey = nation.n_nationkey [type=bool, outer=(13,47), constraints=(/13: (/NULL - ]; /47: (/NULL - ])] + │ │ └── s_nationkey = n_nationkey [type=bool, outer=(13,47), constraints=(/13: (/NULL - ]; /47: (/NULL - ])] │ └── projections [outer=(21-23,36,42,48)] - │ ├── extract('year', orders.o_orderdate) [type=int, outer=(42)] - │ └── (lineitem.l_extendedprice * (1.0 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity) [type=float, outer=(21-23,36)] + │ ├── extract('year', o_orderdate) [type=int, outer=(42)] + │ └── (l_extendedprice * (1.0 - l_discount)) - (ps_supplycost * l_quantity) [type=float, outer=(21-23,36)] └── aggregations [outer=(52)] └── sum [type=float, outer=(52)] └── variable: amount [type=float, outer=(52)] @@ -1311,32 +1311,32 @@ limit │ │ │ │ │ │ │ ├── key: (9) │ │ │ │ │ │ │ └── fd: (9)-->(10,13) │ │ │ │ │ │ └── filters [type=bool, outer=(13), constraints=(/13: [/'1993-10-01' - ])] - │ │ │ │ │ │ ├── orders.o_orderdate >= '1993-10-01' [type=bool, outer=(13), constraints=(/13: [/'1993-10-01' - ]; tight)] - │ │ │ │ │ │ └── orders.o_orderdate < ('1993-10-01' + '3mon') [type=bool, outer=(13), constraints=(/13: (/NULL - ])] + │ │ │ │ │ │ ├── o_orderdate >= '1993-10-01' [type=bool, outer=(13), constraints=(/13: [/'1993-10-01' - ]; tight)] + │ │ │ │ │ │ └── o_orderdate < ('1993-10-01' + '3mon') [type=bool, outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ │ │ │ └── customer.c_custkey = orders.o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ │ │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ │ │ │ └── filters [type=bool, outer=(9,18,26), constraints=(/9: (/NULL - ]; /18: (/NULL - ]; /26: [/'R' - /'R']), fd=()-->(26), (9)==(18), (18)==(9)] - │ │ │ │ ├── lineitem.l_orderkey = orders.o_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ])] - │ │ │ │ └── lineitem.l_returnflag = 'R' [type=bool, outer=(26), constraints=(/26: [/'R' - /'R']; tight)] + │ │ │ │ ├── l_orderkey = o_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ])] + │ │ │ │ └── l_returnflag = 'R' [type=bool, outer=(26), constraints=(/26: [/'R' - /'R']; tight)] │ │ │ └── filters [type=bool, outer=(4,34), constraints=(/4: (/NULL - ]; /34: (/NULL - ]), fd=(4)==(34), (34)==(4)] - │ │ │ └── customer.c_nationkey = nation.n_nationkey [type=bool, outer=(4,34), constraints=(/4: (/NULL - ]; /34: (/NULL - ])] + │ │ │ └── c_nationkey = n_nationkey [type=bool, outer=(4,34), constraints=(/4: (/NULL - ]; /34: (/NULL - ])] │ │ └── projections [outer=(1-3,5,6,8,23,24,35)] - │ │ └── lineitem.l_extendedprice * (1.0 - lineitem.l_discount) [type=float, outer=(23,24)] + │ │ └── l_extendedprice * (1.0 - l_discount) [type=float, outer=(23,24)] │ └── aggregations [outer=(2,3,5,6,8,35,38)] │ ├── sum [type=float, outer=(38)] │ │ └── variable: column38 [type=float, outer=(38)] │ ├── const-agg [type=string, outer=(2)] - │ │ └── variable: customer.c_name [type=string, outer=(2)] + │ │ └── variable: c_name [type=string, outer=(2)] │ ├── const-agg [type=string, outer=(3)] - │ │ └── variable: customer.c_address [type=string, outer=(3)] + │ │ └── variable: c_address [type=string, outer=(3)] │ ├── const-agg [type=string, outer=(5)] - │ │ └── variable: customer.c_phone [type=string, outer=(5)] + │ │ └── variable: c_phone [type=string, outer=(5)] │ ├── const-agg [type=float, outer=(6)] - │ │ └── variable: customer.c_acctbal [type=float, outer=(6)] + │ │ └── variable: c_acctbal [type=float, outer=(6)] │ ├── const-agg [type=string, outer=(8)] - │ │ └── variable: customer.c_comment [type=string, outer=(8)] + │ │ └── variable: c_comment [type=string, outer=(8)] │ └── const-agg [type=string, outer=(35)] - │ └── variable: nation.n_name [type=string, outer=(35)] + │ └── variable: n_name [type=string, outer=(35)] └── const: 20 [type=int] # -------------------------------------------------- @@ -1544,16 +1544,16 @@ sort │ │ │ ├── scan lineitem │ │ │ │ └── columns: l_orderkey:10(int!null) l_shipdate:20(date) l_commitdate:21(date) l_receiptdate:22(date) l_shipmode:24(string) │ │ │ └── filters [type=bool, outer=(20-22,24), constraints=(/20: (/NULL - ]; /21: (/NULL - ]; /22: [/'1994-01-01' - ]; /24: [/'MAIL' - /'MAIL'] [/'SHIP' - /'SHIP'])] - │ │ │ ├── lineitem.l_shipmode IN ('MAIL', 'SHIP') [type=bool, outer=(24), constraints=(/24: [/'MAIL' - /'MAIL'] [/'SHIP' - /'SHIP']; tight)] - │ │ │ ├── lineitem.l_commitdate < lineitem.l_receiptdate [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] - │ │ │ ├── lineitem.l_shipdate < lineitem.l_commitdate [type=bool, outer=(20,21), constraints=(/20: (/NULL - ]; /21: (/NULL - ])] - │ │ │ ├── lineitem.l_receiptdate >= '1994-01-01' [type=bool, outer=(22), constraints=(/22: [/'1994-01-01' - ]; tight)] - │ │ │ └── lineitem.l_receiptdate < ('1994-01-01' + '1y') [type=bool, outer=(22), constraints=(/22: (/NULL - ])] + │ │ │ ├── l_shipmode IN ('MAIL', 'SHIP') [type=bool, outer=(24), constraints=(/24: [/'MAIL' - /'MAIL'] [/'SHIP' - /'SHIP']; tight)] + │ │ │ ├── l_commitdate < l_receiptdate [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] + │ │ │ ├── l_shipdate < l_commitdate [type=bool, outer=(20,21), constraints=(/20: (/NULL - ]; /21: (/NULL - ])] + │ │ │ ├── l_receiptdate >= '1994-01-01' [type=bool, outer=(22), constraints=(/22: [/'1994-01-01' - ]; tight)] + │ │ │ └── l_receiptdate < ('1994-01-01' + '1y') [type=bool, outer=(22), constraints=(/22: (/NULL - ])] │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ └── orders.o_orderkey = lineitem.l_orderkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ └── o_orderkey = l_orderkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ └── projections [outer=(6,24)] - │ ├── CASE WHEN (orders.o_orderpriority = '1-URGENT') OR (orders.o_orderpriority = '2-HIGH') THEN 1 ELSE 0 END [type=int, outer=(6)] - │ └── CASE WHEN (orders.o_orderpriority != '1-URGENT') AND (orders.o_orderpriority != '2-HIGH') THEN 1 ELSE 0 END [type=int, outer=(6)] + │ ├── CASE WHEN (o_orderpriority = '1-URGENT') OR (o_orderpriority = '2-HIGH') THEN 1 ELSE 0 END [type=int, outer=(6)] + │ └── CASE WHEN (o_orderpriority != '1-URGENT') AND (o_orderpriority != '2-HIGH') THEN 1 ELSE 0 END [type=int, outer=(6)] └── aggregations [outer=(26,28)] ├── sum [type=decimal, outer=(26)] │ └── variable: column26 [type=int, outer=(26)] @@ -1623,12 +1623,12 @@ sort │ │ │ │ ├── key: (9) │ │ │ │ └── fd: (9)-->(10,17) │ │ │ └── filters [type=bool, outer=(17)] - │ │ │ └── orders.o_comment NOT LIKE '%special%requests%' [type=bool, outer=(17)] + │ │ │ └── o_comment NOT LIKE '%special%requests%' [type=bool, outer=(17)] │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ └── customer.c_custkey = orders.o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ └── aggregations [outer=(9)] │ └── count [type=int, outer=(9)] - │ └── variable: orders.o_orderkey [type=int, outer=(9)] + │ └── variable: o_orderkey [type=int, outer=(9)] └── aggregations └── count-rows [type=int] @@ -1680,13 +1680,13 @@ project │ │ │ │ ├── scan lineitem │ │ │ │ │ └── columns: l_partkey:2(int!null) l_extendedprice:6(float) l_discount:7(float) l_shipdate:11(date) │ │ │ │ └── filters [type=bool, outer=(11), constraints=(/11: [/'1995-09-01' - ])] - │ │ │ │ ├── lineitem.l_shipdate >= '1995-09-01' [type=bool, outer=(11), constraints=(/11: [/'1995-09-01' - ]; tight)] - │ │ │ │ └── lineitem.l_shipdate < ('1995-09-01' + '1mon') [type=bool, outer=(11), constraints=(/11: (/NULL - ])] + │ │ │ │ ├── l_shipdate >= '1995-09-01' [type=bool, outer=(11), constraints=(/11: [/'1995-09-01' - ]; tight)] + │ │ │ │ └── l_shipdate < ('1995-09-01' + '1mon') [type=bool, outer=(11), constraints=(/11: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] - │ │ │ └── lineitem.l_partkey = part.p_partkey [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ])] + │ │ │ └── l_partkey = p_partkey [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ])] │ │ └── projections [outer=(6,7,21)] - │ │ ├── CASE WHEN part.p_type LIKE 'PROMO%' THEN lineitem.l_extendedprice * (1.0 - lineitem.l_discount) ELSE 0.0 END [type=float, outer=(6,7,21)] - │ │ └── lineitem.l_extendedprice * (1.0 - lineitem.l_discount) [type=float, outer=(6,7)] + │ │ ├── CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1.0 - l_discount) ELSE 0.0 END [type=float, outer=(6,7,21)] + │ │ └── l_extendedprice * (1.0 - l_discount) [type=float, outer=(6,7)] │ └── aggregations [outer=(26,28)] │ ├── sum [type=float, outer=(26)] │ │ └── variable: column26 [type=float, outer=(26)] @@ -1812,7 +1812,7 @@ sort │ └── max [type=float, outer=(43)] │ └── variable: sum [type=float, outer=(43)] └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - └── supplier.s_suppkey = lineitem.l_suppkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + └── s_suppkey = lineitem.l_suppkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] # -------------------------------------------------- # Q16 @@ -1889,9 +1889,9 @@ sort │ │ │ │ ├── key: (15) │ │ │ │ └── fd: (15)-->(21) │ │ │ └── filters [type=bool, outer=(21)] - │ │ │ └── supplier.s_comment LIKE '%Customer%Complaints%' [type=bool, outer=(21)] + │ │ │ └── s_comment LIKE '%Customer%Complaints%' [type=bool, outer=(21)] │ │ └── filters [type=bool, outer=(2,15)] - │ │ └── (partsupp.ps_suppkey = supplier.s_suppkey) IS NOT false [type=bool, outer=(2,15)] + │ │ └── (ps_suppkey = s_suppkey) IS NOT false [type=bool, outer=(2,15)] │ ├── select │ │ ├── columns: p_partkey:6(int!null) p_brand:9(string!null) p_type:10(string) p_size:11(int!null) │ │ ├── key: (6) @@ -1901,15 +1901,15 @@ sort │ │ │ ├── key: (6) │ │ │ └── fd: (6)-->(9-11) │ │ └── filters [type=bool, outer=(9-11), constraints=(/9: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; /11: [/3 - /3] [/9 - /9] [/14 - /14] [/19 - /19] [/23 - /23] [/36 - /36] [/45 - /45] [/49 - /49])] - │ │ ├── part.p_brand != 'Brand#45' [type=bool, outer=(9), constraints=(/9: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] - │ │ ├── part.p_type NOT LIKE 'MEDIUM POLISHED %' [type=bool, outer=(10)] - │ │ └── part.p_size IN (3, 9, 14, 19, 23, 36, 45, 49) [type=bool, outer=(11), constraints=(/11: [/3 - /3] [/9 - /9] [/14 - /14] [/19 - /19] [/23 - /23] [/36 - /36] [/45 - /45] [/49 - /49]; tight)] + │ │ ├── p_brand != 'Brand#45' [type=bool, outer=(9), constraints=(/9: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] + │ │ ├── p_type NOT LIKE 'MEDIUM POLISHED %' [type=bool, outer=(10)] + │ │ └── p_size IN (3, 9, 14, 19, 23, 36, 45, 49) [type=bool, outer=(11), constraints=(/11: [/3 - /3] [/9 - /9] [/14 - /14] [/19 - /19] [/23 - /23] [/36 - /36] [/45 - /45] [/49 - /49]; tight)] │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ └── part.p_partkey = partsupp.ps_partkey [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── p_partkey = ps_partkey [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── aggregations [outer=(2)] └── count [type=int, outer=(2)] └── agg-distinct [type=int, outer=(2)] - └── variable: partsupp.ps_suppkey [type=int, outer=(2)] + └── variable: ps_suppkey [type=int, outer=(2)] # -------------------------------------------------- # Q17 @@ -1985,17 +1985,17 @@ project │ │ │ │ │ │ │ ├── key: (17) │ │ │ │ │ │ │ └── fd: (17)-->(20,23) │ │ │ │ │ │ └── filters [type=bool, outer=(20,23), constraints=(/20: [/'Brand#23' - /'Brand#23']; /23: [/'MED BOX' - /'MED BOX']; tight), fd=()-->(20,23)] - │ │ │ │ │ │ ├── part.p_brand = 'Brand#23' [type=bool, outer=(20), constraints=(/20: [/'Brand#23' - /'Brand#23']; tight)] - │ │ │ │ │ │ └── part.p_container = 'MED BOX' [type=bool, outer=(23), constraints=(/23: [/'MED BOX' - /'MED BOX']; tight)] + │ │ │ │ │ │ ├── p_brand = 'Brand#23' [type=bool, outer=(20), constraints=(/20: [/'Brand#23' - /'Brand#23']; tight)] + │ │ │ │ │ │ └── p_container = 'MED BOX' [type=bool, outer=(23), constraints=(/23: [/'MED BOX' - /'MED BOX']; tight)] │ │ │ │ │ └── filters [type=bool, outer=(17,27), constraints=(/17: (/NULL - ]; /27: (/NULL - ]), fd=(17)==(27), (27)==(17)] - │ │ │ │ │ └── lineitem.l_partkey = part.p_partkey [type=bool, outer=(17,27), constraints=(/17: (/NULL - ]; /27: (/NULL - ])] + │ │ │ │ │ └── lineitem.l_partkey = p_partkey [type=bool, outer=(17,27), constraints=(/17: (/NULL - ]; /27: (/NULL - ])] │ │ │ │ └── aggregations [outer=(30)] │ │ │ │ └── avg [type=float, outer=(30)] │ │ │ │ └── variable: lineitem.l_quantity [type=float, outer=(30)] │ │ │ └── projections [outer=(17,42)] │ │ │ └── column42 * 0.2 [type=float, outer=(42)] │ │ └── filters [type=bool, outer=(2,5,17,43), constraints=(/2: (/NULL - ]; /5: (/NULL - ]; /17: (/NULL - ]; /43: (/NULL - ]), fd=(2)==(17), (17)==(2)] - │ │ ├── part.p_partkey = lineitem.l_partkey [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ])] + │ │ ├── p_partkey = lineitem.l_partkey [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ])] │ │ └── lineitem.l_quantity < ?column? [type=bool, outer=(5,43), constraints=(/5: (/NULL - ]; /43: (/NULL - ])] │ └── aggregations [outer=(6)] │ └── sum [type=float, outer=(6)] @@ -2102,24 +2102,24 @@ limit │ │ │ │ │ └── filters [type=bool, outer=(50), constraints=(/50: [/300.00000000000006 - ]; tight)] │ │ │ │ │ └── column50 > 300.0 [type=bool, outer=(50), constraints=(/50: [/300.00000000000006 - ]; tight)] │ │ │ │ └── filters [type=bool, outer=(9,34), constraints=(/9: (/NULL - ]; /34: (/NULL - ]), fd=(9)==(34), (34)==(9)] - │ │ │ │ └── orders.o_orderkey = lineitem.l_orderkey [type=bool, outer=(9,34), constraints=(/9: (/NULL - ]; /34: (/NULL - ])] + │ │ │ │ └── o_orderkey = lineitem.l_orderkey [type=bool, outer=(9,34), constraints=(/9: (/NULL - ]; /34: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ │ └── customer.c_custkey = orders.o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ │ ├── scan lineitem │ │ │ └── columns: lineitem.l_orderkey:18(int!null) lineitem.l_quantity:22(float) │ │ └── filters [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ]), fd=(9)==(18), (18)==(9)] - │ │ └── orders.o_orderkey = lineitem.l_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ])] + │ │ └── o_orderkey = lineitem.l_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ])] │ └── aggregations [outer=(1,2,12,13,22)] │ ├── sum [type=float, outer=(22)] │ │ └── variable: lineitem.l_quantity [type=float, outer=(22)] │ ├── const-agg [type=int, outer=(1)] - │ │ └── variable: customer.c_custkey [type=int, outer=(1)] + │ │ └── variable: c_custkey [type=int, outer=(1)] │ ├── const-agg [type=string, outer=(2)] - │ │ └── variable: customer.c_name [type=string, outer=(2)] + │ │ └── variable: c_name [type=string, outer=(2)] │ ├── const-agg [type=float, outer=(12)] - │ │ └── variable: orders.o_totalprice [type=float, outer=(12)] + │ │ └── variable: o_totalprice [type=float, outer=(12)] │ └── const-agg [type=date, outer=(13)] - │ └── variable: orders.o_orderdate [type=date, outer=(13)] + │ └── variable: o_orderdate [type=date, outer=(13)] └── const: 100 [type=int] # -------------------------------------------------- @@ -2188,14 +2188,14 @@ scalar-group-by │ │ │ ├── scan lineitem │ │ │ │ └── columns: l_partkey:2(int!null) l_quantity:5(float) l_extendedprice:6(float) l_discount:7(float) l_shipinstruct:14(string) l_shipmode:15(string) │ │ │ └── filters [type=bool, outer=(14,15), constraints=(/14: [/'DELIVER IN PERSON' - /'DELIVER IN PERSON']; /15: [/'AIR' - /'AIR'] [/'AIR REG' - /'AIR REG']; tight), fd=()-->(14)] - │ │ │ ├── lineitem.l_shipmode IN ('AIR', 'AIR REG') [type=bool, outer=(15), constraints=(/15: [/'AIR' - /'AIR'] [/'AIR REG' - /'AIR REG']; tight)] - │ │ │ └── lineitem.l_shipinstruct = 'DELIVER IN PERSON' [type=bool, outer=(14), constraints=(/14: [/'DELIVER IN PERSON' - /'DELIVER IN PERSON']; tight)] + │ │ │ ├── l_shipmode IN ('AIR', 'AIR REG') [type=bool, outer=(15), constraints=(/15: [/'AIR' - /'AIR'] [/'AIR REG' - /'AIR REG']; tight)] + │ │ │ └── l_shipinstruct = 'DELIVER IN PERSON' [type=bool, outer=(14), constraints=(/14: [/'DELIVER IN PERSON' - /'DELIVER IN PERSON']; tight)] │ │ └── filters [type=bool, outer=(2,5,17,20,22,23), constraints=(/2: (/NULL - ]; /17: (/NULL - ]; /22: [/1 - ]), fd=(2)==(17), (17)==(2)] - │ │ ├── part.p_partkey = lineitem.l_partkey [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ])] - │ │ ├── ((((((part.p_brand = 'Brand#12') AND (part.p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (lineitem.l_quantity >= 1.0)) AND (lineitem.l_quantity <= 11.0)) AND (part.p_size <= 5)) OR (((((part.p_brand = 'Brand#23') AND (part.p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (lineitem.l_quantity >= 10.0)) AND (lineitem.l_quantity <= 20.0)) AND (part.p_size <= 10))) OR (((((part.p_brand = 'Brand#34') AND (part.p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (lineitem.l_quantity >= 20.0)) AND (lineitem.l_quantity <= 30.0)) AND (part.p_size <= 15)) [type=bool, outer=(5,20,22,23)] - │ │ └── part.p_size >= 1 [type=bool, outer=(22), constraints=(/22: [/1 - ]; tight)] + │ │ ├── p_partkey = l_partkey [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ])] + │ │ ├── ((((((p_brand = 'Brand#12') AND (p_container IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity >= 1.0)) AND (l_quantity <= 11.0)) AND (p_size <= 5)) OR (((((p_brand = 'Brand#23') AND (p_container IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity >= 10.0)) AND (l_quantity <= 20.0)) AND (p_size <= 10))) OR (((((p_brand = 'Brand#34') AND (p_container IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity >= 20.0)) AND (l_quantity <= 30.0)) AND (p_size <= 15)) [type=bool, outer=(5,20,22,23)] + │ │ └── p_size >= 1 [type=bool, outer=(22), constraints=(/22: [/1 - ]; tight)] │ └── projections [outer=(6,7)] - │ └── lineitem.l_extendedprice * (1.0 - lineitem.l_discount) [type=float, outer=(6,7)] + │ └── l_extendedprice * (1.0 - l_discount) [type=float, outer=(6,7)] └── aggregations [outer=(26)] └── sum [type=float, outer=(26)] └── variable: column26 [type=float, outer=(26)] @@ -2297,18 +2297,18 @@ sort │ │ │ │ │ │ ├── scan lineitem │ │ │ │ │ │ │ └── columns: l_partkey:27(int!null) l_suppkey:28(int!null) l_quantity:30(float) l_shipdate:36(date) │ │ │ │ │ │ └── filters [type=bool, outer=(36), constraints=(/36: [/'1994-01-01' - ])] - │ │ │ │ │ │ ├── lineitem.l_shipdate >= '1994-01-01' [type=bool, outer=(36), constraints=(/36: [/'1994-01-01' - ]; tight)] - │ │ │ │ │ │ └── lineitem.l_shipdate < ('1994-01-01' + '1y') [type=bool, outer=(36), constraints=(/36: (/NULL - ])] + │ │ │ │ │ │ ├── l_shipdate >= '1994-01-01' [type=bool, outer=(36), constraints=(/36: [/'1994-01-01' - ]; tight)] + │ │ │ │ │ │ └── l_shipdate < ('1994-01-01' + '1y') [type=bool, outer=(36), constraints=(/36: (/NULL - ])] │ │ │ │ │ └── filters [type=bool, outer=(12,13,27,28), constraints=(/12: (/NULL - ]; /13: (/NULL - ]; /27: (/NULL - ]; /28: (/NULL - ]), fd=(12)==(27), (27)==(12), (13)==(28), (28)==(13)] - │ │ │ │ │ ├── lineitem.l_partkey = partsupp.ps_partkey [type=bool, outer=(12,27), constraints=(/12: (/NULL - ]; /27: (/NULL - ])] - │ │ │ │ │ └── lineitem.l_suppkey = partsupp.ps_suppkey [type=bool, outer=(13,28), constraints=(/13: (/NULL - ]; /28: (/NULL - ])] + │ │ │ │ │ ├── l_partkey = ps_partkey [type=bool, outer=(12,27), constraints=(/12: (/NULL - ]; /27: (/NULL - ])] + │ │ │ │ │ └── l_suppkey = ps_suppkey [type=bool, outer=(13,28), constraints=(/13: (/NULL - ]; /28: (/NULL - ])] │ │ │ │ └── aggregations [outer=(14,30)] │ │ │ │ ├── sum [type=float, outer=(30)] - │ │ │ │ │ └── variable: lineitem.l_quantity [type=float, outer=(30)] + │ │ │ │ │ └── variable: l_quantity [type=float, outer=(30)] │ │ │ │ └── const-agg [type=int, outer=(14)] - │ │ │ │ └── variable: partsupp.ps_availqty [type=int, outer=(14)] + │ │ │ │ └── variable: ps_availqty [type=int, outer=(14)] │ │ │ └── filters [type=bool, outer=(14,42), constraints=(/14: (/NULL - ])] - │ │ │ └── partsupp.ps_availqty > (column42 * 0.5) [type=bool, outer=(14,42), constraints=(/14: (/NULL - ])] + │ │ │ └── ps_availqty > (column42 * 0.5) [type=bool, outer=(14,42), constraints=(/14: (/NULL - ])] │ │ ├── select │ │ │ ├── columns: p_partkey:17(int!null) p_name:18(string) │ │ │ ├── key: (17) @@ -2318,11 +2318,11 @@ sort │ │ │ │ ├── key: (17) │ │ │ │ └── fd: (17)-->(18) │ │ │ └── filters [type=bool, outer=(18)] - │ │ │ └── part.p_name LIKE 'forest%' [type=bool, outer=(18)] + │ │ │ └── p_name LIKE 'forest%' [type=bool, outer=(18)] │ │ └── filters [type=bool, outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ]), fd=(12)==(17), (17)==(12)] - │ │ └── partsupp.ps_partkey = part.p_partkey [type=bool, outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ])] + │ │ └── ps_partkey = p_partkey [type=bool, outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ])] │ └── filters [type=bool, outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] - │ └── supplier.s_suppkey = partsupp.ps_suppkey [type=bool, outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ])] + │ └── s_suppkey = ps_suppkey [type=bool, outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ])] ├── select │ ├── columns: n_nationkey:8(int!null) n_name:9(string!null) │ ├── key: (8) @@ -2332,9 +2332,9 @@ sort │ │ ├── key: (8) │ │ └── fd: (8)-->(9) │ └── filters [type=bool, outer=(9), constraints=(/9: [/'CANADA' - /'CANADA']; tight), fd=()-->(9)] - │ └── nation.n_name = 'CANADA' [type=bool, outer=(9), constraints=(/9: [/'CANADA' - /'CANADA']; tight)] + │ └── n_name = 'CANADA' [type=bool, outer=(9), constraints=(/9: [/'CANADA' - /'CANADA']; tight)] └── filters [type=bool, outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ]), fd=(4)==(8), (8)==(4)] - └── supplier.s_nationkey = nation.n_nationkey [type=bool, outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ])] + └── s_nationkey = n_nationkey [type=bool, outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ])] # -------------------------------------------------- # Q21 @@ -2460,13 +2460,13 @@ limit │ │ │ │ │ ├── lineitem.l_orderkey = lineitem.l_orderkey [type=bool, outer=(8,37), constraints=(/8: (/NULL - ]; /37: (/NULL - ])] │ │ │ │ │ └── lineitem.l_suppkey != lineitem.l_suppkey [type=bool, outer=(10,39), constraints=(/10: (/NULL - ]; /39: (/NULL - ])] │ │ │ │ └── filters [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ │ │ └── supplier.s_suppkey = lineitem.l_suppkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] + │ │ │ │ └── s_suppkey = lineitem.l_suppkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ])] │ │ │ └── filters [type=bool, outer=(8,24,26), constraints=(/8: (/NULL - ]; /24: (/NULL - ]; /26: [/'F' - /'F']), fd=()-->(26), (8)==(24), (24)==(8)] - │ │ │ ├── orders.o_orderkey = lineitem.l_orderkey [type=bool, outer=(8,24), constraints=(/8: (/NULL - ]; /24: (/NULL - ])] - │ │ │ └── orders.o_orderstatus = 'F' [type=bool, outer=(26), constraints=(/26: [/'F' - /'F']; tight)] + │ │ │ ├── o_orderkey = lineitem.l_orderkey [type=bool, outer=(8,24), constraints=(/8: (/NULL - ]; /24: (/NULL - ])] + │ │ │ └── o_orderstatus = 'F' [type=bool, outer=(26), constraints=(/26: [/'F' - /'F']; tight)] │ │ └── filters [type=bool, outer=(4,33,34), constraints=(/4: (/NULL - ]; /33: (/NULL - ]; /34: [/'SAUDI ARABIA' - /'SAUDI ARABIA']), fd=()-->(34), (4)==(33), (33)==(4)] - │ │ ├── supplier.s_nationkey = nation.n_nationkey [type=bool, outer=(4,33), constraints=(/4: (/NULL - ]; /33: (/NULL - ])] - │ │ └── nation.n_name = 'SAUDI ARABIA' [type=bool, outer=(34), constraints=(/34: [/'SAUDI ARABIA' - /'SAUDI ARABIA']; tight)] + │ │ ├── s_nationkey = n_nationkey [type=bool, outer=(4,33), constraints=(/4: (/NULL - ]; /33: (/NULL - ])] + │ │ └── n_name = 'SAUDI ARABIA' [type=bool, outer=(34), constraints=(/34: [/'SAUDI ARABIA' - /'SAUDI ARABIA']; tight)] │ └── aggregations │ └── count-rows [type=int] └── const: 100 [type=int] @@ -2567,7 +2567,7 @@ sort │ │ ├── scan orders │ │ │ └── columns: o_custkey:19(int!null) │ │ └── filters [type=bool, outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ]), fd=(1)==(19), (19)==(1)] - │ │ └── orders.o_custkey = customer.c_custkey [type=bool, outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ])] + │ │ └── o_custkey = customer.c_custkey [type=bool, outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ])] │ └── projections [outer=(5,6)] │ └── substring(customer.c_phone, 1, 2) [type=string, outer=(5)] └── aggregations [outer=(6)] diff --git a/pkg/sql/opt/xform/testdata/physprops/ordering b/pkg/sql/opt/xform/testdata/physprops/ordering index f14225ea0970..a06c5b890296 100644 --- a/pkg/sql/opt/xform/testdata/physprops/ordering +++ b/pkg/sql/opt/xform/testdata/physprops/ordering @@ -80,7 +80,7 @@ select │ ├── columns: x:1(int!null) y:2(float!null) z:3(decimal) s:4(string!null) │ └── ordering: +1,-2 └── filters [type=bool] - └── a.x > a.y [type=bool] + └── x > y [type=bool] # Pass through ordering to scan operator that can't support it. opt @@ -94,7 +94,7 @@ sort ├── scan a │ └── columns: x:1(int!null) y:2(float!null) z:3(decimal) s:4(string!null) └── filters [type=bool] - └── a.x > a.y [type=bool] + └── x > y [type=bool] # -------------------------------------------------- # Project operator (pass through). @@ -111,7 +111,7 @@ project │ ├── columns: x:1(int!null) y:2(float!null) │ └── ordering: +1,-2 └── projections - └── a.x + 1 [type=int] + └── x + 1 [type=int] # Pass through ordering to scan operator that can't support it. opt @@ -125,7 +125,7 @@ sort ├── scan a │ └── columns: x:1(int!null) y:2(float!null) z:3(decimal) └── projections - └── a.z + 1 [type=decimal] + └── z + 1 [type=decimal] # Ordering cannot be passed through because it includes computed column. opt @@ -139,7 +139,7 @@ sort ├── scan a │ └── columns: x:1(int!null) y:2(float!null) └── projections - └── a.y + 1.0 [type=float] + └── y + 1.0 [type=float] # -------------------------------------------------- # Select + Project operators (pass through both). @@ -159,9 +159,9 @@ project │ │ ├── columns: x:1(int!null) y:2(float!null) │ │ └── ordering: +1,-2 │ └── filters [type=bool] - │ └── a.x > a.y [type=bool] + │ └── x > y [type=bool] └── projections - └── a.x - 1 [type=int] + └── x - 1 [type=int] memo SELECT y, x-1 AS z FROM a WHERE x>y ORDER BY x, y DESC @@ -181,7 +181,7 @@ memo (optimized) │ └── "[ordering: +1,-2]" │ ├── best: (select G4="[ordering: +1,-2]" G5) │ └── cost: 1070.00 - ├── G3: (projections G6 a.x a.y) + ├── G3: (projections G6 x y) ├── G4: (scan a,cols=(1,2)) │ ├── "" │ │ ├── best: (scan a,cols=(1,2)) @@ -193,8 +193,8 @@ memo (optimized) ├── G6: (minus G9 G8) ├── G7: (gt G9 G10) ├── G8: (const 1) - ├── G9: (variable a.x) - └── G10: (variable a.y) + ├── G9: (variable x) + └── G10: (variable y) # Pass through ordering to scan operator that can't support it. opt @@ -210,7 +210,7 @@ sort ├── scan a │ └── columns: x:1(int!null) y:2(float!null) z:3(decimal) └── filters [type=bool] - └── a.x > a.y [type=bool] + └── x > y [type=bool] memo SELECT y, z FROM a WHERE x>y ORDER BY y @@ -230,7 +230,7 @@ memo (optimized) │ └── "[ordering: +2]" │ ├── best: (sort G2) │ └── cost: 1096.21 - ├── G3: (projections a.y a.z) + ├── G3: (projections y z) ├── G4: (scan a,cols=(1-3)) │ ├── "" │ │ ├── best: (scan a,cols=(1-3)) @@ -240,8 +240,8 @@ memo (optimized) │ └── cost: 1279.32 ├── G5: (filters G6) ├── G6: (gt G7 G8) - ├── G7: (variable a.x) - └── G8: (variable a.y) + ├── G7: (variable x) + └── G8: (variable y) # -------------------------------------------------- # GroupBy operator. @@ -259,7 +259,7 @@ scalar-group-by │ └── columns: y:2(float!null) z:3(decimal) └── aggregations └── array-agg [type=decimal[]] - └── variable: a.z [type=decimal] + └── variable: z [type=decimal] opt SELECT array_agg(x) FROM (SELECT * FROM a ORDER BY x, y DESC) @@ -272,7 +272,7 @@ scalar-group-by │ └── ordering: +1,-2 └── aggregations └── array-agg [type=int[]] - └── variable: a.x [type=int] + └── variable: x [type=int] # -------------------------------------------------- # Explain operator. @@ -339,7 +339,7 @@ memo (optimized) │ └── "" │ ├── best: (row-number G4) │ └── cost: 1060.00 - ├── G3: (projections G5 a.y) + ├── G3: (projections G5 y) ├── G4: (scan a,cols=(2)) │ └── "" │ ├── best: (scan a,cols=(2)) @@ -461,7 +461,7 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +4 └── filters [type=bool] - └── abc.a = xyz.x [type=bool] + └── a = x [type=bool] opt SELECT * FROM abc JOIN xyz ON a=x ORDER BY x @@ -479,7 +479,7 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +4 └── filters [type=bool] - └── abc.a = xyz.x [type=bool] + └── a = x [type=bool] # A left join guarantees an ordering on the left side. opt @@ -498,7 +498,7 @@ left-join (merge) ├── left ordering: +1 ├── right ordering: +4 └── filters [type=bool] - └── abc.a = xyz.x [type=bool] + └── a = x [type=bool] # A left join doesn't guarantee an ordering on x (some rows will have NULLs). opt @@ -519,7 +519,7 @@ sort ├── left ordering: +1 ├── right ordering: +4 └── filters [type=bool] - └── abc.a = xyz.x [type=bool] + └── a = x [type=bool] # A right join doesn't guarantee an ordering on a (some rows will have NULLs). opt @@ -540,7 +540,7 @@ sort ├── left ordering: +1 ├── right ordering: +4 └── filters [type=bool] - └── abc.a = xyz.x [type=bool] + └── a = x [type=bool] opt SELECT * FROM abc RIGHT JOIN xyz ON a=x ORDER BY x @@ -558,7 +558,7 @@ right-join (merge) ├── left ordering: +1 ├── right ordering: +4 └── filters [type=bool] - └── abc.a = xyz.x [type=bool] + └── a = x [type=bool] opt SELECT * FROM abc FULL OUTER JOIN xyz ON a=x ORDER BY a @@ -578,7 +578,7 @@ sort ├── left ordering: +1 ├── right ordering: +4 └── filters [type=bool] - └── abc.a = xyz.x [type=bool] + └── a = x [type=bool] opt SELECT * FROM abc JOIN xyz ON a=x AND b=y ORDER BY a @@ -596,8 +596,8 @@ inner-join (merge) ├── left ordering: +1,+2 ├── right ordering: +4,+5 └── filters [type=bool] - ├── abc.a = xyz.x [type=bool] - └── abc.b = xyz.y [type=bool] + ├── a = x [type=bool] + └── b = y [type=bool] opt SELECT * FROM abc JOIN xyz ON a=x AND b=y ORDER BY a, b @@ -615,8 +615,8 @@ inner-join (merge) ├── left ordering: +1,+2 ├── right ordering: +4,+5 └── filters [type=bool] - ├── abc.a = xyz.x [type=bool] - └── abc.b = xyz.y [type=bool] + ├── a = x [type=bool] + └── b = y [type=bool] opt SELECT * FROM abc JOIN xyz ON a=x AND b=y ORDER BY a, y @@ -634,8 +634,8 @@ inner-join (merge) ├── left ordering: +1,+2 ├── right ordering: +4,+5 └── filters [type=bool] - ├── abc.a = xyz.x [type=bool] - └── abc.b = xyz.y [type=bool] + ├── a = x [type=bool] + └── b = y [type=bool] # -------------------------------------------------- # Limit / Offset @@ -666,7 +666,7 @@ limit │ │ ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) │ │ └── ordering: +1,+2 │ └── filters [type=bool] - │ └── abc.c < (abc.a + abc.b) [type=bool] + │ └── c < (a + b) [type=bool] └── const: 10 [type=int] opt @@ -711,7 +711,7 @@ sort │ │ ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) │ │ └── ordering: +1,+2 │ └── filters [type=bool] - │ └── abc.c < (abc.a + abc.b) [type=bool] + │ └── c < (a + b) [type=bool] └── const: 10 [type=int] opt @@ -754,7 +754,7 @@ limit │ │ ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) │ │ └── ordering: +1,+2 │ └── filters [type=bool] - │ └── abc.c < (abc.a + abc.b) [type=bool] + │ └── c < (a + b) [type=bool] └── const: 10 [type=int] opt @@ -794,7 +794,7 @@ limit │ │ ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) │ │ └── ordering: +1,+2,+3 │ └── filters [type=bool] - │ └── abc.c < (abc.a + abc.b) [type=bool] + │ └── c < (a + b) [type=bool] └── const: 10 [type=int] opt @@ -851,7 +851,7 @@ sort │ └── columns: a:1(int!null) b:2(int!null) c:3(int!null) └── aggregations └── first-agg [type=int] - └── variable: abc.a [type=int] + └── variable: a [type=int] opt SELECT DISTINCT ON (b, c) a, b, c FROM abc ORDER BY b, c, a @@ -868,7 +868,7 @@ sort │ └── ordering: +1 opt(2,3) └── aggregations └── first-agg [type=int] - └── variable: abc.a [type=int] + └── variable: a [type=int] opt SELECT DISTINCT ON (a) a, c FROM abc ORDER BY a, c DESC, b @@ -885,7 +885,7 @@ distinct-on │ └── columns: a:1(int!null) b:2(int!null) c:3(int!null) └── aggregations └── first-agg [type=int] - └── variable: abc.c [type=int] + └── variable: c [type=int] # Pass through the ordering from above. opt @@ -900,7 +900,7 @@ distinct-on │ └── ordering: +1 └── aggregations └── first-agg [type=int] - └── variable: abc.c [type=int] + └── variable: c [type=int] # Internal orderings that refer just to ON columns can be ignored. opt @@ -915,7 +915,7 @@ distinct-on │ └── ordering: +1,+2 └── aggregations └── first-agg [type=int] - └── variable: abc.c [type=int] + └── variable: c [type=int] opt SELECT * FROM (SELECT DISTINCT ON (a, b) a, b, c FROM abc ORDER BY a, b) ORDER BY a @@ -929,7 +929,7 @@ distinct-on │ └── ordering: +1 └── aggregations └── first-agg [type=int] - └── variable: abc.c [type=int] + └── variable: c [type=int] # The c,b part of the inner ordering can be ignored. opt @@ -945,7 +945,7 @@ distinct-on │ └── ordering: +1 └── aggregations └── first-agg [type=int] - └── variable: abc.a [type=int] + └── variable: a [type=int] # There is no ordering that satisfies both the intra-group ordering of c+ and the # inter-group ordering of a+; we have to sort twice. @@ -966,9 +966,9 @@ sort │ └── columns: a:1(int!null) b:2(int!null) c:3(int!null) └── aggregations ├── first-agg [type=int] - │ └── variable: abc.a [type=int] + │ └── variable: a [type=int] └── first-agg [type=int] - └── variable: abc.c [type=int] + └── variable: c [type=int] # Same as above, except we can use the index ordering for the distinct input. opt @@ -986,6 +986,6 @@ sort │ └── ordering: +2 opt(1) └── aggregations ├── first-agg [type=int] - │ └── variable: abc.b [type=int] + │ └── variable: b [type=int] └── first-agg [type=int] - └── variable: abc.c [type=int] + └── variable: c [type=int] diff --git a/pkg/sql/opt/xform/testdata/physprops/presentation b/pkg/sql/opt/xform/testdata/physprops/presentation index 61552362d7b3..22111a9481fd 100644 --- a/pkg/sql/opt/xform/testdata/physprops/presentation +++ b/pkg/sql/opt/xform/testdata/physprops/presentation @@ -33,7 +33,7 @@ select ├── scan a │ └── columns: x:1(int!null) y:2(int) └── filters [type=bool] - └── a.y = 1 [type=bool] + └── y = 1 [type=bool] # Project operator. opt @@ -44,7 +44,7 @@ project ├── scan a │ └── columns: x:1(int!null) y:2(int) └── projections - └── a.y + 1 [type=int] + └── y + 1 [type=int] # Join operator. opt @@ -69,6 +69,6 @@ group-by │ └── columns: x:1(int!null) y:2(int) └── aggregations ├── max [type=int] - │ └── variable: a.y [type=int] + │ └── variable: y [type=int] └── const-agg [type=int] - └── variable: a.y [type=int] + └── variable: y [type=int] diff --git a/pkg/sql/opt/xform/testdata/ruleprops/orderings b/pkg/sql/opt/xform/testdata/ruleprops/orderings index cc6d20e47eb3..381644fb6563 100644 --- a/pkg/sql/opt/xform/testdata/ruleprops/orderings +++ b/pkg/sql/opt/xform/testdata/ruleprops/orderings @@ -95,7 +95,7 @@ group-by │ └── interesting orderings: (+1,+2) └── aggregations [outer=(2)] └── min [type=int, outer=(2)] - └── variable: abc.b [type=int, outer=(2)] + └── variable: b [type=int, outer=(2)] opt SELECT min(b), c FROM abc GROUP BY c @@ -114,7 +114,7 @@ group-by │ └── interesting orderings: (+3) └── aggregations [outer=(2)] └── min [type=int, outer=(2)] - └── variable: abc.b [type=int, outer=(2)] + └── variable: b [type=int, outer=(2)] # GroupBy with required ordering. opt @@ -138,7 +138,7 @@ group-by │ └── prune: (1-3) └── aggregations [outer=(1)] └── array-agg [type=int[], outer=(1)] - └── variable: abc.a [type=int, outer=(1)] + └── variable: a [type=int, outer=(1)] # Scalar GroupBy case. opt @@ -157,11 +157,11 @@ scalar-group-by │ └── interesting orderings: (+1,+2) (+3) └── aggregations [outer=(1-3)] ├── max [type=int, outer=(1)] - │ └── variable: abc.a [type=int, outer=(1)] + │ └── variable: a [type=int, outer=(1)] ├── min [type=int, outer=(2)] - │ └── variable: abc.b [type=int, outer=(2)] + │ └── variable: b [type=int, outer=(2)] └── sum [type=decimal, outer=(3)] - └── variable: abc.c [type=int, outer=(3)] + └── variable: c [type=int, outer=(3)] # LookupJoin operator. opt @@ -281,4 +281,4 @@ inner-join │ ├── prune: (5-7) │ └── interesting orderings: (+7) (+5,+6) └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] diff --git a/pkg/sql/opt/xform/testdata/rules/combo b/pkg/sql/opt/xform/testdata/rules/combo index 60d145862adc..a56e987a20a1 100644 --- a/pkg/sql/opt/xform/testdata/rules/combo +++ b/pkg/sql/opt/xform/testdata/rules/combo @@ -61,8 +61,8 @@ Source expression: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join @@ -73,8 +73,8 @@ New expression 1 of 1: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] ================================================================================ GenerateIndexScans @@ -88,8 +88,8 @@ Source expression: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join @@ -100,8 +100,8 @@ New expression 1 of 1: ├── scan xyz@xy │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] ================================================================================ CommuteJoin @@ -115,8 +115,8 @@ Source expression: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join @@ -127,8 +127,8 @@ New expression 1 of 1: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] ================================================================================ GenerateMergeJoins @@ -142,8 +142,8 @@ Source expression: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (merge) @@ -159,8 +159,8 @@ New expression 1 of 1: ├── left ordering: +1,+2 ├── right ordering: +5,+6 └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] ================================================================================ GenerateLookupJoin @@ -174,8 +174,8 @@ Source expression: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (lookup xyz@xy) @@ -185,8 +185,8 @@ New expression 1 of 1: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] ================================================================================ CommuteJoin @@ -200,8 +200,8 @@ Source expression: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] No new expressions. @@ -217,8 +217,8 @@ Source expression: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (merge) @@ -234,8 +234,8 @@ New expression 1 of 1: ├── left ordering: +5,+6 ├── right ordering: +1,+2 └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] ================================================================================ GenerateLookupJoin @@ -249,8 +249,8 @@ Source expression: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (lookup abc@ab) @@ -260,7 +260,7 @@ New expression 1 of 1: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] ---- ---- diff --git a/pkg/sql/opt/xform/testdata/rules/groupby b/pkg/sql/opt/xform/testdata/rules/groupby index a5d8edf71115..689224b57400 100644 --- a/pkg/sql/opt/xform/testdata/rules/groupby +++ b/pkg/sql/opt/xform/testdata/rules/groupby @@ -33,7 +33,7 @@ scalar-group-by │ └── fd: ()-->(1) └── aggregations [outer=(1)] └── const-agg [type=string, outer=(1)] - └── variable: abc.a [type=string, outer=(1)] + └── variable: a [type=string, outer=(1)] # Verify the rule still fires even if DISTINCT is used. opt @@ -51,7 +51,7 @@ scalar-group-by │ └── fd: ()-->(1) └── aggregations [outer=(1)] └── const-agg [type=string, outer=(1)] - └── variable: abc.a [type=string, outer=(1)] + └── variable: a [type=string, outer=(1)] opt SELECT min(b) FROM abc @@ -65,7 +65,7 @@ scalar-group-by │ └── columns: b:2(float) └── aggregations [outer=(2)] └── min [type=float, outer=(2)] - └── variable: abc.b [type=float, outer=(2)] + └── variable: b [type=float, outer=(2)] exec-ddl CREATE TABLE xyz ( @@ -113,7 +113,7 @@ scalar-group-by │ └── fd: ()-->(2,3) └── aggregations [outer=(2)] └── const-agg [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # ReplaceMaxWithLimit has the same behavior with max() as # the previous min() query because z is the prefix of a unique key @@ -133,7 +133,7 @@ scalar-group-by │ └── fd: ()-->(2,3) └── aggregations [outer=(2)] └── const-agg [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # We expect ReplaceMinWithLimit not to be preferred here. # This is because we know nothing about the ordering of y @@ -153,7 +153,7 @@ scalar-group-by │ └── fd: (1)-->(2) └── aggregations [outer=(2)] └── min [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # We expect ReplaceMaxWithLimit not to be preferred here. # This is because we know nothing about the ordering of y @@ -173,7 +173,7 @@ scalar-group-by │ └── fd: (1)-->(2) └── aggregations [outer=(2)] └── max [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] opt SELECT max(x) FROM xyz @@ -190,7 +190,7 @@ scalar-group-by │ └── fd: ()-->(1) └── aggregations [outer=(1)] └── const-agg [type=int, outer=(1)] - └── variable: xyz.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] opt SELECT min(x) FROM xyz @@ -207,7 +207,7 @@ scalar-group-by │ └── fd: ()-->(1) └── aggregations [outer=(1)] └── const-agg [type=int, outer=(1)] - └── variable: xyz.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] opt SELECT min(x) FROM xyz WHERE x in (0, 4, 7) @@ -225,7 +225,7 @@ scalar-group-by │ └── fd: ()-->(1) └── aggregations [outer=(1)] └── const-agg [type=int, outer=(1)] - └── variable: xyz.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] opt SELECT max(x) FROM xyz WHERE x in (0, 4, 7) @@ -243,7 +243,7 @@ scalar-group-by │ └── fd: ()-->(1) └── aggregations [outer=(1)] └── const-agg [type=int, outer=(1)] - └── variable: xyz.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] opt SELECT min(y) FROM xyz @@ -261,7 +261,7 @@ scalar-group-by │ └── fd: ()-->(2) └── aggregations [outer=(2)] └── const-agg [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] opt SELECT min(y), min(y) FROM xyz @@ -279,7 +279,7 @@ scalar-group-by │ └── fd: ()-->(2) └── aggregations [outer=(2)] └── const-agg [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # ReplaceMinWithLimit does not apply when there is # a grouping column @@ -297,7 +297,7 @@ project │ └── columns: y:2(int) └── aggregations [outer=(2)] └── min [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # ReplaceMaxWithLimit does not apply when there is # a grouping column @@ -315,7 +315,7 @@ project │ └── columns: y:2(int) └── aggregations [outer=(2)] └── max [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # ReplaceMinWithLimit does not apply when there is # a grouping column @@ -335,7 +335,7 @@ project │ └── fd: (1)-->(2) └── aggregations [outer=(2)] └── min [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # ReplaceMinWithLimit does not apply on multiple aggregations # on different columns @@ -353,9 +353,9 @@ scalar-group-by │ └── fd: (1)-->(2) └── aggregations [outer=(1,2)] ├── min [type=int, outer=(2)] - │ └── variable: xyz.y [type=int, outer=(2)] + │ └── variable: y [type=int, outer=(2)] └── min [type=int, outer=(1)] - └── variable: xyz.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] # ReplaceMaxWithLimit does not apply on multiple aggregations @@ -374,9 +374,9 @@ scalar-group-by │ └── fd: (1)-->(2) └── aggregations [outer=(1,2)] ├── max [type=int, outer=(2)] - │ └── variable: xyz.y [type=int, outer=(2)] + │ └── variable: y [type=int, outer=(2)] └── max [type=int, outer=(1)] - └── variable: xyz.x [type=int, outer=(1)] + └── variable: x [type=int, outer=(1)] # ReplaceMinWithLimit does not apply with # multiple grouping columns @@ -394,7 +394,7 @@ group-by │ └── fd: (1)-->(2) └── aggregations [outer=(2)] └── min [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # ReplaceMaxWithLimit does not apply with # multiple grouping columns @@ -412,7 +412,7 @@ group-by │ └── fd: (1)-->(2) └── aggregations [outer=(2)] └── max [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # ReplaceMinWithLimit does not apply to non-scalar # aggregates @@ -432,9 +432,9 @@ project │ └── fd: (1)-->(2) └── aggregations [outer=(1,2)] ├── min [type=int, outer=(1)] - │ └── variable: xyz.x [type=int, outer=(1)] + │ └── variable: x [type=int, outer=(1)] └── count [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] # ReplaceMaxWithLimit does not apply to non-scalar # aggregates @@ -454,9 +454,9 @@ project │ └── fd: (1)-->(2) └── aggregations [outer=(1,2)] ├── max [type=int, outer=(1)] - │ └── variable: xyz.x [type=int, outer=(1)] + │ └── variable: x [type=int, outer=(1)] └── count [type=int, outer=(2)] - └── variable: xyz.y [type=int, outer=(2)] + └── variable: y [type=int, outer=(2)] memo SELECT min(a) FROM abc @@ -482,7 +482,7 @@ memo (optimized) │ └── cost: 1050.00 ├── G7: (const 1) ├── G8: (const-agg G9) - └── G9: (variable abc.a) + └── G9: (variable a) memo SELECT min(b) FROM abc @@ -517,7 +517,7 @@ memo (optimized) │ └── cost: 1259.32 ├── G10: (filters G11) ├── G11: (is-not G12 G13) - ├── G12: (variable abc.b) + ├── G12: (variable b) └── G13: (null) memo @@ -544,7 +544,7 @@ memo (optimized) │ └── cost: 1149.66 ├── G7: (const 1) ├── G8: (const-agg G9) - └── G9: (variable abc.a) + └── G9: (variable a) memo SELECT max(b) FROM abc @@ -579,7 +579,7 @@ memo (optimized) │ └── cost: 1259.32 ├── G10: (filters G11) ├── G11: (is-not G12 G13) - ├── G12: (variable abc.b) + ├── G12: (variable b) └── G13: (null) # -------------------------------------------------- @@ -601,7 +601,7 @@ scalar-group-by │ └── fd: ()-->(1) └── aggregations [outer=(1)] └── const-agg [type=string, outer=(1)] - └── variable: abc.a [type=string, outer=(1)] + └── variable: a [type=string, outer=(1)] # Verify the rule still fires even if DISTINCT is used. opt @@ -619,7 +619,7 @@ scalar-group-by │ └── fd: ()-->(1) └── aggregations [outer=(1)] └── const-agg [type=string, outer=(1)] - └── variable: abc.a [type=string, outer=(1)] + └── variable: a [type=string, outer=(1)] opt select max(b) FROM abc @@ -633,7 +633,7 @@ scalar-group-by │ └── columns: b:2(float) └── aggregations [outer=(2)] └── max [type=float, outer=(2)] - └── variable: abc.b [type=float, outer=(2)] + └── variable: b [type=float, outer=(2)] memo select max(b) from abc @@ -668,5 +668,5 @@ memo (optimized) │ └── cost: 1259.32 ├── G10: (filters G11) ├── G11: (is-not G12 G13) - ├── G12: (variable abc.b) + ├── G12: (variable b) └── G13: (null) diff --git a/pkg/sql/opt/xform/testdata/rules/join b/pkg/sql/opt/xform/testdata/rules/join index cbda961ebf8d..da2791fc4b6c 100644 --- a/pkg/sql/opt/xform/testdata/rules/join +++ b/pkg/sql/opt/xform/testdata/rules/join @@ -108,8 +108,8 @@ memo (optimized) │ └── cost: 1279.32 ├── G5: (filters G6) ├── G6: (eq G7 G8) - ├── G7: (variable abc.a) - └── G8: (variable xyz.z) + ├── G7: (variable a) + └── G8: (variable z) memo SELECT * FROM abc FULL OUTER JOIN xyz ON a=z @@ -136,8 +136,8 @@ memo (optimized) ├── G4: (merge-on G5 full-join,+1,+7) ├── G5: (filters G6) ├── G6: (eq G7 G8) - ├── G7: (variable abc.a) - └── G8: (variable xyz.z) + ├── G7: (variable a) + └── G8: (variable z) # Verify that we swap to get the smaller side on the right. opt @@ -156,7 +156,7 @@ inner-join │ │ ├── constraint: /2/3/4: (/1/NULL - /1] │ │ └── fd: ()-->(2) │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ └── abc.a = abc.c [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ └── a = c [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] └── true [type=bool] opt @@ -172,7 +172,7 @@ full-join │ ├── constraint: /2/3/4: [/1 - /1] │ └── fd: ()-->(2) └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── abc.a = xyz.z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + └── a = z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] # -------------------------------------------------- # CommuteLeftJoin @@ -203,8 +203,8 @@ memo (optimized) ├── G4: (merge-on G5 left-join,+1,+7) ├── G5: (filters G6) ├── G6: (eq G7 G8) - ├── G7: (variable abc.a) - └── G8: (variable xyz.z) + ├── G7: (variable a) + └── G8: (variable z) opt SELECT * FROM abc LEFT OUTER JOIN xyz ON a=z WHERE b=1 @@ -219,7 +219,7 @@ right-join │ ├── constraint: /2/3/4: [/1 - /1] │ └── fd: ()-->(2) └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── abc.a = xyz.z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + └── a = z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] # -------------------------------------------------- # CommuteRightJoin @@ -250,8 +250,8 @@ memo (optimized) │ └── cost: 1279.32 ├── G5: (filters G6) ├── G6: (eq G7 G8) - ├── G7: (variable abc.a) - └── G8: (variable xyz.z) + ├── G7: (variable a) + └── G8: (variable z) opt SELECT * FROM (SELECT * FROM abc WHERE b=1) RIGHT OUTER JOIN xyz ON a=z @@ -266,7 +266,7 @@ left-join │ ├── constraint: /2/3/4: [/1 - /1] │ └── fd: ()-->(2) └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── abc.a = xyz.z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + └── a = z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] # -------------------------------------------------- # GenerateMergeJoins @@ -288,7 +288,7 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] memo SELECT * FROM abc JOIN xyz ON a=x @@ -316,8 +316,8 @@ memo (optimized) │ └── cost: 1070.00 ├── G6: (filters G7) ├── G7: (eq G8 G9) - ├── G8: (variable abc.a) - └── G9: (variable xyz.x) + ├── G8: (variable a) + └── G9: (variable x) opt SELECT * FROM abc JOIN xyz ON x=a @@ -335,7 +335,7 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── xyz.x = abc.a [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── x = a [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] opt SELECT * FROM abc JOIN xyz ON a=x AND a=x AND x=a @@ -353,9 +353,9 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── xyz.x = abc.a [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── x = a [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] # Use constraints to force the choice of an index which doesn't help, and # verify that we don't prefer a merge-join that has to sort both of its inputs. @@ -374,8 +374,8 @@ inner-join │ ├── constraint: /6/7/8: [/1 - /1] │ └── fd: ()-->(6) └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] # Verify case where we generate multiple merge-joins. memo @@ -580,9 +580,9 @@ left-join (merge) ├── left ordering: +1,+2,+3 ├── right ordering: +6,+5,+4 └── filters [type=bool, outer=(1-6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /3: (/NULL - ]; /4: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1), (2)==(5), (5)==(2), (3)==(4), (4)==(3)] - ├── abc.c = stu.s [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - ├── abc.b = stu.t [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = stu.u [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ])] + ├── c = s [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── b = t [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] + └── a = u [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ])] # The ordering is coming from the right side. opt @@ -603,9 +603,9 @@ left-join (merge) ├── left ordering: +5,+6,+7 ├── right ordering: +3,+2,+1 └── filters [type=bool, outer=(1-3,5-7), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /3: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(5), (5)==(3), (2)==(6), (6)==(2), (1)==(7), (7)==(1)] - ├── abc.c = stu.s [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] - ├── abc.b = stu.t [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] - └── abc.a = stu.u [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + ├── c = s [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] + ├── b = t [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + └── a = u [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] # In these cases, we shouldn't pick up equivalencies. memo @@ -639,8 +639,8 @@ memo (optimized) │ └── cost: 356.67 ├── G8: (filters G9) ├── G9: (eq G10 G11) - ├── G10: (variable abc.a) - └── G11: (variable abc.b) + ├── G10: (variable a) + └── G11: (variable b) exec-ddl CREATE TABLE kfloat (k FLOAT PRIMARY KEY) @@ -668,8 +668,8 @@ memo (optimized) │ └── cost: 1070.00 ├── G4: (filters G5) ├── G5: (eq G6 G7) - ├── G6: (variable abc.a) - └── G7: (variable kfloat.k) + ├── G6: (variable a) + └── G7: (variable k) # We should only pick up one equivalency. opt @@ -688,8 +688,8 @@ inner-join (merge) ├── left ordering: +1 ├── right ordering: +5 └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] # Verify that the required orderings are simplified: the equality columns are # (u,t)=(x,z) but we are able to utilize indexes on s,t,u and y,z. @@ -709,7 +709,7 @@ full-join (merge) │ │ ├── key: (1-3) │ │ └── ordering: +(1|3),+2 │ └── filters [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ └── stu.s = stu.u [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + │ └── s = u [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] ├── select │ ├── columns: x:4(int!null) y:5(int!null) z:6(int) │ ├── fd: (4)==(5), (5)==(4) @@ -719,13 +719,13 @@ full-join (merge) │ │ ├── constraint: /5/6/7: (/NULL - ] │ │ └── ordering: +(4|5),+6 │ └── filters [type=bool, outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ]), fd=(4)==(5), (5)==(4)] - │ └── xyz.x = xyz.y [type=bool, outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ])] + │ └── x = y [type=bool, outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ])] └── merge-on ├── left ordering: +3,+2 ├── right ordering: +4,+6 └── filters [type=bool, outer=(2-4,6), constraints=(/2: (/NULL - ]; /3: (/NULL - ]; /4: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(4), (4)==(3), (2)==(6), (6)==(2)] - ├── stu.u = xyz.x [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ])] - └── stu.t = xyz.z [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── u = x [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ])] + └── t = z [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] # Verify multiple merge-joins can be chained. opt @@ -748,8 +748,8 @@ right-join (merge) │ ├── left ordering: +1,+2 │ ├── right ordering: +5,+6 │ └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - │ ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - │ └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + │ ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + │ └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] ├── scan stu │ ├── columns: s:9(int!null) t:10(int!null) u:11(int!null) │ ├── key: (9-11) @@ -758,7 +758,7 @@ right-join (merge) ├── left ordering: +1 ├── right ordering: +9 └── filters [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] - └── abc.a = stu.s [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] + └── a = s [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ])] opt SELECT * FROM abc JOIN xyz ON a=x AND b=y RIGHT OUTER JOIN stu ON a=u AND y=t @@ -784,14 +784,14 @@ left-join (merge) │ ├── left ordering: +1,+2 │ ├── right ordering: +5,+6 │ └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5), (5)==(1), (2)==(6), (6)==(2)] - │ ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - │ └── abc.b = xyz.y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + │ ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + │ └── b = y [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] └── merge-on ├── left ordering: +11,+10 ├── right ordering: +1,+6 └── filters [type=bool, outer=(1,6,10,11), constraints=(/1: (/NULL - ]; /6: (/NULL - ]; /10: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1), (6)==(10), (10)==(6)] - ├── abc.a = stu.u [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ])] - └── xyz.y = stu.t [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] + ├── a = u [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ])] + └── y = t [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ])] # -------------------------------------------------- # GenerateLookupJoin @@ -814,8 +814,8 @@ Source expression: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (lookup xyz@xy) @@ -825,8 +825,8 @@ New expression 1 of 1: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ================================================================================ GenerateLookupJoin @@ -840,8 +840,8 @@ Source expression: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (lookup abc@ab) @@ -851,8 +851,8 @@ New expression 1 of 1: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ---- ---- @@ -873,7 +873,7 @@ Source expression: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── abc.a = xyz.z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + └── a = z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] New expression 1 of 1: inner-join (lookup abc@ab) @@ -883,7 +883,7 @@ New expression 1 of 1: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── abc.a = xyz.z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + └── a = z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] ---- ---- @@ -902,7 +902,7 @@ Source expression: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── abc.a = xyz.z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + └── a = z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] New expression 1 of 1: left-join (lookup abc@ab) @@ -911,7 +911,7 @@ New expression 1 of 1: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── abc.a = xyz.z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + └── a = z [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] ---- ---- @@ -938,7 +938,7 @@ Source expression: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] - └── abc.c = xyz.x [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] + └── c = x [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] New expression 1 of 1: inner-join (lookup xyz@xy) @@ -948,7 +948,7 @@ New expression 1 of 1: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] - └── abc.c = xyz.x [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] + └── c = x [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] ---- ---- @@ -967,7 +967,7 @@ Source expression: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] - └── abc.c = xyz.x [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] + └── c = x [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] New expression 1 of 1: left-join (lookup xyz@xy) @@ -976,7 +976,7 @@ New expression 1 of 1: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] - └── abc.c = xyz.x [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] + └── c = x [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ])] ---- ---- @@ -1008,8 +1008,8 @@ Source expression: │ ├── columns: x:5(int!null) y:6(int) z:7(int) │ └── constraint: /5/6/8: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (lookup xyz@xy) @@ -1020,9 +1020,9 @@ New expression 1 of 1: │ ├── columns: a:1(int!null) b:2(int) c:3(int) │ └── constraint: /1/2/4: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: [/2 - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── xyz.x > 1 [type=bool, outer=(5), constraints=(/5: [/2 - ]; tight)] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + ├── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x > 1 [type=bool, outer=(5), constraints=(/5: [/2 - ]; tight)] ================================================================================ GenerateLookupJoinWithFilter @@ -1038,8 +1038,8 @@ Source expression: │ ├── columns: a:1(int!null) b:2(int) c:3(int) │ └── constraint: /1/2/4: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (lookup abc@ab) @@ -1050,9 +1050,9 @@ New expression 1 of 1: │ ├── columns: x:5(int!null) y:6(int) z:7(int) │ └── constraint: /5/6/8: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: [/2 - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── abc.a > 1 [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + ├── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── a > 1 [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] ---- ---- @@ -1075,8 +1075,8 @@ Source expression: │ ├── columns: x:5(int!null) y:6(int) z:7(int) │ └── constraint: /5/6/8: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (lookup xyz@xy) @@ -1087,9 +1087,9 @@ New expression 1 of 1: │ ├── columns: a:1(int!null) b:2(int) c:3(int) │ └── constraint: /1/2/4: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: [/2 - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── xyz.x > 1 [type=bool, outer=(5), constraints=(/5: [/2 - ]; tight)] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + ├── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x > 1 [type=bool, outer=(5), constraints=(/5: [/2 - ]; tight)] ================================================================================ GenerateLookupJoinWithFilter @@ -1105,8 +1105,8 @@ Source expression: │ ├── columns: a:1(int!null) b:2(int) c:3(int) │ └── constraint: /1/2/4: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: inner-join (lookup abc@ab) @@ -1117,9 +1117,9 @@ New expression 1 of 1: │ ├── columns: x:5(int!null) y:6(int) z:7(int) │ └── constraint: /5/6/8: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: [/2 - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── abc.a > 1 [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + ├── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── a > 1 [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] ---- ---- @@ -1140,8 +1140,8 @@ Source expression: │ ├── columns: x:5(int!null) y:6(int) z:7(int) │ └── constraint: /5/6/8: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: left-join (lookup xyz@xy) @@ -1150,9 +1150,9 @@ New expression 1 of 1: ├── scan abc │ └── columns: a:1(int) b:2(int) c:3(int) └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: [/2 - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── xyz.x > 1 [type=bool, outer=(5), constraints=(/5: [/2 - ]; tight)] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + ├── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── x > 1 [type=bool, outer=(5), constraints=(/5: [/2 - ]; tight)] ---- ---- @@ -1172,8 +1172,8 @@ Source expression: │ ├── columns: a:1(int!null) b:2(int) c:3(int) │ └── constraint: /1/2/4: [/2 - ] └── filters [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - └── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] New expression 1 of 1: left-join (lookup abc@ab) @@ -1182,9 +1182,9 @@ New expression 1 of 1: ├── scan xyz │ └── columns: x:5(int) y:6(int) z:7(int) └── filters [type=bool, outer=(1,5,6), constraints=(/1: [/2 - ]; /5: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(5,6), (5)==(1,6), (6)==(1,5)] - ├── abc.a = xyz.x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] - ├── abc.a = xyz.y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - └── abc.a > 1 [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] + ├── a = x [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + ├── a = y [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── a > 1 [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] ---- ---- @@ -1229,7 +1229,7 @@ Source expression: │ ├── key: (5) │ └── fd: (5)-->(6,7) └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── abc.a = uvw.u [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + └── a = u [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] New expression 1 of 1: inner-join (lookup uvw) @@ -1246,7 +1246,7 @@ New expression 1 of 1: │ │ ├── key: (5) │ │ └── fd: (5)-->(6) │ └── filters [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── abc.a = uvw.u [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] + │ └── a = u [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ])] └── true [type=bool] ---- ---- @@ -1287,7 +1287,7 @@ inner-join (lookup uvw) │ ├── scan abc │ │ └── columns: a:1(int) b:2(int) c:3(int) │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ └── abc.a = uvw.v [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── a = v [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── true [type=bool] opt @@ -1304,8 +1304,8 @@ inner-join (lookup uvw) │ ├── scan abc │ │ └── columns: a:1(int) b:2(int) c:3(int) │ └── filters [type=bool, outer=(1,2,5,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ ├── abc.a = uvw.v [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - │ └── (abc.b + uvw.u) > 1 [type=bool, outer=(2,5)] + │ ├── a = v [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── (b + u) > 1 [type=bool, outer=(2,5)] └── true [type=bool] opt @@ -1322,6 +1322,6 @@ inner-join (lookup uvw) │ ├── scan abc │ │ └── columns: a:1(int) b:2(int) c:3(int) │ └── filters [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ └── abc.a = uvw.v [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + │ └── a = v [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] └── filters [type=bool, outer=(2,7)] - └── (abc.b + uvw.w) > 1 [type=bool, outer=(2,7)] + └── (b + w) > 1 [type=bool, outer=(2,7)] diff --git a/pkg/sql/opt/xform/testdata/rules/limit b/pkg/sql/opt/xform/testdata/rules/limit index 4103c2e59ae2..834ae08f111e 100644 --- a/pkg/sql/opt/xform/testdata/rules/limit +++ b/pkg/sql/opt/xform/testdata/rules/limit @@ -133,7 +133,7 @@ memo (optimized) │ └── cost: 1050.00 ├── G5: (filters G6) ├── G6: (eq G7 G8) - ├── G7: (variable a.s) + ├── G7: (variable s) └── G8: (const 'foo') # -------------------------------------------------- diff --git a/pkg/sql/opt/xform/testdata/rules/scan b/pkg/sql/opt/xform/testdata/rules/scan index 738db6b190e6..ea5087f2b583 100644 --- a/pkg/sql/opt/xform/testdata/rules/scan +++ b/pkg/sql/opt/xform/testdata/rules/scan @@ -342,7 +342,7 @@ memo (optimized) │ └── "" │ ├── best: (scan abc,cols=(4)) │ └── cost: 1050.00 - ├── G3: (projections G6 abc.d) + ├── G3: (projections G6 d) ├── G4: (scan abc@bc,cols=(1-3)) │ └── "" │ ├── best: (scan abc@bc,cols=(1-3)) @@ -352,7 +352,7 @@ memo (optimized) │ ├── best: (scan abc@ba,cols=(1-3)) │ └── cost: 1060.00 ├── G6: (function G7 lower) - └── G7: (variable abc.d) + └── G7: (variable d) memo SELECT j FROM a WHERE s = 'foo' @@ -366,7 +366,7 @@ memo (optimized) │ └── "" │ ├── best: (scan a@si_idx,cols=(4,5),constrained) │ └── cost: 1.51 - ├── G3: (projections a.j) + ├── G3: (projections j) ├── G4: (scan a,cols=(4,5)) (index-join G7 a,cols=(4,5)) (scan a@si_idx,cols=(4,5)) │ └── "" │ ├── best: (scan a@si_idx,cols=(4,5)) @@ -385,7 +385,7 @@ memo (optimized) │ └── cost: 1060.00 ├── G8: (filters G9) ├── G9: (eq G10 G11) - ├── G10: (variable a.s) + ├── G10: (variable s) └── G11: (const 'foo') # Scan of primary index is lowest cost. @@ -482,12 +482,12 @@ memo (optimized) │ └── "" │ ├── best: (scan a@s_idx,cols=(1,2,4),constrained) │ └── cost: 356.67 - ├── G3: (projections a.k a.i) + ├── G3: (projections k i) ├── G4: (scan a,cols=(1,2,4)) (scan a@s_idx,cols=(1,2,4)) (scan a@si_idx,cols=(1,2,4)) │ └── "" │ ├── best: (scan a@s_idx,cols=(1,2,4)) │ └── cost: 1070.00 ├── G5: (filters G6) ├── G6: (ge G7 G8) - ├── G7: (variable a.s) + ├── G7: (variable s) └── G8: (const 'foo') diff --git a/pkg/sql/opt/xform/testdata/rules/select b/pkg/sql/opt/xform/testdata/rules/select index 4e1742be49aa..246ccf9c5bea 100644 --- a/pkg/sql/opt/xform/testdata/rules/select +++ b/pkg/sql/opt/xform/testdata/rules/select @@ -80,7 +80,7 @@ memo (optimized) │ └── cost: 1040.00 ├── G3: (filters G4) ├── G4: (eq G5 G6) - ├── G5: (variable a.k) + ├── G5: (variable k) └── G6: (const 1) opt @@ -107,14 +107,14 @@ memo (optimized) │ └── "" │ ├── best: (scan a@v,cols=(1,3),constrained) │ └── cost: 350.00 - ├── G3: (projections a.k) + ├── G3: (projections k) ├── G4: (scan a,cols=(1,3)) (scan a@u,cols=(1,3)) (scan a@v,cols=(1,3)) │ └── "" │ ├── best: (scan a,cols=(1,3)) │ └── cost: 1050.00 ├── G5: (filters G6) ├── G6: (gt G7 G8) - ├── G7: (variable a.v) + ├── G7: (variable v) └── G8: (const 1) opt @@ -144,7 +144,7 @@ memo (optimized) │ └── "" │ ├── best: (scan a@u,cols=(1,2),constrained) │ └── cost: 1.05 - ├── G3: (projections a.k) + ├── G3: (projections k) ├── G4: (scan a,cols=(1,2)) (scan a@u,cols=(1,2)) (scan a@v,cols=(1,2)) │ └── "" │ ├── best: (scan a,cols=(1,2)) @@ -157,9 +157,9 @@ memo (optimized) ├── G7: (filters G9) ├── G8: (eq G10 G11) ├── G9: (eq G12 G13) - ├── G10: (variable a.k) + ├── G10: (variable k) ├── G11: (const 5) - ├── G12: (variable a.u) + ├── G12: (variable u) └── G13: (const 1) # Constraint + remaining filter. @@ -179,7 +179,7 @@ project │ ├── key: (1) │ └── fd: ()-->(2) └── filters [type=bool, outer=(1,2)] - └── (a.k + a.u) = 1 [type=bool, outer=(1,2)] + └── (k + u) = 1 [type=bool, outer=(1,2)] memo SELECT k FROM a WHERE u = 1 AND k+u = 1 @@ -193,7 +193,7 @@ memo (optimized) │ └── "" │ ├── best: (select G6 G7) │ └── cost: 1.51 - ├── G3: (projections a.k) + ├── G3: (projections k) ├── G4: (scan a,cols=(1,2)) (scan a@u,cols=(1,2)) (scan a@v,cols=(1,2)) │ └── "" │ ├── best: (scan a,cols=(1,2)) @@ -208,8 +208,8 @@ memo (optimized) ├── G9: (eq G10 G11) ├── G10: (plus G12 G13) ├── G11: (const 1) - ├── G12: (variable a.k) - └── G13: (variable a.u) + ├── G12: (variable k) + └── G13: (variable u) opt SELECT k FROM a WHERE u = 1 AND v = 5 @@ -231,7 +231,7 @@ project │ ├── key: () │ └── fd: ()-->(1-3) └── filters [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] - └── a.u = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + └── u = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] memo SELECT k FROM a WHERE u = 1 AND v = 5 @@ -245,7 +245,7 @@ memo (optimized) │ └── "" │ ├── best: (select G8 G9) │ └── cost: 1.07 - ├── G3: (projections a.k) + ├── G3: (projections k) ├── G4: (scan a) (scan a@u) (scan a@v) │ └── "" │ ├── best: (scan a) @@ -263,9 +263,9 @@ memo (optimized) ├── G9: (filters G11) ├── G10: (eq G12 G13) ├── G11: (eq G14 G15) - ├── G12: (variable a.v) + ├── G12: (variable v) ├── G13: (const 5) - ├── G14: (variable a.u) + ├── G14: (variable u) └── G15: (const 1) # Only not-null constraint is pushed down. @@ -285,7 +285,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2,3), (3)~~>(1,2) └── filters [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] - └── a.u = a.v [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] + └── u = v [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] # Don't push constraint into already limited scan. opt @@ -307,7 +307,7 @@ project │ ├── key: () │ └── fd: ()-->(1,2) └── filters [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] - └── a.k = 1 [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] + └── k = 1 [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight)] # -------------------------------------------------- # PushFilterIntoLookupJoinNoRemainder @@ -358,7 +358,7 @@ memo (optimized) ├── G8: (ge G11 G10) ├── G9: (le G11 G12) ├── G10: (const 1) - ├── G11: (variable b.v) + ├── G11: (variable v) └── G12: (const 10) # Don't choose lookup join if it's not beneficial. @@ -374,7 +374,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-4), (3)~~>(1,2,4) └── filters [type=bool, outer=(3), constraints=(/3: [/2 - ]; tight)] - └── b.v > 1 [type=bool, outer=(3), constraints=(/3: [/2 - ]; tight)] + └── v > 1 [type=bool, outer=(3), constraints=(/3: [/2 - ]; tight)] opt SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k > 5 @@ -393,7 +393,7 @@ index-join b │ ├── key: (1) │ └── fd: (1)-->(3), (3)-->(1) └── filters [type=bool, outer=(1), constraints=(/1: [/6 - ]; tight)] - └── b.k > 5 [type=bool, outer=(1), constraints=(/1: [/6 - ]; tight)] + └── k > 5 [type=bool, outer=(1), constraints=(/1: [/6 - ]; tight)] memo SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k > 5 @@ -453,10 +453,10 @@ memo (optimized) │ └── cost: 1040.00 ├── G17: (filters G21) ├── G18: (const 1) - ├── G19: (variable b.v) + ├── G19: (variable v) ├── G20: (const 10) ├── G21: (gt G22 G23) - ├── G22: (variable b.k) + ├── G22: (variable k) └── G23: (const 5) # Ensure the rule doesn't match at all when the first column of the index is @@ -481,7 +481,7 @@ Source expression: │ ├── key: (1) │ └── fd: (1)-->(2) └── filters [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] - └── a.u = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] + └── u = 1 [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight)] New expression 1 of 1: project @@ -517,7 +517,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(3), (3)-->(1) └── filters [type=bool, outer=(1,2)] - └── (b.k + b.u) = 1 [type=bool, outer=(1,2)] + └── (k + u) = 1 [type=bool, outer=(1,2)] memo SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k+u = 1 @@ -571,10 +571,10 @@ memo (optimized) ├── G16: (le G19 G20) ├── G17: (plus G21 G22) ├── G18: (const 1) - ├── G19: (variable b.v) + ├── G19: (variable v) ├── G20: (const 10) - ├── G21: (variable b.k) - └── G22: (variable b.u) + ├── G21: (variable k) + └── G22: (variable u) opt SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k+u = 1 AND k > 5 @@ -597,9 +597,9 @@ select │ │ ├── key: (1) │ │ └── fd: (1)-->(3), (3)-->(1) │ └── filters [type=bool, outer=(1), constraints=(/1: [/6 - ]; tight)] - │ └── b.k > 5 [type=bool, outer=(1), constraints=(/1: [/6 - ]; tight)] + │ └── k > 5 [type=bool, outer=(1), constraints=(/1: [/6 - ]; tight)] └── filters [type=bool, outer=(1,2)] - └── (b.k + b.u) = 1 [type=bool, outer=(1,2)] + └── (k + u) = 1 [type=bool, outer=(1,2)] memo SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k+u = 1 AND k > 5 @@ -674,11 +674,11 @@ memo (optimized) ├── G25: (gt G31 G30) ├── G26: (plus G31 G32) ├── G27: (const 1) - ├── G28: (variable b.v) + ├── G28: (variable v) ├── G29: (const 10) ├── G30: (const 5) - ├── G31: (variable b.k) - └── G32: (variable b.u) + ├── G31: (variable k) + └── G32: (variable u) # -------------------------------------------------- # ConstrainLookupJoinIndexScan @@ -701,8 +701,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2) └── filters [type=bool, outer=(1-3), constraints=(/2/1/3: [/1/2/4 - /8/9/9]; tight)] - ├── (b.u, b.k, b.v) > (1, 2, 3) [type=bool, outer=(1-3), constraints=(/2/1/3: [/1/2/4 - ]; tight)] - └── (b.u, b.k, b.v) < (8, 9, 10) [type=bool, outer=(1-3), constraints=(/2/1/3: (/NULL - /8/9/9]; tight)] + ├── (u, k, v) > (1, 2, 3) [type=bool, outer=(1-3), constraints=(/2/1/3: [/1/2/4 - ]; tight)] + └── (u, k, v) < (8, 9, 10) [type=bool, outer=(1-3), constraints=(/2/1/3: (/NULL - /8/9/9]; tight)] memo SELECT * FROM b WHERE (u, k, v) > (1, 2, 3) AND (u, k, v) < (8, 9, 10) @@ -741,9 +741,9 @@ memo (optimized) ├── G13: (const 1) ├── G14: (const 2) ├── G15: (const 3) - ├── G16: (variable b.u) - ├── G17: (variable b.k) - ├── G18: (variable b.v) + ├── G16: (variable u) + ├── G17: (variable k) + ├── G18: (variable v) ├── G19: (const 8) ├── G20: (const 9) └── G21: (const 10) @@ -788,7 +788,7 @@ project │ ├── constraint: /4/1: [/'{"a": "b"}' - /'{"a": "b"}'] │ └── key: (1) └── filters [type=bool, outer=(4)] - └── b.j @> '{"a": "b", "c": "d"}' [type=bool, outer=(4)] + └── j @> '{"a": "b", "c": "d"}' [type=bool, outer=(4)] # Query requiring an index join with no remaining filter. opt @@ -849,7 +849,7 @@ select │ ├── constraint: /4/1: [/'{"a": "b"}' - /'{"a": "b"}'] │ └── key: (1) └── filters [type=bool, outer=(4)] - └── b.j @> '{"a": "b", "c": "d"}' [type=bool, outer=(4)] + └── j @> '{"a": "b", "c": "d"}' [type=bool, outer=(4)] opt SELECT * FROM b WHERE j @> '{"a": {"b": "c", "d": "e"}, "f": "g"}' @@ -867,4 +867,4 @@ select │ ├── constraint: /4/1: [/'{"a": {"b": "c"}}' - /'{"a": {"b": "c"}}'] │ └── key: (1) └── filters [type=bool, outer=(4)] - └── b.j @> '{"a": {"b": "c", "d": "e"}, "f": "g"}' [type=bool, outer=(4)] + └── j @> '{"a": {"b": "c", "d": "e"}, "f": "g"}' [type=bool, outer=(4)] diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index 4c27b4f2f1a5..22888ce71c38 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -2694,8 +2694,8 @@ func (desc *ColumnDescriptor) IsNullable() bool { } // ColName is part of the opt.Column interface. -func (desc *ColumnDescriptor) ColName() opt.ColumnName { - return opt.ColumnName(desc.Name) +func (desc *ColumnDescriptor) ColName() tree.Name { + return tree.Name(desc.Name) } // DatumType is part of the opt.Column interface. diff --git a/pkg/sql/sqlbase/system.go b/pkg/sql/sqlbase/system.go index dad69b7edacc..73312650929d 100644 --- a/pkg/sql/sqlbase/system.go +++ b/pkg/sql/sqlbase/system.go @@ -128,7 +128,7 @@ CREATE TABLE system.jobs ( // Design outlined in /docs/RFCS/web_session_login.rfc WebSessionsTableSchema = ` CREATE TABLE system.web_sessions ( - id SERIAL PRIMARY KEY, + id INT NOT NULL DEFAULT unique_rowid() PRIMARY KEY, "hashedSecret" BYTES NOT NULL, username STRING NOT NULL, "createdAt" TIMESTAMP NOT NULL DEFAULT now(), diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index 4ca9f7821e83..2a65a44df8ec 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -115,7 +115,8 @@ func declareKeysEndTransaction( } if mt := et.InternalCommitTrigger.MergeTrigger; mt != nil { // Merges write to the left side's abort span and the right side's data - // and range-local spans. + // and range-local spans. They also read from the right side's range ID + // span. leftRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(header.RangeID) spans.Add(spanset.SpanReadWrite, roachpb.Span{ Key: leftRangeIDPrefix, @@ -129,6 +130,10 @@ func declareKeysEndTransaction( Key: keys.MakeRangeKeyPrefix(mt.RightDesc.StartKey), EndKey: keys.MakeRangeKeyPrefix(mt.RightDesc.EndKey), }) + spans.Add(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID), + EndKey: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID).PrefixEnd(), + }) } } } @@ -992,51 +997,31 @@ func mergeTrigger( desc.EndKey, merge.LeftDesc.EndKey) } - // Create a scratch engine to rewrite the RHS data. - // - // TODO(benesch): the cache size may need to be tuned. - eng := engine.NewInMem(roachpb.Attributes{}, 1<<20) - defer eng.Close() - - // Load the data from the RHS. - if err := eng.ApplyBatchRepr(merge.RightData, false /* sync */); err != nil { - return result.Result{}, err - } - if err := abortspan.New(merge.RightDesc.RangeID).CopyTo( - ctx, eng, batch, ms, ts, merge.LeftDesc.RangeID, + ctx, batch, batch, ms, ts, merge.LeftDesc.RangeID, ); err != nil { return result.Result{}, err } - // Copy the RHS data into the command's batch. We skip over the range-ID local - // keys. The abort span is the only relevant part of the range-ID local - // keyspace, and we already copied it above. - rhsRelevantStartKey := engine.MakeMVCCMetadataKey(keys.MakeRangeKeyPrefix(merge.RightDesc.StartKey)) - iter := eng.NewIterator(engine.IterOptions{ - UpperBound: roachpb.KeyMax, // all the data in this engine is relevant - }) - defer iter.Close() - for iter.Seek(rhsRelevantStartKey); ; iter.Next() { - if ok, err := iter.Valid(); err != nil { - return result.Result{}, err - } else if !ok { - break - } - if err := batch.Put(iter.UnsafeKey(), iter.UnsafeValue()); err != nil { + // The stats for the merged range are the sum of the LHS and RHS stats, less + // the RHS's replicated range ID stats. The only replicated range ID keys we + // copy from the RHS are the keys in the abort span, and we've already + // accounted for those stats above. + ms.Add(merge.RightMVCCStats) + { + ridPrefix := keys.MakeRangeIDReplicatedPrefix(merge.RightDesc.RangeID) + iter := batch.NewIterator(engine.IterOptions{UpperBound: ridPrefix.PrefixEnd()}) + defer iter.Close() + sysMS, err := iter.ComputeStats( + engine.MakeMVCCMetadataKey(ridPrefix), + engine.MakeMVCCMetadataKey(ridPrefix.PrefixEnd()), + 0 /* nowNanos */) + if err != nil { return result.Result{}, err } + ms.Subtract(sysMS) } - // Adjust stats for the rewritten RHS data. Again, we skip over the range-ID - // local keys, as only the abort span is relevant and its stats were accounted - // for above. - rhsMS, err := iter.ComputeStats(rhsRelevantStartKey, engine.MVCCKeyMax, 0 /* nowNanos */) - if err != nil { - return result.Result{}, err - } - ms.Add(rhsMS) - var pd result.Result pd.Replicated.BlockReads = true pd.Replicated.Merge = &storagebase.Merge{ diff --git a/pkg/storage/batcheval/cmd_get_snapshot_for_merge.go b/pkg/storage/batcheval/cmd_get_snapshot_for_merge.go index e26bd4200883..3d02240d8f06 100644 --- a/pkg/storage/batcheval/cmd_get_snapshot_for_merge.go +++ b/pkg/storage/batcheval/cmd_get_snapshot_for_merge.go @@ -19,7 +19,6 @@ import ( "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/storage/rditer" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/pkg/errors" @@ -124,28 +123,7 @@ func GetSnapshotForMerge( // but the check is too expensive as it would involve a network roundtrip on // most nodes. - eng := engine.NewInMem(roachpb.Attributes{}, 1<<20) - defer eng.Close() - - // TODO(benesch): This command reads the whole replica into memory. We'll need - // to be more careful when merging large ranges. - snapBatch := eng.NewBatch() - defer snapBatch.Close() - - iter := rditer.NewReplicaDataIterator(desc, batch, true /* replicatedOnly */) - defer iter.Close() - for ; ; iter.Next() { - if ok, err := iter.Valid(); err != nil { - return result.Result{}, err - } else if !ok { - break - } - if err := snapBatch.Put(iter.Key(), iter.Value()); err != nil { - return result.Result{}, err - } - } - reply.Data = snapBatch.Repr() - + reply.MVCCStats = cArgs.EvalCtx.GetMVCCStats() reply.LeaseAppliedIndex = cArgs.EvalCtx.GetLeaseAppliedIndex() return result.Result{ diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index f5d87d8eaf82..960749ad0458 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -5333,35 +5333,6 @@ func (r *Replica) processRaftCommand( tmpBatch.Close() } - if merge := raftCmd.ReplicatedEvalResult.Merge; merge != nil { - // The merge trigger contains an up-to-date copy of the RHS that we're - // about to apply, but we must ensure that we install that copy into empty - // keyspace. Otherwise we'll end up merging the out-of-date data with the - // up-to-date data, instead of outright replacing it with the up-to-date - // data, leading to consistency violations. - // - // To that end, we inject ClearRange commands into the beginning of the - // write batch for the key spans that make up the RHS. This ensures that - // the up-to-date state replaces the RHS atomically. - // - // TODO(benesch): This will be unnecessary once we mandate that followers - // are up-to-date before the merge transaction commits. As such, this is - // an incredibly naive implementation. A real implementation would a) - // avoid rewriting the whole batch and b) avoid range deletion tombstones - // when there are only a few keys to delete. - tmpBatch := r.store.engine.NewBatch() - for _, keyRange := range rditer.MakeAllKeyRanges(&merge.RightDesc) { - if err := tmpBatch.ClearRange(keyRange.Start, keyRange.End); err != nil { - log.Fatal(ctx, err) - } - } - if err := tmpBatch.ApplyBatchRepr(writeBatch.Data, false); err != nil { - log.Fatal(ctx, err) - } - writeBatch.Data = tmpBatch.Repr() - tmpBatch.Close() - } - var delta enginepb.MVCCStats { var err error diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index 936db9a086ec..e14800be5abd 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -519,9 +519,9 @@ func (r *Replica) AdminMerge( Commit: true, InternalCommitTrigger: &roachpb.InternalCommitTrigger{ MergeTrigger: &roachpb.MergeTrigger{ - LeftDesc: updatedLeftDesc, - RightDesc: rightDesc, - RightData: rhsSnapshotRes.Data, + LeftDesc: updatedLeftDesc, + RightDesc: rightDesc, + RightMVCCStats: rhsSnapshotRes.MVCCStats, }, }, })