From ad07f3789813220348c1933940935d21f1f12f24 Mon Sep 17 00:00:00 2001 From: akarnokd Date: Wed, 18 Dec 2013 20:56:29 +0100 Subject: [PATCH] Operators BO.chunkify, BO.collect, O.forIterable --- rxjava-core/src/main/java/rx/Observable.java | 15 ++ .../rx/observables/BlockingObservable.java | 28 ++ .../java/rx/operators/OperationCollect.java | 252 ++++++++++++++++++ .../java/rx/operators/OperationConcat.java | 91 +++++++ .../rx/operators/OperationCollectTest.java | 220 +++++++++++++++ .../rx/operators/OperationConcatTest.java | 252 ++++++++++++++++++ 6 files changed, 858 insertions(+) create mode 100644 rxjava-core/src/main/java/rx/operators/OperationCollect.java create mode 100644 rxjava-core/src/test/java/rx/operators/OperationCollectTest.java diff --git a/rxjava-core/src/main/java/rx/Observable.java b/rxjava-core/src/main/java/rx/Observable.java index 5d98df26b9..3b390bb567 100644 --- a/rxjava-core/src/main/java/rx/Observable.java +++ b/rxjava-core/src/main/java/rx/Observable.java @@ -2460,6 +2460,21 @@ public Boolean call(T first, T second) { } }); } + + /** + * Return an Observable which concatenates the observable sequences obtained by running the + * resultSelector for each element in the given Iterable source. + * @param the Iterable sequence value type + * @param the result type + * @param source the source iterable + * @param resultSelector the selector function that returns an Observable + * sequence for each value of the {@code source} iterable sequence + * @return an Observable which concatenates the observable sequences obtained by running the + * resultSelector for each element in the given Iterable source. + */ + public static Observable forIterable(Iterable source, Func1> resultSelector) { + return create(OperationConcat.forIterable(source, resultSelector)); + } /** * Returns an Observable that emits a Boolean value that indicates whether diff --git a/rxjava-core/src/main/java/rx/observables/BlockingObservable.java b/rxjava-core/src/main/java/rx/observables/BlockingObservable.java index 8fe13c9e88..d6f937f251 100644 --- a/rxjava-core/src/main/java/rx/observables/BlockingObservable.java +++ b/rxjava-core/src/main/java/rx/observables/BlockingObservable.java @@ -16,6 +16,7 @@ package rx.observables; import java.util.Iterator; +import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicReference; @@ -23,6 +24,7 @@ import rx.Observable; import rx.Observer; import rx.Subscription; +import rx.operators.OperationCollect; import rx.operators.OperationMostRecent; import rx.operators.OperationNext; import rx.operators.OperationToFuture; @@ -30,7 +32,9 @@ import rx.operators.SafeObservableSubscription; import rx.operators.SafeObserver; import rx.util.functions.Action1; +import rx.util.functions.Func0; import rx.util.functions.Func1; +import rx.util.functions.Func2; /** * An extension of {@link Observable} that provides blocking operators. @@ -354,4 +358,28 @@ public Iterator iterator() { } }; } + + /** + * Return an Iterable sequence that returns elements + * collected/aggregated from the source sequence between consecutive next calls. + * @param the type of the collector + * @param initialCollector factory function to create the initial collector + * @param merger function that merges the current collector with the observed value and returns a (new) collector + * @param replaceCollector function that replaces the current collector with a new collector when the current collector is consumed by an Iterator.next() + * @return an Iterable sequence that returns elements + * collected/aggregated from the source sequence between + * consecutive next calls. + */ + public Iterable collect(Func0 initialCollector, + Func2 merger, + Func1 replaceCollector) { + return OperationCollect.collect(o, initialCollector, merger, replaceCollector); + } + /** + * Return an Iterable that produces a sequence of consecutive (possibly empty) chunks of this Observable sequence. + * @return an Iterable that produces a sequence of consecutive (possibly empty) chunks of this Observable sequence. + */ + public Iterable> chunkify() { + return OperationCollect.chunkify(o); + } } diff --git a/rxjava-core/src/main/java/rx/operators/OperationCollect.java b/rxjava-core/src/main/java/rx/operators/OperationCollect.java new file mode 100644 index 0000000000..202c7e88a0 --- /dev/null +++ b/rxjava-core/src/main/java/rx/operators/OperationCollect.java @@ -0,0 +1,252 @@ +/** + * Copyright 2013 Netflix, Inc. + * + * 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 rx.operators; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import rx.Observable; +import rx.Observer; +import rx.Subscription; +import rx.subscriptions.SingleAssignmentSubscription; +import rx.util.Exceptions; +import rx.util.functions.Func0; +import rx.util.functions.Func1; +import rx.util.functions.Func2; + +/** + * Observable to iterable mapping by using custom collector, merger and + * collector-replacer functions. + */ +public final class OperationCollect { + /** Utility class. */ + private OperationCollect() { throw new IllegalStateException("No instances!"); } + + /** + * Produces an Iterable sequence that returns elements + * collected/aggregated from the source sequence between consecutive next calls. + * @param the source value type + * @param the aggregation type + * @param source the source Observable + * @param initialCollector the factory to create the initial collector + * @param merger the merger that combines the current collector with the observed value and returns a (new) collector + * @param replaceCollector the function that replaces the current collector with a new collector. + * @return the iterable sequence + */ + public static Iterable collect( + final Observable source, + final Func0 initialCollector, + final Func2 merger, + final Func1 replaceCollector) { + return new Iterable() { + @Override + public Iterator iterator() { + SingleAssignmentSubscription sas = new SingleAssignmentSubscription(); + Collect collect = new Collect(initialCollector, merger, replaceCollector, sas); + if (!sas.isUnsubscribed()) { + sas.set(source.subscribe(collect)); + } + return collect; + } + }; + } + + /** + * Produces an Iterable sequence of consecutive (possibly empty) chunks of the source sequence. + * @param the source value type + * @param source the source Observable + * @return an iterable sequence of the chunks + */ + public static Iterable> chunkify(final Observable source) { + ListManager na = new ListManager(); + return collect(source, na, na, na); + } + /** Creates a new ArrayList and manages its content. */ + private static final class ListManager implements Func1, List>, Func0>, Func2, T, List> { + @Override + public List call() { + return new ArrayList(); + } + + @Override + public List call(List t1) { + return call(); + } + @Override + public List call(List t1, T t2) { + t1.add(t2); + return t1; + } + } + + /** The observer and iterator. */ + private static final class Collect implements Observer, Iterator { + final Func2 merger; + final Func1 replaceCollector; + final Subscription cancel; + final Lock lock = new ReentrantLock(); + U current; + boolean hasDone; + boolean hasError; + Throwable error; + /** Iterator's current collector. */ + U iCurrent; + /** Iterator has unclaimed collector. */ + boolean iHasValue; + /** Iterator completed. */ + boolean iDone; + /** Iterator error. */ + Throwable iError; + + public Collect(final Func0 initialCollector, + final Func2 merger, + final Func1 replaceCollector, + final Subscription cancel) { + this.merger = merger; + this.replaceCollector = replaceCollector; + this.cancel = cancel; + try { + current = initialCollector.call(); + } catch (Throwable t) { + hasError = true; + error = t; + cancel.unsubscribe(); + } + } + + @Override + public void onNext(T args) { + boolean unsubscribe = false; + lock.lock(); + try { + if (hasDone || hasError) { + return; + } + try { + current = merger.call(current, args); + } catch (Throwable t) { + error = t; + hasError = true; + unsubscribe = true; + } + } finally { + lock.unlock(); + } + if (unsubscribe) { + cancel.unsubscribe(); + } + } + + @Override + public void onCompleted() { + boolean unsubscribe = false; + lock.lock(); + try { + if (hasDone || hasError) { + return; + } + hasDone = true; + unsubscribe = true; + } finally { + lock.unlock(); + } + if (unsubscribe) { + cancel.unsubscribe(); + } + } + + @Override + public void onError(Throwable e) { + boolean unsubscribe = false; + lock.lock(); + try { + if (hasDone || hasError) { + return; + } + hasError = true; + error = e; + unsubscribe = true; + } finally { + lock.unlock(); + } + if (unsubscribe) { + cancel.unsubscribe(); + } + } + + @Override + public boolean hasNext() { + if (iError != null) { + throw Exceptions.propagate(iError); + } + if (!iHasValue) { + if (!iDone) { + lock.lock(); + try { + if (hasError) { + iError = error; + iDone = true; + current = null; + iCurrent = null; + } else { + iCurrent = current; + iHasValue = true; + if (hasDone) { + current = null; + iDone = true; + } else { + try { + current = replaceCollector.call(iCurrent); + } catch (Throwable t) { + iError = t; + iDone = true; + } + } + } + } finally { + lock.unlock(); + } + if (iDone && iError != null) { + cancel.unsubscribe(); + throw Exceptions.propagate(iError); + } + return true; + } + return false; + } + return true; + } + + @Override + public U next() { + if (hasNext()) { + U value = iCurrent; + iCurrent = null; + iHasValue = false; + return value; + } + throw new NoSuchElementException(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Read-only sequence"); + } + } +} diff --git a/rxjava-core/src/main/java/rx/operators/OperationConcat.java b/rxjava-core/src/main/java/rx/operators/OperationConcat.java index e2c9be1ebf..9540932743 100644 --- a/rxjava-core/src/main/java/rx/operators/OperationConcat.java +++ b/rxjava-core/src/main/java/rx/operators/OperationConcat.java @@ -15,6 +15,7 @@ */ package rx.operators; +import java.util.Iterator; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; @@ -23,6 +24,8 @@ import rx.Observable.OnSubscribeFunc; import rx.Observer; import rx.Subscription; +import rx.subscriptions.SerialSubscription; +import rx.util.functions.Func1; /** * Returns an Observable that emits the items emitted by two or more Observables, one after the @@ -168,4 +171,92 @@ public void unsubscribe() { }; } } + /** + * Concatenates the observable sequences obtained by running the + * resultSelector for each element in the given Iterable source. + * @param the source value type + * @param the result value type + * @param source the source sequence + * @param resultSelector the result selector function to return + * an Observable sequence for concatenation + * @return a subscriber function + */ + public static OnSubscribeFunc forIterable(Iterable source, Func1> resultSelector) { + return new For(source, resultSelector); + } + + /** + * For each source element in an iterable, concatenate + * the observables returned by a selector function. + * @param the source value type + * @param the result value type + */ + private static final class For implements OnSubscribeFunc { + final Iterable source; + final Func1> resultSelector; + + public For(Iterable source, Func1> resultSelector) { + this.source = source; + this.resultSelector = resultSelector; + } + + @Override + public Subscription onSubscribe(Observer t1) { + SerialSubscription ssub = new SerialSubscription(); + Iterator it = source.iterator(); + ValueObserver vo = new ValueObserver(t1, ssub, it, resultSelector); + vo.onCompleted(); // trigger the first subscription + return ssub; + } + } + /** + * The observer of values in the returned observables. + */ + private static final class ValueObserver implements Observer { + final Observer observer; + final SerialSubscription cancel; + final Iterator iterator; + final Func1> resultSelector; + public ValueObserver( + Observer observer, + SerialSubscription cancel, + Iterator iterator, + Func1> resultSelector + ) { + this.observer = observer; + this.cancel = cancel; + this.iterator = iterator; + this.resultSelector = resultSelector; + } + + @Override + public void onNext(R args) { + observer.onNext(args); + } + + @Override + public void onError(Throwable e) { + observer.onError(e); + cancel.unsubscribe(); + } + + @Override + public void onCompleted() { + try { + if (iterator.hasNext()) { + T v = iterator.next(); + Observable o = resultSelector.call(v); + cancel.setSubscription(o.subscribe(this)); + return; + } + } catch (Throwable t) { + observer.onError(t); + cancel.unsubscribe(); + return; + } + observer.onCompleted(); + cancel.unsubscribe(); + } + + } } diff --git a/rxjava-core/src/test/java/rx/operators/OperationCollectTest.java b/rxjava-core/src/test/java/rx/operators/OperationCollectTest.java new file mode 100644 index 0000000000..9708ff17f2 --- /dev/null +++ b/rxjava-core/src/test/java/rx/operators/OperationCollectTest.java @@ -0,0 +1,220 @@ +/** + * Copyright 2013 Netflix, Inc. + * + * 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 rx.operators; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import static org.junit.Assert.*; +import org.junit.Test; +import rx.Observable; +import rx.subjects.PublishSubject; +import rx.util.functions.Func0; +import rx.util.functions.Func1; +import rx.util.functions.Func2; + +public class OperationCollectTest { + @Test + public void testChunkify() { + PublishSubject source = PublishSubject.create(); + + Iterable> iterable = source.toBlockingObservable().chunkify(); + + Iterator> it = iterable.iterator(); + + assertEquals(Arrays.asList(), it.next()); + + source.onNext(1); + + assertEquals(Arrays.asList(1), it.next()); + + source.onNext(2); + source.onNext(3); + + assertEquals(Arrays.asList(2, 3), it.next()); + + assertEquals(Arrays.asList(), it.next()); + + source.onNext(4); + source.onNext(5); + source.onNext(6); + + it.hasNext(); + + source.onNext(7); + source.onNext(8); + source.onNext(9); + source.onNext(10); + + it.hasNext(); + assertEquals(Arrays.asList(4, 5, 6), it.next()); + + assertEquals(Arrays.asList(7, 8, 9, 10), it.next()); + + source.onCompleted(); + + assertEquals(Arrays.asList(), it.next()); + + try { + it.next(); + fail("Should have thrown NoSuchElementException"); + } catch (NoSuchElementException ex) { + // this is fine + } + } + @Test + public void testIterateManyTimes() { + Observable source = Observable.from(1, 2, 3); + + Iterable> iter = source.toBlockingObservable().chunkify(); + + for (int i = 0; i < 3; i++) { + Iterator> it = iter.iterator(); + + assertTrue(it.hasNext()); + + List list = it.next(); + + assertEquals(Arrays.asList(1, 2, 3), list); + } + } + static final class CustomException extends RuntimeException { + public CustomException(String message) { + super(message); + } + } + @Test + public void testInitialBufferThrows() { + Observable source = Observable.from(1, 2, 3); + + Func0 initialBuffer = new Func0() { + @Override + public Integer call() { + throw new CustomException("Forced failure!"); + } + }; + + Func2 collector = new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + return t1 + t2; + } + }; + + Func1 replaceBuffer = new Func1() { + + @Override + public Integer call(Integer t1) { + return 0; + } + + }; + + Iterable iter = source.toBlockingObservable().collect(initialBuffer, collector, replaceBuffer); + + Iterator it = iter.iterator(); + + try { + it.next(); + fail("Failed to throw CustomException"); + } catch (CustomException ex) { + // okay to get here + + } + } + + @Test + public void testCollectorThrows() { + Observable source = Observable.from(1, 2, 3); + + Func0 initialBuffer = new Func0() { + @Override + public Integer call() { + return 0; + } + }; + + Func2 collector = new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + throw new CustomException("Forced failure!"); + } + }; + + Func1 replaceBuffer = new Func1() { + + @Override + public Integer call(Integer t1) { + return 0; + } + + }; + + Iterable iter = source.toBlockingObservable().collect(initialBuffer, collector, replaceBuffer); + + Iterator it = iter.iterator(); + + try { + it.next(); + fail("Failed to throw CustomException"); + } catch (CustomException ex) { + // okay to get here + } + } + @Test + public void testReplaceBufferThrows() { + PublishSubject source = PublishSubject.create(); + + Func0 initialBuffer = new Func0() { + @Override + public Integer call() { + return 0; + } + }; + + Func2 collector = new Func2() { + @Override + public Integer call(Integer t1, Integer t2) { + return t1 + t2; + } + }; + + Func1 replaceBuffer = new Func1() { + + @Override + public Integer call(Integer t1) { + throw new CustomException("Forced failure!"); + } + + }; + + Iterable iter = source.toBlockingObservable().collect(initialBuffer, collector, replaceBuffer); + + Iterator it = iter.iterator(); + + source.onNext(1); + source.onNext(2); + source.onNext(3); + + try { + it.next(); + fail("Failed to throw CustomException"); + } catch (CustomException ex) { + // okay to get here + } + } +} diff --git a/rxjava-core/src/test/java/rx/operators/OperationConcatTest.java b/rxjava-core/src/test/java/rx/operators/OperationConcatTest.java index 24146ec131..d8bc14f36b 100644 --- a/rxjava-core/src/test/java/rx/operators/OperationConcatTest.java +++ b/rxjava-core/src/test/java/rx/operators/OperationConcatTest.java @@ -21,9 +21,11 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import org.junit.Test; @@ -32,8 +34,10 @@ import rx.Observable; import rx.Observer; import rx.Subscription; +import rx.schedulers.Schedulers; import rx.schedulers.TestScheduler; import rx.subscriptions.BooleanSubscription; +import rx.util.functions.Func1; public class OperationConcatTest { @@ -598,4 +602,252 @@ public void testMultipleObservers() { verify(o1, never()).onError(any(Throwable.class)); verify(o2, never()).onError(any(Throwable.class)); } + + @Test + public void testForIterable() { + List keys = Arrays.asList(0, 1, 2); + final List> values = Arrays.asList( + Observable.from(1, 2, 3), + Observable.from(4, 5, 6), + Observable.from(7, 8, 9) + ); + Func1> selector = new Func1>() { + + @Override + public Observable call(Integer t1) { + return values.get(t1); + } + }; + + Observable result = Observable.forIterable(keys, selector); + + Observer o = mock(Observer.class); + InOrder inOrder = inOrder(o); + + result.subscribe(o); + + for (int i = 1; i < 10; i++) { + inOrder.verify(o, times(1)).onNext(i); + } + inOrder.verify(o, times(1)).onCompleted(); + inOrder.verifyNoMoreInteractions(); + verify(o, never()).onError(any(Throwable.class)); + } + + @Test + public void testForIterableLongSequence() { + int n = 250; + List keys = new ArrayList(n); + List expected = new ArrayList(3 * n); + for (int i = 1; i <= n; i++) { + keys.add(n); + for (int j = 1; j <= 3; j++) { + expected.add(j); + } + } + + Func1> selector = new Func1>() { + @Override + public Observable call(Integer t1) { + return Observable.from(1, 2, 3).subscribeOn(Schedulers.currentThread()); + } + }; + + Observable result = Observable.forIterable(keys, selector); + + Observer o = mock(Observer.class); + InOrder inOrder = inOrder(o); + + result.subscribe(o); + + for (Integer i : expected) { + inOrder.verify(o, times(1)).onNext(i); + } + inOrder.verify(o, times(1)).onCompleted(); + inOrder.verifyNoMoreInteractions(); + verify(o, never()).onError(any(Throwable.class)); + + } + @Test + public void testForIterableEmpty() { + List keys = Arrays.asList(); + final AtomicBoolean selectorCalled = new AtomicBoolean(); + Func1> selector = new Func1>() { + @Override + public Observable call(Integer t1) { + selectorCalled.set(true); + throw new IllegalStateException("Shouldn't get here!"); + } + }; + Observable result = Observable.forIterable(keys, selector); + + Observer o = mock(Observer.class); + InOrder inOrder = inOrder(o); + + result.subscribe(o); + + inOrder.verify(o, times(1)).onCompleted(); + inOrder.verifyNoMoreInteractions(); + verify(o, never()).onNext(any()); + verify(o, never()).onError(any(Throwable.class)); + + assertFalse(selectorCalled.get()); + } + static final class CustomException extends RuntimeException { + public CustomException(String message) { + super(message); + } + } + @Test + public void testForIterableIteratorHasNextThrows() { + final AtomicBoolean nextCalled = new AtomicBoolean(); + Iterable keys = new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + + @Override + public boolean hasNext() { + throw new CustomException("Forced failure"); + } + + @Override + public Integer next() { + nextCalled.set(true); + throw new IllegalStateException("Shouldn't get here!"); + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Read-only sequence"); + } + + }; + } + + }; + + final AtomicBoolean selectorCalled = new AtomicBoolean(); + + Func1> selector = new Func1>() { + @Override + public Observable call(Integer t1) { + selectorCalled.set(true); + throw new IllegalStateException("Shouldn't get here!"); + } + }; + + Observable result = Observable.forIterable(keys, selector); + Observer o = mock(Observer.class); + + result.subscribe(o); + + verify(o, times(1)).onError(any(CustomException.class)); + verify(o, never()).onNext(any()); + verify(o, never()).onCompleted(); + + assertFalse(selectorCalled.get()); + assertFalse(nextCalled.get()); + } + @Test + public void testForIterableIteratorNextThrows() { + Iterable keys = new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + + @Override + public boolean hasNext() { + return true; + } + + @Override + public Integer next() { + throw new CustomException("Forced failure"); + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Read-only sequence"); + } + + }; + } + + }; + + final AtomicBoolean selectorCalled = new AtomicBoolean(); + + Func1> selector = new Func1>() { + @Override + public Observable call(Integer t1) { + selectorCalled.set(true); + throw new IllegalStateException("Shouldn't get here!"); + } + }; + + Observable result = Observable.forIterable(keys, selector); + Observer o = mock(Observer.class); + + result.subscribe(o); + + verify(o, times(1)).onError(any(CustomException.class)); + verify(o, never()).onNext(any()); + verify(o, never()).onCompleted(); + + assertFalse(selectorCalled.get()); + } + + @Test + public void testForIterableSelectorThrows() { + List keys = Arrays.asList(0); + + + Func1> selector = new Func1>() { + @Override + public Observable call(Integer t1) { + throw new CustomException("Forced failure!"); + } + }; + + Observable result = Observable.forIterable(keys, selector); + Observer o = mock(Observer.class); + + result.subscribe(o); + + verify(o, times(1)).onError(any(CustomException.class)); + verify(o, never()).onNext(any()); + verify(o, never()).onCompleted(); + } + + @Test + public void testForIterableOneThrows() { + List keys = Arrays.asList(0, 1, 2); + final List> values = Arrays.asList( + Observable.from(1, 2, 3), + Observable.from(4, 5, 6), + Observable.error(new CustomException("Forced failure")) + ); + Func1> selector = new Func1>() { + + @Override + public Observable call(Integer t1) { + return values.get(t1); + } + }; + + Observable result = Observable.forIterable(keys, selector); + + Observer o = mock(Observer.class); + InOrder inOrder = inOrder(o); + + result.subscribe(o); + + for (int i = 1; i < 7; i++) { + inOrder.verify(o, times(1)).onNext(i); + } + inOrder.verify(o, times(1)).onError(any(CustomException.class)); + inOrder.verifyNoMoreInteractions(); + verify(o, never()).onCompleted(); + } }