Skip to content

Commit

Permalink
ConcurrentSubscripiton avoid concurrent access for invalid demand (ap…
Browse files Browse the repository at this point in the history
…ple#1015)

Motivation:
ConcurrentSubscription currently propagates invalid demand without any
concurrency protection. In general this is invalid use of the API but
may invalidate underlying data structures that are not thread safe and
result in undefined results.

Modifications:
- Use a simpler locking scheme inspired by Publisher#flatMapMerge design
which allows for re-entry and also notification is another thread has
attempted to acquire the lock which will trigger re-processing.

Result:
ConcurrentSubscripiton no longer allows any concurrent access and uses a
more common/shareable locking utility.
  • Loading branch information
Scottmitch authored and Nitesh Kant committed May 6, 2020
1 parent daac219 commit 419b6d9
Show file tree
Hide file tree
Showing 9 changed files with 652 additions and 103 deletions.
@@ -0,0 +1,67 @@
/*
* Copyright © 2020 Apple Inc. and the ServiceTalk project 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 io.servicetalk.benchmark.concurrent;

import io.servicetalk.concurrent.PublisherSource.Subscription;
import io.servicetalk.concurrent.internal.ConcurrentSubscription;

import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Group;
import org.openjdk.jmh.annotations.GroupThreads;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;

@Fork(value = 1)
@State(Scope.Benchmark)
@Warmup(iterations = 5, time = 3)
@Measurement(iterations = 5, time = 3)
@BenchmarkMode(Mode.Throughput)
public class ConcurrentSubscriptionBenchmark {
private final InnerSubscription innerSubscription = new InnerSubscription();
private final Subscription subscription = ConcurrentSubscription.wrap(innerSubscription);

@Benchmark
public long singleThread() {
subscription.request(1);
return innerSubscription.requestN;
}

@Group("multiThread")
@GroupThreads(2)
@Benchmark
public long multiThread() {
subscription.request(1);
return innerSubscription.requestN;
}

private static final class InnerSubscription implements Subscription {
private long requestN;

@Override
public void request(final long n) {
requestN += n;
}

@Override
public void cancel() {
}
}
}
@@ -0,0 +1,88 @@
/*
* Copyright © 2020 Apple Inc. and the ServiceTalk project 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 io.servicetalk.benchmark.concurrent;

import io.servicetalk.concurrent.PublisherSource.Subscription;
import io.servicetalk.concurrent.internal.ConcurrentSubscription;

import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;

import javax.annotation.Nullable;

@Fork(value = 1)
@State(Scope.Benchmark)
@Warmup(iterations = 5, time = 3)
@Measurement(iterations = 5, time = 3)
@BenchmarkMode(Mode.Throughput)
public class ConcurrentSubscriptionReentrantBenchmark {
@Param({"2", "10", "100"})
public int stackDepth;
private InnerSubscription innerSubscription;
private Subscription subscription;

@Setup(Level.Iteration)
public void setup() {
innerSubscription = new InnerSubscription(stackDepth);
subscription = ConcurrentSubscription.wrap(innerSubscription);
innerSubscription.outerSubscription(subscription);
}

@Benchmark
public long singleThread() {
innerSubscription.reentrantCount = 0;
subscription.request(1);
return innerSubscription.requestN;
}

private static final class InnerSubscription implements Subscription {
private long requestN;
private int reentrantCount;
private final int reentrantLimit;
@Nullable
private Subscription outerSubscription;

private InnerSubscription(final int reentrantLimit) {
this.reentrantLimit = reentrantLimit;
}

void outerSubscription(Subscription s) {
outerSubscription = s;
}

@Override
public void request(final long n) {
assert outerSubscription != null;
requestN += n;
if (++reentrantCount < reentrantLimit) {
outerSubscription.request(1);
}
}

@Override
public void cancel() {
}
}
}
Expand Up @@ -25,23 +25,33 @@

import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;

import static io.servicetalk.concurrent.internal.ConcurrentUtils.releaseLock;
import static io.servicetalk.concurrent.internal.ConcurrentUtils.releaseReentrantLock;
import static io.servicetalk.concurrent.internal.ConcurrentUtils.tryAcquireLock;
import static io.servicetalk.concurrent.internal.ConcurrentUtils.tryAcquireReentrantLock;
import static io.servicetalk.concurrent.internal.ServiceTalkTestTimeout.DEFAULT_TIMEOUT_SECONDS;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;

public final class ConcurrentUtilsTest {
private static final AtomicIntegerFieldUpdater<ConcurrentUtilsTest> lockUpdater =
AtomicIntegerFieldUpdater.newUpdater(ConcurrentUtilsTest.class, "lock");
private static final AtomicLongFieldUpdater<ConcurrentUtilsTest> reentrantLockUpdater =
AtomicLongFieldUpdater.newUpdater(ConcurrentUtilsTest.class, "reentrantLock");
@Rule
public final Timeout timeout = new ServiceTalkTestTimeout();

@SuppressWarnings("unused")
private volatile int lock;
@SuppressWarnings("unused")
private volatile long reentrantLock;
private ExecutorService executor;

@Before
Expand All @@ -56,13 +66,23 @@ public void tearDown() throws Exception {
}

@Test
public void singleThread() {
public void lockSingleThread() {
assertTrue(tryAcquireLock(lockUpdater, this));
assertTrue(releaseLock(lockUpdater, this));
}

@Test
public void pendingFromDifferentThread() throws Exception {
public void reentrantLockSingleThread() {
long acquireId = tryAcquireReentrantLock(reentrantLockUpdater, this);
assertThat(acquireId, greaterThan(0L));
long acquireId2 = tryAcquireReentrantLock(reentrantLockUpdater, this);
assertThat(acquireId2, is(-acquireId));
assertTrue(releaseReentrantLock(reentrantLockUpdater, acquireId2, this));
assertTrue(releaseReentrantLock(reentrantLockUpdater, acquireId, this));
}

@Test
public void lockFromDifferentThread() throws Exception {
assertTrue(tryAcquireLock(lockUpdater, this));
executor.submit(() -> assertFalse(tryAcquireLock(lockUpdater, this))).get();

Expand All @@ -75,7 +95,22 @@ public void pendingFromDifferentThread() throws Exception {
}

@Test
public void pendingFromDifferentThreadReAcquireFromDifferentThread() throws Exception {
public void reentrantLockFromDifferentThread() throws Exception {
long acquireId = tryAcquireReentrantLock(reentrantLockUpdater, this);
assertThat(acquireId, greaterThan(0L));
executor.submit(() -> assertThat(tryAcquireReentrantLock(reentrantLockUpdater, this), is(0L))).get();

// we expect false because we are expected to re-acquire and release the lock. This is a feature of the lock
// that requires checking the condition protected by the lock again.
assertFalse(releaseReentrantLock(reentrantLockUpdater, acquireId, this));

acquireId = tryAcquireReentrantLock(reentrantLockUpdater, this);
assertThat(acquireId, greaterThan(0L));
assertTrue(releaseReentrantLock(reentrantLockUpdater, acquireId, this));
}

@Test
public void lockFromDifferentThreadReAcquireFromDifferentThread() throws Exception {
assertTrue(tryAcquireLock(lockUpdater, this));
executor.submit(() -> assertFalse(tryAcquireLock(lockUpdater, this))).get();

Expand All @@ -88,4 +123,21 @@ public void pendingFromDifferentThreadReAcquireFromDifferentThread() throws Exce
assertTrue(releaseLock(lockUpdater, this));
}).get();
}

@Test
public void reentrantLockFromDifferentThreadReAcquireFromDifferentThread() throws Exception {
long acquireId = tryAcquireReentrantLock(reentrantLockUpdater, this);
assertThat(acquireId, greaterThan(0L));
executor.submit(() -> assertThat(tryAcquireReentrantLock(reentrantLockUpdater, this), is(0L))).get();

// we expect false because we are expected to re-acquire and release the lock. This is a feature of the lock
// that requires checking the condition protected by the lock again.
assertFalse(releaseReentrantLock(reentrantLockUpdater, acquireId, this));

executor.submit(() -> {
long acquireId2 = tryAcquireReentrantLock(reentrantLockUpdater, this);
assertThat(acquireId2, greaterThan(0L));
assertTrue(releaseReentrantLock(reentrantLockUpdater, acquireId2, this));
}).get();
}
}
Expand Up @@ -219,9 +219,7 @@ public void concurrentRequestNAndOnNext() throws BrokenBarrierException, Interru
barrier.await();

for (int i = 0; i < expectedSubscribers; ++i) {
while (subscription.requested() - i <= 0) {
Thread.yield();
}
subscription.awaitRequestN(i + 1);
source.onNext(i);
}

Expand Down
Expand Up @@ -46,11 +46,11 @@
import static io.servicetalk.concurrent.internal.DeliberateException.DELIBERATE_EXCEPTION;
import static io.servicetalk.concurrent.internal.TerminalNotification.complete;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static java.util.stream.Collectors.toList;
import static java.util.stream.IntStream.range;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.instanceOf;
Expand All @@ -67,7 +67,7 @@

public class PublisherFlatMapSingleTest {
@Rule
public final Timeout timeout = new ServiceTalkTestTimeout(30, SECONDS);
public final Timeout timeout = new ServiceTalkTestTimeout();

private final TestPublisherSubscriber<Integer> subscriber = new TestPublisherSubscriber<>();
private final TestPublisher<Integer> source = new TestPublisher<>();
Expand Down Expand Up @@ -557,26 +557,20 @@ public void testRequestAndEmitConcurrency() throws Exception {
toSource(source.flatMapMergeSingle(Single::succeeded, 2).beforeOnNext(received::add)).subscribe(subscriber);
source.onSubscribe(subscription);
CountDownLatch requestingStarting = new CountDownLatch(1);
Future<?> submit = executorService.submit(() -> {
Future<?> submitFuture = executorService.submit(() -> {
requestingStarting.countDown();
for (int i = 0; i < totalToRequest; i++) {
subscriber.request(1);
}
});
// Just to make sure we have both threads running concurrently.
requestingStarting.await();
for (int i = 1; i <= totalToRequest; i++) {
int sent = i - 1;
//noinspection StatementWithEmptyBody
while (subscription.requested() - sent <= 0) {
// Don't send if we emit faster than request.
}
for (int i = 0; i < totalToRequest; i++) {
subscription.awaitRequestN(i + 1);
source.onNext(i);
}
submit.get(); // Await everything requested.
submitFuture.get(); // Await everything requested.
assertThat("Unexpected items emitted.", received, hasSize(totalToRequest));
List<Integer> last = received.stream().skip(totalToRequest - 1).collect(toList());
assertThat("Unexpected number of items in last.", last, hasSize(1));
assertThat("Unexpected order of items received: " + received, last.get(0), equalTo(totalToRequest));
assertThat(received, containsInAnyOrder(range(0, totalToRequest).boxed().toArray()));
}
}

0 comments on commit 419b6d9

Please sign in to comment.