Auto merge of #69072 - ecstatic-morse:associated-items, r=petrochenkov

O(log n) lookup of associated items by name

Resolves #68957, in which compile time is quadratic in the number of associated items. This PR makes name lookup use binary search instead of a linear scan to improve its asymptotic performance. As a result, the pathological case from that issue now runs in 8 seconds on my local machine, as opposed to many minutes on the current stable.

Currently, method resolution must do a linear scan through all associated items of a type to find one with a certain name. This PR changes the result of the `associated_items` query to a data structure that preserves the definition order of associated items (which is used, e.g., for the layout of trait object vtables) while adding an index of those items sorted by (unhygienic) name. When doing name lookup, we first find all items with the same `Symbol` using binary search, then run hygienic comparison to find the one we are looking for. Ideally, this would be implemented using an insertion-order preserving, hash-based multi-map, but one is not readily available.

Someone who is more familiar with identifier hygiene could probably make this better by auditing the uses of the `AssociatedItems` interface. My goal was to preserve the current behavior exactly, even if it seemed strange (I left at least one FIXME to this effect). For example, some places use comparison with `ident.modern()` and some places use `tcx.hygienic_eq` which requires the `DefId` of the containing `impl`. I don't know whether those approaches are equivalent or which one should be preferred.
This commit is contained in:
bors 2020-02-20 22:44:01 +00:00
commit d3ebd592d0
35 changed files with 497 additions and 183 deletions

View File

@ -48,6 +48,7 @@ macro_rules! arena_types {
[] item_local_set: rustc_hir::ItemLocalSet,
[decode] mir_const_qualif: rustc_index::bit_set::BitSet<rustc::mir::Local>,
[] trait_impls_of: rustc::ty::trait_def::TraitImpls,
[] associated_items: rustc::ty::AssociatedItems,
[] dropck_outlives:
rustc::infer::canonical::Canonical<'tcx,
rustc::infer::canonical::QueryResponse<'tcx,

View File

@ -333,7 +333,7 @@ rustc_queries! {
query associated_item(_: DefId) -> ty::AssocItem {}
/// Collects the associated items defined on a trait or impl.
query associated_items(key: DefId) -> &'tcx [ty::AssocItem] {
query associated_items(key: DefId) -> &'tcx ty::AssociatedItems {
desc { |tcx| "collecting associated items of {}", tcx.def_path_str(key) }
}

View File

@ -81,8 +81,8 @@ impl<'tcx> Node {
}
/// Iterate over the items defined directly by the given (impl or trait) node.
pub fn items(&self, tcx: TyCtxt<'tcx>) -> &'tcx [ty::AssocItem] {
tcx.associated_items(self.def_id())
pub fn items(&self, tcx: TyCtxt<'tcx>) -> impl 'tcx + Iterator<Item = &'tcx ty::AssocItem> {
tcx.associated_items(self.def_id()).in_definition_order()
}
/// Finds an associated item defined in this node.
@ -99,7 +99,7 @@ impl<'tcx> Node {
use crate::ty::AssocKind::*;
tcx.associated_items(self.def_id())
.iter()
.filter_by_name_unhygienic(trait_item_name.name)
.find(move |impl_item| {
match (trait_item_kind, impl_item.kind) {
| (Const, Const)

View File

@ -122,7 +122,7 @@ impl<'tcx> OverloadedDeref<'tcx> {
};
let method_def_id = tcx
.associated_items(trait_def_id.unwrap())
.iter()
.in_definition_order()
.find(|m| m.kind == ty::AssocKind::Method)
.unwrap()
.def_id;

View File

@ -337,7 +337,7 @@ impl<'tcx> Instance<'tcx> {
let fn_once = tcx.lang_items().fn_once_trait().unwrap();
let call_once = tcx
.associated_items(fn_once)
.iter()
.in_definition_order()
.find(|it| it.kind == ty::AssocKind::Method)
.unwrap()
.def_id;

View File

@ -30,10 +30,11 @@ use rustc_attr as attr;
use rustc_data_structures::captures::Captures;
use rustc_data_structures::fx::FxHashMap;
use rustc_data_structures::fx::FxIndexMap;
use rustc_data_structures::sorted_map::SortedIndexMultiMap;
use rustc_data_structures::stable_hasher::{HashStable, StableHasher};
use rustc_data_structures::sync::{self, par_iter, Lrc, ParallelIterator};
use rustc_hir as hir;
use rustc_hir::def::{CtorKind, CtorOf, DefKind, Res};
use rustc_hir::def::{CtorKind, CtorOf, DefKind, Namespace, Res};
use rustc_hir::def_id::{CrateNum, DefId, DefIdMap, LocalDefId, CRATE_DEF_INDEX, LOCAL_CRATE};
use rustc_hir::{Constness, GlobMap, Node, TraitMap};
use rustc_index::vec::{Idx, IndexVec};
@ -216,6 +217,13 @@ impl AssocKind {
ty::AssocKind::Const => "associated constant",
}
}
pub fn namespace(&self) -> Namespace {
match *self {
ty::AssocKind::OpaqueTy | ty::AssocKind::Type => Namespace::TypeNS,
ty::AssocKind::Const | ty::AssocKind::Method => Namespace::ValueNS,
}
}
}
impl AssocItem {
@ -257,6 +265,81 @@ impl AssocItem {
}
}
/// A list of `ty::AssocItem`s in definition order that allows for efficient lookup by name.
///
/// When doing lookup by name, we try to postpone hygienic comparison for as long as possible since
/// it is relatively expensive. Instead, items are indexed by `Symbol` and hygienic comparison is
/// done only on items with the same name.
#[derive(Debug, Clone, PartialEq, HashStable)]
pub struct AssociatedItems {
items: SortedIndexMultiMap<u32, Symbol, ty::AssocItem>,
}
impl AssociatedItems {
/// Constructs an `AssociatedItems` map from a series of `ty::AssocItem`s in definition order.
pub fn new(items_in_def_order: impl IntoIterator<Item = ty::AssocItem>) -> Self {
let items = items_in_def_order.into_iter().map(|item| (item.ident.name, item)).collect();
AssociatedItems { items }
}
/// Returns a slice of associated items in the order they were defined.
///
/// New code should avoid relying on definition order. If you need a particular associated item
/// for a known trait, make that trait a lang item instead of indexing this array.
pub fn in_definition_order(&self) -> impl '_ + Iterator<Item = &ty::AssocItem> {
self.items.iter().map(|(_, v)| v)
}
/// Returns an iterator over all associated items with the given name, ignoring hygiene.
pub fn filter_by_name_unhygienic(
&self,
name: Symbol,
) -> impl '_ + Iterator<Item = &ty::AssocItem> {
self.items.get_by_key(&name)
}
/// Returns an iterator over all associated items with the given name.
///
/// Multiple items may have the same name if they are in different `Namespace`s. For example,
/// an associated type can have the same name as a method. Use one of the `find_by_name_and_*`
/// methods below if you know which item you are looking for.
pub fn filter_by_name(
&'a self,
tcx: TyCtxt<'a>,
ident: Ident,
parent_def_id: DefId,
) -> impl 'a + Iterator<Item = &'a ty::AssocItem> {
self.filter_by_name_unhygienic(ident.name)
.filter(move |item| tcx.hygienic_eq(ident, item.ident, parent_def_id))
}
/// Returns the associated item with the given name and `AssocKind`, if one exists.
pub fn find_by_name_and_kind(
&self,
tcx: TyCtxt<'_>,
ident: Ident,
kind: AssocKind,
parent_def_id: DefId,
) -> Option<&ty::AssocItem> {
self.filter_by_name_unhygienic(ident.name)
.filter(|item| item.kind == kind)
.find(|item| tcx.hygienic_eq(ident, item.ident, parent_def_id))
}
/// Returns the associated item with the given name in the given `Namespace`, if one exists.
pub fn find_by_name_and_namespace(
&self,
tcx: TyCtxt<'_>,
ident: Ident,
ns: Namespace,
parent_def_id: DefId,
) -> Option<&ty::AssocItem> {
self.filter_by_name_unhygienic(ident.name)
.filter(|item| item.kind.namespace() == ns)
.find(|item| tcx.hygienic_eq(ident, item.ident, parent_def_id))
}
}
#[derive(Clone, Debug, PartialEq, Eq, Copy, RustcEncodable, RustcDecodable, HashStable)]
pub enum Visibility {
/// Visible everywhere (including in other crates).
@ -2731,14 +2814,14 @@ impl<'tcx> TyCtxt<'tcx> {
.for_each(|&body_id| f(self.hir().body_owner_def_id(body_id)));
}
pub fn provided_trait_methods(self, id: DefId) -> impl Iterator<Item = &'tcx AssocItem> {
pub fn provided_trait_methods(self, id: DefId) -> impl 'tcx + Iterator<Item = &'tcx AssocItem> {
self.associated_items(id)
.iter()
.in_definition_order()
.filter(|item| item.kind == AssocKind::Method && item.defaultness.has_value())
}
pub fn trait_relevant_for_never(self, did: DefId) -> bool {
self.associated_items(did).iter().any(|item| item.relevant_for_never())
self.associated_items(did).in_definition_order().any(|item| item.relevant_for_never())
}
pub fn opt_item_name(self, def_id: DefId) -> Option<Ident> {

View File

@ -1066,11 +1066,7 @@ impl<'tcx> ProjectionTy<'tcx> {
) -> ProjectionTy<'tcx> {
let item_def_id = tcx
.associated_items(trait_ref.def_id)
.iter()
.find(|item| {
item.kind == ty::AssocKind::Type
&& tcx.hygienic_eq(item_name, item.ident, trait_ref.def_id)
})
.find_by_name_and_kind(tcx, item_name, ty::AssocKind::Type, trait_ref.def_id)
.unwrap()
.def_id;

View File

@ -357,7 +357,7 @@ impl<'tcx> TyCtxt<'tcx> {
let mut dtor_did = None;
let ty = self.type_of(adt_did);
self.for_each_relevant_impl(drop_trait, ty, |impl_did| {
if let Some(item) = self.associated_items(impl_did).first() {
if let Some(item) = self.associated_items(impl_did).in_definition_order().nth(0) {
if validate(self, impl_did).is_ok() {
dtor_did = Some(item.def_id);
}

View File

@ -4,6 +4,10 @@ use std::iter::FromIterator;
use std::mem;
use std::ops::{Bound, Index, IndexMut, RangeBounds};
mod index_map;
pub use index_map::SortedIndexMultiMap;
/// `SortedMap` is a data structure with similar characteristics as BTreeMap but
/// slightly different trade-offs: lookup, insertion, and removal are O(log(N))
/// and elements can be iterated in order cheaply.

View File

@ -0,0 +1,218 @@
//! A variant of `SortedMap` that preserves insertion order.
use std::borrow::Borrow;
use std::hash::{Hash, Hasher};
use std::iter::FromIterator;
use crate::stable_hasher::{HashStable, StableHasher};
use rustc_index::vec::{Idx, IndexVec};
/// An indexed multi-map that preserves insertion order while permitting both `O(log n)` lookup of
/// an item by key and `O(1)` lookup by index.
///
/// This data structure is a hybrid of an [`IndexVec`] and a [`SortedMap`]. Like `IndexVec`,
/// `SortedIndexMultiMap` assigns a typed index to each item while preserving insertion order.
/// Like `SortedMap`, `SortedIndexMultiMap` has efficient lookup of items by key. However, this
/// is accomplished by sorting an array of item indices instead of the items themselves.
///
/// Unlike `SortedMap`, this data structure can hold multiple equivalent items at once, so the
/// `get_by_key` method and its variants return an iterator instead of an `Option`. Equivalent
/// items will be yielded in insertion order.
///
/// Unlike a general-purpose map like `BTreeSet` or `HashSet`, `SortedMap` and
/// `SortedIndexMultiMap` require `O(n)` time to insert a single item. This is because we may need
/// to insert into the middle of the sorted array. Users should avoid mutating this data structure
/// in-place.
///
/// [`IndexVec`]: ../../rustc_index/vec/struct.IndexVec.html
/// [`SortedMap`]: ../sorted_map/struct.SortedMap.html
#[derive(Clone, Debug)]
pub struct SortedIndexMultiMap<I: Idx, K, V> {
/// The elements of the map in insertion order.
items: IndexVec<I, (K, V)>,
/// Indices of the items in the set, sorted by the item's key.
idx_sorted_by_item_key: Vec<I>,
}
impl<I: Idx, K: Ord, V> SortedIndexMultiMap<I, K, V> {
pub fn new() -> Self {
SortedIndexMultiMap { items: IndexVec::new(), idx_sorted_by_item_key: Vec::new() }
}
pub fn len(&self) -> usize {
self.items.len()
}
pub fn is_empty(&self) -> bool {
self.items.is_empty()
}
/// Returns an iterator over the items in the map in insertion order.
pub fn into_iter(self) -> impl DoubleEndedIterator<Item = (K, V)> {
self.items.into_iter()
}
/// Returns an iterator over the items in the map in insertion order along with their indices.
pub fn into_iter_enumerated(self) -> impl DoubleEndedIterator<Item = (I, (K, V))> {
self.items.into_iter_enumerated()
}
/// Returns an iterator over the items in the map in insertion order.
pub fn iter(&self) -> impl '_ + DoubleEndedIterator<Item = (&K, &V)> {
self.items.iter().map(|(ref k, ref v)| (k, v))
}
/// Returns an iterator over the items in the map in insertion order along with their indices.
pub fn iter_enumerated(&self) -> impl '_ + DoubleEndedIterator<Item = (I, (&K, &V))> {
self.items.iter_enumerated().map(|(i, (ref k, ref v))| (i, (k, v)))
}
/// Returns the item in the map with the given index.
pub fn get(&self, idx: I) -> Option<&(K, V)> {
self.items.get(idx)
}
/// Returns an iterator over the items in the map that are equal to `key`.
///
/// If there are multiple items that are equivalent to `key`, they will be yielded in
/// insertion order.
pub fn get_by_key<Q: 'a>(&'a self, key: &Q) -> impl 'a + Iterator<Item = &'a V>
where
Q: Ord + ?Sized,
K: Borrow<Q>,
{
self.get_by_key_enumerated(key).map(|(_, v)| v)
}
/// Returns an iterator over the items in the map that are equal to `key` along with their
/// indices.
///
/// If there are multiple items that are equivalent to `key`, they will be yielded in
/// insertion order.
pub fn get_by_key_enumerated<Q>(&self, key: &Q) -> impl '_ + Iterator<Item = (I, &V)>
where
Q: Ord + ?Sized,
K: Borrow<Q>,
{
// FIXME: This should be in the standard library as `equal_range`. See rust-lang/rfcs#2184.
match self.binary_search_idx(key) {
Err(_) => self.idxs_to_items_enumerated(&[]),
Ok(idx) => {
let start = self.find_lower_bound(key, idx);
let end = self.find_upper_bound(key, idx);
self.idxs_to_items_enumerated(&self.idx_sorted_by_item_key[start..end])
}
}
}
fn binary_search_idx<Q>(&self, key: &Q) -> Result<usize, usize>
where
Q: Ord + ?Sized,
K: Borrow<Q>,
{
self.idx_sorted_by_item_key.binary_search_by(|&idx| self.items[idx].0.borrow().cmp(key))
}
/// Returns the index into the `idx_sorted_by_item_key` array of the first item equal to
/// `key`.
///
/// `initial` must be an index into that same array for an item that is equal to `key`.
fn find_lower_bound<Q>(&self, key: &Q, initial: usize) -> usize
where
Q: Ord + ?Sized,
K: Borrow<Q>,
{
debug_assert!(self.items[self.idx_sorted_by_item_key[initial]].0.borrow() == key);
// FIXME: At present, this uses linear search, meaning lookup is only `O(log n)` if duplicate
// entries are rare. It would be better to start with a linear search for the common case but
// fall back to an exponential search if many duplicates are found. This applies to
// `upper_bound` as well.
let mut start = initial;
while start != 0 && self.items[self.idx_sorted_by_item_key[start - 1]].0.borrow() == key {
start -= 1;
}
start
}
/// Returns the index into the `idx_sorted_by_item_key` array of the first item greater than
/// `key`, or `self.len()` if no such item exists.
///
/// `initial` must be an index into that same array for an item that is equal to `key`.
fn find_upper_bound<Q>(&self, key: &Q, initial: usize) -> usize
where
Q: Ord + ?Sized,
K: Borrow<Q>,
{
debug_assert!(self.items[self.idx_sorted_by_item_key[initial]].0.borrow() == key);
// See the FIXME for `find_lower_bound`.
let mut end = initial + 1;
let len = self.items.len();
while end < len && self.items[self.idx_sorted_by_item_key[end]].0.borrow() == key {
end += 1;
}
end
}
fn idxs_to_items_enumerated(&'a self, idxs: &'a [I]) -> impl 'a + Iterator<Item = (I, &'a V)> {
idxs.iter().map(move |&idx| (idx, &self.items[idx].1))
}
}
impl<I: Idx, K: Eq, V: Eq> Eq for SortedIndexMultiMap<I, K, V> {}
impl<I: Idx, K: PartialEq, V: PartialEq> PartialEq for SortedIndexMultiMap<I, K, V> {
fn eq(&self, other: &Self) -> bool {
// No need to compare the sorted index. If the items are the same, the index will be too.
self.items == other.items
}
}
impl<I: Idx, K, V> Hash for SortedIndexMultiMap<I, K, V>
where
K: Hash,
V: Hash,
{
fn hash<H: Hasher>(&self, hasher: &mut H) {
self.items.hash(hasher)
}
}
impl<I: Idx, K, V, C> HashStable<C> for SortedIndexMultiMap<I, K, V>
where
K: HashStable<C>,
V: HashStable<C>,
{
fn hash_stable(&self, ctx: &mut C, hasher: &mut StableHasher) {
self.items.hash_stable(ctx, hasher)
}
}
impl<I: Idx, K: Ord, V> FromIterator<(K, V)> for SortedIndexMultiMap<I, K, V> {
fn from_iter<J>(iter: J) -> Self
where
J: IntoIterator<Item = (K, V)>,
{
let items = IndexVec::from_iter(iter);
let mut idx_sorted_by_item_key: Vec<_> = items.indices().collect();
// `sort_by_key` is stable, so insertion order is preserved for duplicate items.
idx_sorted_by_item_key.sort_by_key(|&idx| &items[idx].0);
SortedIndexMultiMap { items, idx_sorted_by_item_key }
}
}
impl<I: Idx, K, V> std::ops::Index<I> for SortedIndexMultiMap<I, K, V> {
type Output = V;
fn index(&self, idx: I) -> &Self::Output {
&self.items[idx].1
}
}
#[cfg(tests)]
mod tests;

View File

@ -1,4 +1,30 @@
use super::SortedMap;
use super::{SortedIndexMultiMap, SortedMap};
#[test]
fn test_sorted_index_multi_map() {
let entries: Vec<_> = vec![(2, 0), (1, 0), (2, 1), (3, 0), (2, 2)];
let set: SortedIndexMultiMap<usize, _, _> = entries.iter().copied().collect();
// Insertion order is preserved.
assert!(entries.iter().map(|(ref k, ref v)| (k, v)).eq(set.iter()));
// Indexing
for (i, expect) in entries.iter().enumerate() {
assert_eq!(set[i], expect.1);
}
// `get_by_key` works.
assert_eq!(set.get_by_key(&3).copied().collect::<Vec<_>>(), vec![0]);
assert!(set.get_by_key(&4).next().is_none());
// `get_by_key` returns items in insertion order.
let twos: Vec<_> = set.get_by_key_enumerated(&2).collect();
let idxs: Vec<usize> = twos.iter().map(|(i, _)| *i).collect();
let values: Vec<usize> = twos.iter().map(|(_, &v)| v).collect();
assert_eq!(idxs, vec![0, 2, 4]);
assert_eq!(values, vec![0, 1, 2]);
}
#[test]
fn test_insert_and_iter() {

View File

@ -1,4 +1,4 @@
use crate::def::{DefKind, Res};
use crate::def::{DefKind, Namespace, Res};
use crate::def_id::DefId;
crate use crate::hir_id::HirId;
use crate::itemlikevisit;
@ -1897,6 +1897,15 @@ pub enum ImplItemKind<'hir> {
OpaqueTy(GenericBounds<'hir>),
}
impl ImplItemKind<'_> {
pub fn namespace(&self) -> Namespace {
match self {
ImplItemKind::OpaqueTy(..) | ImplItemKind::TyAlias(..) => Namespace::TypeNS,
ImplItemKind::Const(..) | ImplItemKind::Method(..) => Namespace::ValueNS,
}
}
}
// The name of the associated type for `Fn` return types.
pub const FN_OUTPUT_NAME: Symbol = sym::Output;

View File

@ -537,7 +537,7 @@ fn vtable_methods<'tcx>(
tcx.arena.alloc_from_iter(supertraits(tcx, trait_ref).flat_map(move |trait_ref| {
let trait_methods = tcx
.associated_items(trait_ref.def_id())
.iter()
.in_definition_order()
.filter(|item| item.kind == ty::AssocKind::Method);
// Now list each method's DefId and InternalSubsts (for within its trait).

View File

@ -213,7 +213,7 @@ fn object_safety_violations_for_trait(
// Check methods for violations.
let mut violations: Vec<_> = tcx
.associated_items(trait_def_id)
.iter()
.in_definition_order()
.filter(|item| item.kind == ty::AssocKind::Method)
.filter_map(|item| {
object_safety_violation_for_method(tcx, trait_def_id, &item)
@ -289,7 +289,7 @@ fn object_safety_violations_for_trait(
violations.extend(
tcx.associated_items(trait_def_id)
.iter()
.in_definition_order()
.filter(|item| item.kind == ty::AssocKind::Const)
.map(|item| ObjectSafetyViolation::AssocConst(item.ident.name, item.ident.span)),
);
@ -646,7 +646,7 @@ fn object_ty_for_trait<'tcx>(
let mut associated_types = traits::supertraits(tcx, ty::Binder::dummy(trait_ref))
.flat_map(|super_trait_ref| {
tcx.associated_items(super_trait_ref.def_id())
.iter()
.in_definition_order()
.map(move |item| (super_trait_ref, item))
})
.filter(|(_, item)| item.kind == ty::AssocKind::Type)

View File

@ -586,7 +586,7 @@ pub fn count_own_vtable_entries(tcx: TyCtxt<'tcx>, trait_ref: ty::PolyTraitRef<'
let mut entries = 0;
// Count number of methods and add them to the total offset.
// Skip over associated types and constants.
for trait_item in tcx.associated_items(trait_ref.def_id()) {
for trait_item in tcx.associated_items(trait_ref.def_id()).in_definition_order() {
if trait_item.kind == ty::AssocKind::Method {
entries += 1;
}
@ -606,7 +606,7 @@ pub fn get_vtable_index_of_object_method<N>(
// add them to the total offset.
// Skip over associated types and constants.
let mut entries = object.vtable_base;
for trait_item in tcx.associated_items(object.upcast_trait_ref.def_id()) {
for trait_item in tcx.associated_items(object.upcast_trait_ref.def_id()).in_definition_order() {
if trait_item.def_id == method_def_id {
// The item with the ID we were given really ought to be a method.
assert_eq!(trait_item.kind, ty::AssocKind::Method);

View File

@ -318,7 +318,10 @@ impl<'a, 'tcx> WfPredicates<'a, 'tcx> {
};
if let Elaborate::All = elaborate {
let trait_assoc_items = tcx.associated_items(trait_ref.def_id);
// FIXME: Make `extend_cause_with_original_assoc_item_obligation` take an iterator
// instead of a slice.
let trait_assoc_items: Vec<_> =
tcx.associated_items(trait_ref.def_id).in_definition_order().copied().collect();
let predicates = obligations.iter().map(|obligation| obligation.predicate).collect();
let implied_obligations = traits::elaborate_predicates(tcx, predicates);
@ -327,7 +330,7 @@ impl<'a, 'tcx> WfPredicates<'a, 'tcx> {
extend_cause_with_original_assoc_item_obligation(
&mut cause,
&pred,
trait_assoc_items,
&*trait_assoc_items,
);
traits::Obligation::new(cause, param_env, pred)
});

View File

@ -68,7 +68,7 @@ fn make_shim<'tcx>(tcx: TyCtxt<'tcx>, instance: ty::InstanceDef<'tcx>) -> &'tcx
let fn_mut = tcx.lang_items().fn_mut_trait().unwrap();
let call_mut = tcx
.associated_items(fn_mut)
.iter()
.in_definition_order()
.find(|it| it.kind == ty::AssocKind::Method)
.unwrap()
.def_id;

View File

@ -539,7 +539,7 @@ where
debug!("destructor_call_block({:?}, {:?})", self, succ);
let tcx = self.tcx();
let drop_trait = tcx.lang_items().drop_trait().unwrap();
let drop_fn = tcx.associated_items(drop_trait)[0];
let drop_fn = tcx.associated_items(drop_trait).in_definition_order().nth(0).unwrap();
let ty = self.place_ty(self.place);
let substs = tcx.mk_substs_trait(ty, &[]);

View File

@ -167,17 +167,19 @@ impl<'a, 'tcx> Cx<'a, 'tcx> {
params: &[GenericArg<'tcx>],
) -> &'tcx ty::Const<'tcx> {
let substs = self.tcx.mk_substs_trait(self_ty, params);
for item in self.tcx.associated_items(trait_def_id) {
// The unhygienic comparison here is acceptable because this is only
// used on known traits.
if item.kind == ty::AssocKind::Method && item.ident.name == method_name {
let method_ty = self.tcx.type_of(item.def_id);
let method_ty = method_ty.subst(self.tcx, substs);
return ty::Const::zero_sized(self.tcx, method_ty);
}
}
bug!("found no method `{}` in `{:?}`", method_name, trait_def_id);
// The unhygienic comparison here is acceptable because this is only
// used on known traits.
let item = self
.tcx
.associated_items(trait_def_id)
.filter_by_name_unhygienic(method_name)
.find(|item| item.kind == ty::AssocKind::Method)
.expect("trait method not found");
let method_ty = self.tcx.type_of(item.def_id);
let method_ty = method_ty.subst(self.tcx, substs);
ty::Const::zero_sized(self.tcx, method_ty)
}
crate fn all_fields(&mut self, adt_def: &ty::AdtDef, variant_index: VariantIdx) -> Vec<Field> {

View File

@ -523,8 +523,8 @@ impl Visitor<'tcx> for Checker<'tcx> {
let trait_item_def_id = self
.tcx
.associated_items(trait_did)
.iter()
.find(|item| item.ident.name == impl_item.ident.name)
.filter_by_name_unhygienic(impl_item.ident.name)
.next()
.map(|item| item.def_id);
if let Some(def_id) = trait_item_def_id {
// Pass `None` to skip deprecation warnings.

View File

@ -408,7 +408,6 @@ impl<'l, 'tcx> SaveContext<'l, 'tcx> {
qualname.push_str(&self.tcx.hir().hir_to_pretty_string(self_ty.hir_id));
let trait_id = self.tcx.trait_id_of_impl(impl_id);
let mut decl_id = None;
let mut docs = String::new();
let mut attrs = vec![];
let hir_id = self.tcx.hir().node_to_hir_id(id);
@ -417,15 +416,18 @@ impl<'l, 'tcx> SaveContext<'l, 'tcx> {
attrs = item.attrs.to_vec();
}
let mut decl_id = None;
if let Some(def_id) = trait_id {
// A method in a trait impl.
qualname.push_str(" as ");
qualname.push_str(&self.tcx.def_path_str(def_id));
self.tcx
decl_id = self
.tcx
.associated_items(def_id)
.iter()
.find(|item| item.ident.name == ident.name)
.map(|item| decl_id = Some(item.def_id));
.filter_by_name_unhygienic(ident.name)
.next()
.map(|item| item.def_id);
}
qualname.push_str(">");
@ -716,12 +718,11 @@ impl<'l, 'tcx> SaveContext<'l, 'tcx> {
Res::Def(HirDefKind::Method, decl_id) => {
let def_id = if decl_id.is_local() {
let ti = self.tcx.associated_item(decl_id);
self.tcx
.associated_items(ti.container.id())
.iter()
.find(|item| {
item.ident.name == ti.ident.name && item.defaultness.has_value()
})
.filter_by_name_unhygienic(ti.ident.name)
.find(|item| item.defaultness.has_value())
.map(|item| item.def_id)
} else {
None

View File

@ -210,10 +210,9 @@ fn associated_item_def_ids(tcx: TyCtxt<'_>, def_id: DefId) -> &[DefId] {
}
}
fn associated_items<'tcx>(tcx: TyCtxt<'tcx>, def_id: DefId) -> &'tcx [ty::AssocItem] {
tcx.arena.alloc_from_iter(
tcx.associated_item_def_ids(def_id).iter().map(|did| tcx.associated_item(*did)),
)
fn associated_items<'tcx>(tcx: TyCtxt<'tcx>, def_id: DefId) -> &'tcx ty::AssociatedItems {
let items = tcx.associated_item_def_ids(def_id).iter().map(|did| tcx.associated_item(*did));
tcx.arena.alloc(ty::AssociatedItems::new(items))
}
fn def_span(tcx: TyCtxt<'_>, def_id: DefId) -> Span {

View File

@ -9,7 +9,6 @@ use crate::collect::PlaceholderHirTyCollector;
use crate::lint;
use crate::middle::lang_items::SizedTraitLangItem;
use crate::middle::resolve_lifetime as rl;
use crate::namespace::Namespace;
use crate::require_c_abi_if_c_variadic;
use crate::util::common::ErrorReported;
use rustc::lint::builtin::AMBIGUOUS_ASSOCIATED_ITEMS;
@ -20,7 +19,7 @@ use rustc::ty::{GenericParamDef, GenericParamDefKind};
use rustc_data_structures::fx::{FxHashMap, FxHashSet};
use rustc_errors::{pluralize, struct_span_err, Applicability, DiagnosticId};
use rustc_hir as hir;
use rustc_hir::def::{CtorOf, DefKind, Res};
use rustc_hir::def::{CtorOf, DefKind, Namespace, Res};
use rustc_hir::def_id::DefId;
use rustc_hir::intravisit::Visitor;
use rustc_hir::print;
@ -1109,10 +1108,10 @@ impl<'o, 'tcx> dyn AstConv<'tcx> + 'o {
trait_def_id: DefId,
assoc_name: ast::Ident,
) -> bool {
self.tcx().associated_items(trait_def_id).iter().any(|item| {
item.kind == ty::AssocKind::Type
&& self.tcx().hygienic_eq(assoc_name, item.ident, trait_def_id)
})
self.tcx()
.associated_items(trait_def_id)
.find_by_name_and_kind(self.tcx(), assoc_name, ty::AssocKind::Type, trait_def_id)
.is_some()
}
// Returns `true` if a bounds list includes `?Sized`.
@ -1345,9 +1344,12 @@ impl<'o, 'tcx> dyn AstConv<'tcx> + 'o {
let (assoc_ident, def_scope) =
tcx.adjust_ident_and_get_scope(binding.item_name, candidate.def_id(), hir_ref_id);
// We have already adjusted the item name above, so compare with `ident.modern()` instead
// of calling `filter_by_name_and_kind`.
let assoc_ty = tcx
.associated_items(candidate.def_id())
.iter()
.filter_by_name_unhygienic(assoc_ident.name)
.find(|i| i.kind == ty::AssocKind::Type && i.ident.modern() == assoc_ident)
.expect("missing associated type");
@ -1513,7 +1515,7 @@ impl<'o, 'tcx> dyn AstConv<'tcx> + 'o {
ty::Predicate::Trait(pred, _) => {
associated_types.entry(span).or_default().extend(
tcx.associated_items(pred.def_id())
.iter()
.in_definition_order()
.filter(|item| item.kind == ty::AssocKind::Type)
.map(|item| item.def_id),
);
@ -1968,14 +1970,11 @@ impl<'o, 'tcx> dyn AstConv<'tcx> + 'o {
let mut where_bounds = vec![];
for bound in bounds {
let bound_id = bound.def_id();
let bound_span = self
.tcx()
.associated_items(bound.def_id())
.iter()
.find(|item| {
item.kind == ty::AssocKind::Type
&& self.tcx().hygienic_eq(assoc_name, item.ident, bound.def_id())
})
.associated_items(bound_id)
.find_by_name_and_kind(self.tcx(), assoc_name, ty::AssocKind::Type, bound_id)
.and_then(|item| self.tcx().hir().span_if_local(item.def_id));
if let Some(bound_span) = bound_span {
@ -2053,7 +2052,7 @@ impl<'o, 'tcx> dyn AstConv<'tcx> + 'o {
);
let all_candidate_names: Vec<_> = all_candidates()
.map(|r| self.tcx().associated_items(r.def_id()))
.map(|r| self.tcx().associated_items(r.def_id()).in_definition_order())
.flatten()
.filter_map(
|item| if item.kind == ty::AssocKind::Type { Some(item.ident.name) } else { None },
@ -2199,10 +2198,13 @@ impl<'o, 'tcx> dyn AstConv<'tcx> + 'o {
let trait_did = bound.def_id();
let (assoc_ident, def_scope) =
tcx.adjust_ident_and_get_scope(assoc_ident, trait_did, hir_ref_id);
// We have already adjusted the item name above, so compare with `ident.modern()` instead
// of calling `filter_by_name_and_kind`.
let item = tcx
.associated_items(trait_did)
.iter()
.find(|i| Namespace::from(i.kind) == Namespace::Type && i.ident.modern() == assoc_ident)
.in_definition_order()
.find(|i| i.kind.namespace() == Namespace::TypeNS && i.ident.modern() == assoc_ident)
.expect("missing associated type");
let ty = self.projected_ty_from_poly_trait_ref(span, item.def_id, assoc_segment, bound);

View File

@ -248,7 +248,8 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
if is_gen {
// Check that we deduce the signature from the `<_ as std::ops::Generator>::Return`
// associated item and not yield.
let return_assoc_item = self.tcx.associated_items(gen_trait)[1].def_id;
let return_assoc_item =
self.tcx.associated_items(gen_trait).in_definition_order().nth(1).unwrap().def_id;
if return_assoc_item != projection.projection_def_id() {
debug!("deduce_sig_from_projection: not return assoc item of generator");
return None;
@ -673,7 +674,8 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
// The `Future` trait has only one associted item, `Output`,
// so check that this is what we see.
let output_assoc_item = self.tcx.associated_items(future_trait)[0].def_id;
let output_assoc_item =
self.tcx.associated_items(future_trait).in_definition_order().nth(0).unwrap().def_id;
if output_assoc_item != predicate.projection_ty.item_def_id {
span_bug!(
cause_span,

View File

@ -539,7 +539,7 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
let item_def_id = self
.tcx
.associated_items(deref_trait)
.iter()
.in_definition_order()
.find(|item| item.kind == ty::AssocKind::Type)
.unwrap()
.def_id;

View File

@ -11,7 +11,6 @@ pub use self::CandidateSource::*;
pub use self::MethodError::*;
use crate::check::FnCtxt;
use crate::namespace::Namespace;
use rustc::ty::subst::Subst;
use rustc::ty::subst::{InternalSubsts, SubstsRef};
use rustc::ty::GenericParamDefKind;
@ -19,7 +18,7 @@ use rustc::ty::{self, ToPolyTraitRef, ToPredicate, TraitRef, Ty, TypeFoldable, W
use rustc_data_structures::sync::Lrc;
use rustc_errors::{Applicability, DiagnosticBuilder};
use rustc_hir as hir;
use rustc_hir::def::{CtorOf, DefKind};
use rustc_hir::def::{CtorOf, DefKind, Namespace};
use rustc_hir::def_id::DefId;
use rustc_infer::infer::{self, InferOk};
use rustc_infer::traits;
@ -342,7 +341,7 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
// Trait must have a method named `m_name` and it should not have
// type parameters or early-bound regions.
let tcx = self.tcx;
let method_item = match self.associated_item(trait_def_id, m_name, Namespace::Value) {
let method_item = match self.associated_item(trait_def_id, m_name, Namespace::ValueNS) {
Some(method_item) => method_item,
None => {
tcx.sess.delay_span_bug(
@ -484,11 +483,7 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
) -> Option<ty::AssocItem> {
self.tcx
.associated_items(def_id)
.iter()
.find(|item| {
Namespace::from(item.kind) == ns
&& self.tcx.hygienic_eq(item_name, item.ident, def_id)
})
.find_by_name_and_namespace(self.tcx, item_name, ns, def_id)
.copied()
}
}

View File

@ -7,7 +7,6 @@ use crate::check::autoderef::{self, Autoderef};
use crate::check::FnCtxt;
use crate::hir::def::DefKind;
use crate::hir::def_id::DefId;
use crate::namespace::Namespace;
use rustc::lint;
use rustc::middle::stability;
@ -22,6 +21,7 @@ use rustc_data_structures::fx::FxHashSet;
use rustc_data_structures::sync::Lrc;
use rustc_errors::struct_span_err;
use rustc_hir as hir;
use rustc_hir::def::Namespace;
use rustc_infer::infer::canonical::OriginalQueryValues;
use rustc_infer::infer::canonical::{Canonical, QueryResponse};
use rustc_infer::infer::type_variable::{TypeVariableOrigin, TypeVariableOriginKind};
@ -1696,20 +1696,20 @@ impl<'a, 'tcx> ProbeContext<'a, 'tcx> {
let max_dist = max(name.as_str().len(), 3) / 3;
self.tcx
.associated_items(def_id)
.iter()
.in_definition_order()
.filter(|x| {
let dist = lev_distance(&*name.as_str(), &x.ident.as_str());
Namespace::from(x.kind) == Namespace::Value && dist > 0 && dist <= max_dist
x.kind.namespace() == Namespace::ValueNS && dist > 0 && dist <= max_dist
})
.copied()
.collect()
} else {
self.fcx
.associated_item(def_id, name, Namespace::Value)
.associated_item(def_id, name, Namespace::ValueNS)
.map_or(Vec::new(), |x| vec![x])
}
} else {
self.tcx.associated_items(def_id).to_vec()
self.tcx.associated_items(def_id).in_definition_order().copied().collect()
}
}
}

View File

@ -3,7 +3,6 @@
use crate::check::FnCtxt;
use crate::middle::lang_items::FnOnceTraitLangItem;
use crate::namespace::Namespace;
use rustc::hir::map as hir_map;
use rustc::hir::map::Map;
use rustc::ty::print::with_crate_prefix;
@ -11,7 +10,7 @@ use rustc::ty::{self, ToPolyTraitRef, ToPredicate, Ty, TyCtxt, TypeFoldable, Wit
use rustc_data_structures::fx::FxHashSet;
use rustc_errors::{pluralize, struct_span_err, Applicability, DiagnosticBuilder};
use rustc_hir as hir;
use rustc_hir::def::{DefKind, Res};
use rustc_hir::def::{DefKind, Namespace, Res};
use rustc_hir::def_id::{DefId, CRATE_DEF_INDEX, LOCAL_CRATE};
use rustc_hir::intravisit;
use rustc_hir::{ExprKind, Node, QPath};
@ -97,13 +96,13 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
// Provide the best span we can. Use the item, if local to crate, else
// the impl, if local to crate (item may be defaulted), else nothing.
let item = match self
.associated_item(impl_did, item_name, Namespace::Value)
.associated_item(impl_did, item_name, Namespace::ValueNS)
.or_else(|| {
let impl_trait_ref = self.tcx.impl_trait_ref(impl_did)?;
self.associated_item(
impl_trait_ref.def_id,
item_name,
Namespace::Value,
Namespace::ValueNS,
)
}) {
Some(item) => item,
@ -185,7 +184,7 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
}
CandidateSource::TraitSource(trait_did) => {
let item =
match self.associated_item(trait_did, item_name, Namespace::Value) {
match self.associated_item(trait_did, item_name, Namespace::ValueNS) {
Some(item) => item,
None => continue,
};
@ -264,7 +263,7 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
// be used exists at all, and the type is an ambiguous numeric type
// ({integer}/{float}).
let mut candidates = all_traits(self.tcx).into_iter().filter_map(|info| {
self.associated_item(info.def_id, item_name, Namespace::Value)
self.associated_item(info.def_id, item_name, Namespace::ValueNS)
});
if let (true, false, SelfSource::MethodCall(expr), Some(_)) = (
actual.is_numeric(),
@ -779,7 +778,7 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
// here).
(type_is_local || info.def_id.is_local())
&& self
.associated_item(info.def_id, item_name, Namespace::Value)
.associated_item(info.def_id, item_name, Namespace::ValueNS)
.filter(|item| {
// We only want to suggest public or local traits (#45781).
item.vis == ty::Visibility::Public || info.def_id.is_local()

View File

@ -89,7 +89,6 @@ pub mod writeback;
use crate::astconv::{AstConv, PathSeg};
use crate::middle::lang_items;
use crate::namespace::Namespace;
use rustc::hir::map::blocks::FnLikeNode;
use rustc::hir::map::Map;
use rustc::middle::region;
@ -1972,19 +1971,16 @@ fn check_impl_items_against_trait<'tcx>(
// Check existing impl methods to see if they are both present in trait
// and compatible with trait signature
for impl_item in impl_items() {
let namespace = impl_item.kind.namespace();
let ty_impl_item = tcx.associated_item(tcx.hir().local_def_id(impl_item.hir_id));
let ty_trait_item = tcx
.associated_items(impl_trait_ref.def_id)
.iter()
.find(|ac| {
Namespace::from(&impl_item.kind) == Namespace::from(ac.kind)
&& tcx.hygienic_eq(ty_impl_item.ident, ac.ident, impl_trait_ref.def_id)
})
.find_by_name_and_namespace(tcx, ty_impl_item.ident, namespace, impl_trait_ref.def_id)
.or_else(|| {
// Not compatible, but needed for the error message
tcx.associated_items(impl_trait_ref.def_id)
.iter()
.find(|ac| tcx.hygienic_eq(ty_impl_item.ident, ac.ident, impl_trait_ref.def_id))
.filter_by_name(tcx, ty_impl_item.ident, impl_trait_ref.def_id)
.next()
});
// Check that impl definition matches trait definition
@ -2088,7 +2084,7 @@ fn check_impl_items_against_trait<'tcx>(
let mut missing_items = Vec::new();
let mut invalidated_items = Vec::new();
let associated_type_overridden = overridden_associated_type.is_some();
for trait_item in tcx.associated_items(impl_trait_ref.def_id) {
for trait_item in tcx.associated_items(impl_trait_ref.def_id).in_definition_order() {
let is_implemented = trait_def
.ancestors(tcx, impl_id)
.leaf_def(tcx, trait_item.ident, trait_item.kind)
@ -5265,7 +5261,13 @@ impl<'a, 'tcx> FnCtxt<'a, 'tcx> {
// Check for `Future` implementations by constructing a predicate to
// prove: `<T as Future>::Output == U`
let future_trait = self.tcx.lang_items().future_trait().unwrap();
let item_def_id = self.tcx.associated_items(future_trait)[0].def_id;
let item_def_id = self
.tcx
.associated_items(future_trait)
.in_definition_order()
.nth(0)
.unwrap()
.def_id;
let predicate =
ty::Predicate::Projection(ty::Binder::bind(ty::ProjectionPredicate {
// `<T as Future>::Output`

View File

@ -1,5 +1,4 @@
use crate::namespace::Namespace;
use rustc::ty::{AssocItem, TyCtxt};
use rustc::ty::TyCtxt;
use rustc_errors::struct_span_err;
use rustc_hir as hir;
use rustc_hir::def_id::{CrateNum, DefId, LOCAL_CRATE};
@ -23,18 +22,18 @@ impl InherentOverlapChecker<'tcx> {
let impl_items1 = self.tcx.associated_items(impl1);
let impl_items2 = self.tcx.associated_items(impl2);
for item1 in &impl_items1[..] {
for item2 in &impl_items2[..] {
// Avoid costly `.modern()` calls as much as possible by doing them as late as we
// can. Compare raw symbols first.
if item1.ident.name == item2.ident.name
&& Namespace::from(item1.kind) == Namespace::from(item2.kind)
{
for item1 in impl_items1.in_definition_order() {
let collision = impl_items2
.filter_by_name_unhygienic(item1.ident.name)
.find(|item2| {
// Symbols and namespace match, compare hygienically.
if item1.ident.modern() == item2.ident.modern() {
return true;
}
}
item1.kind.namespace() == item2.kind.namespace()
&& item1.ident.modern() == item2.ident.modern()
})
.is_some();
if collision {
return true;
}
}
@ -47,43 +46,43 @@ impl InherentOverlapChecker<'tcx> {
impl2: DefId,
overlap: traits::OverlapResult<'_>,
) {
let name_and_namespace =
|assoc: &AssocItem| (assoc.ident.modern(), Namespace::from(assoc.kind));
let impl_items1 = self.tcx.associated_items(impl1);
let impl_items2 = self.tcx.associated_items(impl2);
for item1 in &impl_items1[..] {
let (name, namespace) = name_and_namespace(item1);
for item1 in impl_items1.in_definition_order() {
let collision = impl_items2.filter_by_name_unhygienic(item1.ident.name).find(|item2| {
// Symbols and namespace match, compare hygienically.
item1.kind.namespace() == item2.kind.namespace()
&& item1.ident.modern() == item2.ident.modern()
});
for item2 in &impl_items2[..] {
if (name, namespace) == name_and_namespace(item2) {
let mut err = struct_span_err!(
self.tcx.sess,
self.tcx.span_of_impl(item1.def_id).unwrap(),
E0592,
"duplicate definitions with name `{}`",
name
);
err.span_label(
self.tcx.span_of_impl(item1.def_id).unwrap(),
format!("duplicate definitions for `{}`", name),
);
err.span_label(
self.tcx.span_of_impl(item2.def_id).unwrap(),
format!("other definition for `{}`", name),
);
if let Some(item2) = collision {
let name = item1.ident.modern();
let mut err = struct_span_err!(
self.tcx.sess,
self.tcx.span_of_impl(item1.def_id).unwrap(),
E0592,
"duplicate definitions with name `{}`",
name
);
err.span_label(
self.tcx.span_of_impl(item1.def_id).unwrap(),
format!("duplicate definitions for `{}`", name),
);
err.span_label(
self.tcx.span_of_impl(item2.def_id).unwrap(),
format!("other definition for `{}`", name),
);
for cause in &overlap.intercrate_ambiguity_causes {
cause.add_intercrate_ambiguity_hint(&mut err);
}
if overlap.involves_placeholder {
traits::add_placeholder_note(&mut err);
}
err.emit();
for cause in &overlap.intercrate_ambiguity_causes {
cause.add_intercrate_ambiguity_hint(&mut err);
}
if overlap.involves_placeholder {
traits::add_placeholder_note(&mut err);
}
err.emit();
}
}
}

View File

@ -83,7 +83,6 @@ mod collect;
mod constrained_generic_params;
mod impl_wf_check;
mod mem_categorization;
mod namespace;
mod outlives;
mod structured_errors;
mod variance;

View File

@ -1,27 +0,0 @@
use rustc::ty;
use rustc_hir as hir;
// Whether an item exists in the type or value namespace.
#[derive(Copy, Clone, PartialEq, Eq, Debug)]
pub enum Namespace {
Type,
Value,
}
impl From<ty::AssocKind> for Namespace {
fn from(a_kind: ty::AssocKind) -> Self {
match a_kind {
ty::AssocKind::OpaqueTy | ty::AssocKind::Type => Namespace::Type,
ty::AssocKind::Const | ty::AssocKind::Method => Namespace::Value,
}
}
}
impl<'a> From<&'a hir::ImplItemKind<'_>> for Namespace {
fn from(impl_kind: &'a hir::ImplItemKind<'_>) -> Self {
match *impl_kind {
hir::ImplItemKind::OpaqueTy(..) | hir::ImplItemKind::TyAlias(..) => Namespace::Type,
hir::ImplItemKind::Const(..) | hir::ImplItemKind::Method(..) => Namespace::Value,
}
}
}

View File

@ -114,8 +114,7 @@ impl<'a, 'tcx> BlanketImplFinder<'a, 'tcx> {
.cx
.tcx
.associated_items(impl_def_id)
.iter()
.copied()
.in_definition_order()
.collect::<Vec<_>>()
.clean(self.cx),
polarity: None,

View File

@ -190,8 +190,10 @@ pub fn record_extern_fqn(cx: &DocContext<'_>, did: DefId, kind: clean::TypeKind)
}
pub fn build_external_trait(cx: &DocContext<'_>, did: DefId) -> clean::Trait {
let trait_items =
cx.tcx.associated_items(did).in_definition_order().map(|item| item.clean(cx)).collect();
let auto_trait = cx.tcx.trait_def(did).has_auto_impl;
let trait_items = cx.tcx.associated_items(did).iter().map(|item| item.clean(cx)).collect();
let predicates = cx.tcx.predicates_of(did);
let generics = (cx.tcx.generics_of(did), predicates).clean(cx);
let generics = filter_non_trait_generics(did, generics);
@ -376,7 +378,7 @@ pub fn build_impl(
} else {
(
tcx.associated_items(did)
.iter()
.in_definition_order()
.filter_map(|item| {
if associated_trait.is_some() || item.vis == ty::Visibility::Public {
Some(item.clean(cx))

View File

@ -95,7 +95,7 @@ impl<'a, 'tcx> LinkCollector<'a, 'tcx> {
.tcx
.inherent_impls(did)
.iter()
.flat_map(|imp| cx.tcx.associated_items(*imp))
.flat_map(|imp| cx.tcx.associated_items(*imp).in_definition_order())
.any(|item| item.ident.name == variant_name)
{
return Err(ErrorKind::ResolutionFailure);
@ -206,8 +206,8 @@ impl<'a, 'tcx> LinkCollector<'a, 'tcx> {
return cx
.tcx
.associated_items(did)
.iter()
.find(|item| item.ident.name == item_name)
.filter_by_name_unhygienic(item_name)
.next()
.and_then(|item| match item.kind {
ty::AssocKind::Method => Some("method"),
_ => None,
@ -234,7 +234,7 @@ impl<'a, 'tcx> LinkCollector<'a, 'tcx> {
.tcx
.inherent_impls(did)
.iter()
.flat_map(|imp| cx.tcx.associated_items(*imp))
.flat_map(|imp| cx.tcx.associated_items(*imp).in_definition_order())
.find(|item| item.ident.name == item_name);
if let Some(item) = item {
let out = match item.kind {