@Override public boolean matches(Object item) { return supplier.get().matches(item); }
/** * A {@link SerializableMatcher} with identical criteria to {@link * Matchers#containsInAnyOrder(Object[])}. * * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. It is * explicitly <i>not</i> required or expected to be serializable via Java serialization. */ @SafeVarargs public static <T> SerializableMatcher<Iterable<? extends T>> containsInAnyOrder( Coder<T> coder, T... items) { final SerializableSupplier<T[]> itemsSupplier = new SerializableArrayViaCoder<>(coder, items); return fromSupplier(() -> Matchers.containsInAnyOrder(itemsSupplier.get())); }
@Override public void describeTo(Description description) { supplier.get().describeTo(description); }
/** * A {@link SerializableMatcher} with identical criteria to {@link * Matchers#greaterThan(Comparable)}. * * <p>The target value of type {@code T} will be serialized using the provided {@link Coder}. It * is explicitly <i>not</i> required or expected to be serializable via Java serialization. */ public static <T extends Comparable<T> & Serializable> SerializableMatcher<T> greaterThan( final Coder<T> coder, T target) { final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target); return fromSupplier(() -> Matchers.greaterThan(targetSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link Matchers#contains(Object[])}. * * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. They are * explicitly <i>not</i> required or expected to be serializable via Java serialization. */ @SafeVarargs public static <T> SerializableMatcher<Iterable<? extends T>> contains( Coder<T> coder, T... items) { final SerializableSupplier<T[]> itemsSupplier = new SerializableArrayViaCoder<>(coder, items); return fromSupplier(() -> Matchers.containsInAnyOrder(itemsSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link * Matchers#arrayContaining(Object[])}. * * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. They are * explicitly <i>not</i> required or expected to be serializable via Java serialization. */ @SafeVarargs public static <T> SerializableMatcher<T[]> arrayContaining(Coder<T> coder, T... items) { final SerializableSupplier<T[]> itemsSupplier = new SerializableArrayViaCoder<>(coder, items); return fromSupplier(() -> Matchers.arrayContaining(itemsSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link * Matchers#lessThanOrEqualTo(Comparable)}. * * <p>The target value of type {@code T} will be serialized using the provided {@link Coder}. It * is explicitly <i>not</i> required or expected to be serializable via Java serialization. */ public static <T extends Comparable<T>> SerializableMatcher<T> lessThanOrEqualTo( Coder<T> coder, T target) { final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target); return fromSupplier(() -> Matchers.lessThanOrEqualTo(targetSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link Matchers#isIn(Object[])}. * * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. They are * explicitly <i>not</i> required or expected to be serializable via Java serialization. */ public static <T> SerializableMatcher<T> isIn(Coder<T> coder, T[] items) { final SerializableSupplier<T[]> itemsSupplier = new SerializableArrayViaCoder<>(coder, items); return fromSupplier(() -> Matchers.isIn(itemsSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link * Matchers#arrayContainingInAnyOrder(Object[])}. * * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. They are * explicitly <i>not</i> required or expected to be serializable via Java serialization. */ @SafeVarargs public static <T> SerializableMatcher<T[]> arrayContainingInAnyOrder(Coder<T> coder, T... items) { final SerializableSupplier<T[]> itemsSupplier = new SerializableArrayViaCoder<>(coder, items); return fromSupplier(() -> Matchers.arrayContaining(itemsSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link Matchers#lessThan(Comparable)}. * * <p>The target value of type {@code T} will be serialized using the provided {@link Coder}. It * is explicitly <i>not</i> required or expected to be serializable via Java serialization. */ public static <T extends Comparable<T>> SerializableMatcher<T> lessThan( Coder<T> coder, T target) { final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target); return fromSupplier(() -> Matchers.lessThan(targetSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link Matchers#hasItem(Object)}. * * <p>The item of type {@code T} will be serialized using the provided {@link Coder}. It is * explicitly <i>not</i> required or expected to be serializable via Java serialization. */ public static <T> SerializableMatcher<Iterable<? super T>> hasItem(Coder<T> coder, T target) { final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target); return fromSupplier(() -> Matchers.hasItem(targetSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link * Matchers#greaterThanOrEqualTo(Comparable)}. * * <p>The target value of type {@code T} will be serialized using the provided {@link Coder}. It * is explicitly <i>not</i> required or expected to be serializable via Java serialization. */ public static <T extends Comparable<T> & Serializable> SerializableMatcher<T> greaterThanOrEqualTo(final Coder<T> coder, T target) { final SerializableSupplier<T> targetSupplier = new SerializableViaCoder<>(coder, target); return fromSupplier(() -> Matchers.greaterThanOrEqualTo(targetSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link Matchers#equalTo(Object)}. * * <p>The expected value of type {@code T} will be serialized using the provided {@link Coder}. It * is explicitly <i>not</i> required or expected to be serializable via Java serialization. */ public static <T> SerializableMatcher<T> equalTo(Coder<T> coder, T expected) { final SerializableSupplier<T> expectedSupplier = new SerializableViaCoder<>(coder, expected); return fromSupplier(() -> Matchers.equalTo(expectedSupplier.get())); }
@Override public void describeMismatch(Object item, Description mismatchDescription) { supplier.get().describeMismatch(item, mismatchDescription); } }
/** * A {@link SerializableMatcher} with identical criteria to {@link Matchers#isOneOf}. * * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. They are * explicitly <i>not</i> required or expected to be serializable via Java serialization. */ @SafeVarargs public static <T> SerializableMatcher<T> isOneOf(Coder<T> coder, T... items) { final SerializableSupplier<T[]> itemsSupplier = new SerializableArrayViaCoder<>(coder, items); return fromSupplier(() -> Matchers.isOneOf(itemsSupplier.get())); }
/** * A {@link SerializableMatcher} with identical criteria to {@link Matchers#isIn(Collection)}. * * <p>The items of type {@code T} will be serialized using the provided {@link Coder}. They are * explicitly <i>not</i> required or expected to be serializable via Java serialization. */ public static <T> SerializableMatcher<T> isIn(Coder<T> coder, Collection<T> collection) { @SuppressWarnings("unchecked") T[] items = (T[]) collection.toArray(); final SerializableSupplier<T[]> itemsSupplier = new SerializableArrayViaCoder<>(coder, items); return fromSupplier(() -> Matchers.isIn(itemsSupplier.get())); }