Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
276 changes: 151 additions & 125 deletions samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.samza.util;

import java.time.Duration;
import java.util.Collection;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;


/**
* A utility class to perform complex state transitions.
*/
public class StateTransitionUtil {
private static final Logger LOG = LoggerFactory.getLogger(StateTransitionUtil.class);

/**
* Atomically sets the value to the given updated value if current value is one of the expected values.
*
* @param reference the atomic reference
* @param expectedValues set of expected values
* @param update the new value
* @param <T> type of the atomic reference
*
* @return true if current state is one of the expected value and transition was successful; false otherwise
*/
public static <T> boolean compareAndSet(AtomicReference<T> reference, Collection<T> expectedValues, T update) {
while (true) {
T currentValue = reference.get();
if (expectedValues.contains(currentValue)) {
if (reference.compareAndSet(currentValue, update)) {
return true;
}
} else {
return false;
}
}
}

/**
* Atomically sets the value to the given updated value if the black listed values does not contain current value.
*
* @param reference the atomic reference
* @param blacklistedValues set of blacklisted values
* @param update the new value
* @param <T> type of the atomic reference
*
* @return true if current state is not in the blacklisted values and transition was successful; false otherwise
*/
public static <T> boolean compareNotInAndSet(AtomicReference<T> reference, Collection<T> blacklistedValues, T update) {
Copy link
Contributor

@shanthoosh shanthoosh Sep 26, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just curious.

Though it looks like we use atomic reference in StreamProcessor, we lock implicitly in StateTransitionUtil. From what I can understand, we've replaced synchronized lock with a busy spinning lock in this patch for performing state transition on a object.

My concern with the spinlock is that, while waiting to acquire a lock, it wastes the CPU cycles. Can you please share the value-add and benefits that we get by replacing synchronized lock with the spinning lock. If we need to lock, why can't we use a explicit lock.

while (true) {
T currentValue = reference.get();
if (blacklistedValues.contains(currentValue)) {
return false;
}

if (reference.compareAndSet(currentValue, update)) {
return true;
}
}
}

/**
* Atomically sets the value to the updated value if the current value == expected value and the barrier
* latch counts down to zero. If the barrier times out determined by the timeout parameter, the atomic reference
* is not updated.
*
* @param reference the atomic reference
* @param expect the expected value
* @param update the new value
* @param barrier the barrier latch
* @param timeout the timeout for barrier
* @param <T> type of the atomic reference
*
* @return true if current state == expected value and the barrier completed and transition was successful; false otherwise
*/
public static <T> boolean compareAndSetWithBarrier(AtomicReference<T> reference, T expect, T update,
CountDownLatch barrier, Duration timeout) {
if (reference.get() != expect) {
LOG.error("Failed to transition from {} to {}", expect, update);
throw new IllegalStateException("Cannot transition to " + update + " from state " + expect
+ " since the current state is " + reference.get());
}

try {
if (timeout.isNegative()) {
barrier.await();
} else {
boolean completed = barrier.await(timeout.toMillis(), TimeUnit.MILLISECONDS);
if (!completed) {
LOG.error("Failed to transition from {} to {} due to barrier timeout.", expect, update);
return false;
}
}
} catch (InterruptedException e) {
LOG.error("Failed to transition from {} to {} due to {}", new Object[] {expect, update, e});
return false;
}

return reference.compareAndSet(expect, update);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@
import org.powermock.api.mockito.PowerMockito;

import static org.junit.Assert.assertEquals;
import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.*;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
Expand Down Expand Up @@ -363,14 +363,18 @@ public void testOnJobModelExpiredShouldMakeCorrectStateTransitions() {
* state and should shutdown JobCoordinator.
*/
Mockito.doNothing().when(mockJobCoordinator).start();
Mockito.doNothing().when(mockJobCoordinator).stop();
Mockito.doAnswer(ans -> {
streamProcessor.state.set(State.STOPPING);
return null;
}).when(mockJobCoordinator).stop();
Mockito.doNothing().when(mockSamzaContainer).shutdown();
Mockito.when(mockSamzaContainer.hasStopped()).thenReturn(false);
Mockito.when(mockSamzaContainer.getStatus())
.thenReturn(SamzaContainerStatus.STARTED)
.thenReturn(SamzaContainerStatus.STOPPED);
streamProcessor.container = mockSamzaContainer;
streamProcessor.state = State.STARTED;
streamProcessor.containerShutdownLatch = new CountDownLatch(1);
streamProcessor.state.set(State.STARTED);

streamProcessor.jobCoordinatorListener.onJobModelExpired();

Expand All @@ -379,27 +383,28 @@ public void testOnJobModelExpiredShouldMakeCorrectStateTransitions() {
Mockito.verify(mockJobCoordinator, Mockito.times(1)).stop();

// If StreamProcessor is in IN_REBALANCE state, onJobModelExpired should be a NO_OP.
streamProcessor.state = State.IN_REBALANCE;
streamProcessor.state.set(State.IN_REBALANCE);

streamProcessor.jobCoordinatorListener.onJobModelExpired();

assertEquals(State.IN_REBALANCE, streamProcessor.state);
assertEquals(State.IN_REBALANCE, streamProcessor.getState());
}

@Test
public void testOnNewJobModelShouldResultInValidStateTransitions() throws Exception {
public void testOnNewJobModelShouldResultInValidStateTransitions() {
JobCoordinator mockJobCoordinator = Mockito.mock(JobCoordinator.class);
ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class);
SamzaContainer mockSamzaContainer = Mockito.mock(SamzaContainer.class);
MapConfig config = new MapConfig(ImmutableMap.of("task.shutdown.ms", "0"));
StreamProcessor streamProcessor = PowerMockito.spy(new StreamProcessor(config, new HashMap<>(), null, lifecycleListener, mockJobCoordinator));
StreamProcessor streamProcessor = new TestableStreamProcessor(config, new HashMap<>(), null,
lifecycleListener, mockJobCoordinator, mockSamzaContainer);

streamProcessor.container = mockSamzaContainer;
streamProcessor.state = State.IN_REBALANCE;
streamProcessor.state.set(State.IN_REBALANCE);
Mockito.doNothing().when(mockSamzaContainer).run();

streamProcessor.jobCoordinatorListener.onNewJobModel("TestProcessorId", new JobModel(new MapConfig(), new HashMap<>()));

Mockito.verify(mockSamzaContainer, Mockito.times(1)).setContainerListener(any());
Mockito.verify(mockSamzaContainer, Mockito.atMost(1)).run();
}

Expand All @@ -418,11 +423,11 @@ public void testStopShouldBeIdempotent() {
.thenReturn(SamzaContainerStatus.STARTED)
.thenReturn(SamzaContainerStatus.STOPPED);

streamProcessor.state = State.RUNNING;
streamProcessor.state.set(State.RUNNING);

streamProcessor.stop();

assertEquals(State.STOPPING, streamProcessor.state);
assertEquals(State.STOPPING, streamProcessor.getState());
}

@Test
Expand All @@ -436,13 +441,13 @@ public void testCoordinatorFailureShouldStopTheStreamProcessor() {
Exception failureException = new Exception("dummy exception");

streamProcessor.container = mockSamzaContainer;
streamProcessor.state = State.RUNNING;
streamProcessor.state.set(State.RUNNING);
streamProcessor.jobCoordinatorListener.onCoordinatorFailure(failureException);
Mockito.doNothing().when(mockSamzaContainer).shutdown();
Mockito.when(mockSamzaContainer.hasStopped()).thenReturn(false);


assertEquals(State.STOPPED, streamProcessor.state);
assertEquals(State.STOPPED, streamProcessor.state.get());
Mockito.verify(lifecycleListener).afterFailure(failureException);
Mockito.verify(mockSamzaContainer).shutdown();
}
Expand All @@ -454,10 +459,10 @@ public void testCoordinatorStopShouldStopTheStreamProcessor() {
MapConfig config = new MapConfig(ImmutableMap.of("task.shutdown.ms", "0"));
StreamProcessor streamProcessor = new StreamProcessor(config, new HashMap<>(), null, lifecycleListener, mockJobCoordinator);

streamProcessor.state = State.RUNNING;
streamProcessor.state.set(State.RUNNING);
streamProcessor.jobCoordinatorListener.onCoordinatorStop();

assertEquals(State.STOPPED, streamProcessor.state);
assertEquals(State.STOPPED, streamProcessor.getState());
Mockito.verify(lifecycleListener).afterStop();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,163 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.samza.util;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import org.junit.Test;

import static org.junit.Assert.*;


/**
* Test class for {@link StateTransitionUtil}
*/
public class TestStateTransitionUtil {
private static final List<String> STATES = ImmutableList.of("A", "B", "C", "D", "E");
private static final Map<String, List<String>> VALID_TRANSITIONS = new ImmutableMap.Builder<String, List<String>>()
.put("A", ImmutableList.of("C", "D"))
.put("B", ImmutableList.of("D", "E"))
.put("C", ImmutableList.of("D", "A", "E"))
.put("D", ImmutableList.of("A", "B"))
.put("E", ImmutableList.of("B", "C"))
.build();

private static final Map<String, List<String>> INVALID_TRANSITIONS = new ImmutableMap.Builder<String, List<String>>()
.put("A", ImmutableList.of("A", "B", "E"))
.put("B", ImmutableList.of("A", "B", "C"))
.put("C", ImmutableList.of("B", "C"))
.put("D", ImmutableList.of("C", "D", "E"))
.put("E", ImmutableList.of("A", "D", "E"))
.build();

private static final Random RANDOM = new Random();

@Test
public void testCompareAndSet() {
AtomicReference<String> reference = new AtomicReference<>();

for (String desiredState : STATES) {
String currentState = STATES.get(RANDOM.nextInt(STATES.size()));
reference.set(currentState);

boolean actual = StateTransitionUtil.compareAndSet(reference, VALID_TRANSITIONS.get(desiredState), desiredState);
boolean expected = VALID_TRANSITIONS.get(desiredState).contains(currentState);
assertEquals(expected, actual);
}
}

@Test
public void testCompareNotInAndSet() {
AtomicReference<String> reference = new AtomicReference<>();

for (String desiredState : STATES) {
String currentState = STATES.get(RANDOM.nextInt(STATES.size()));
reference.set(currentState);

boolean actual = StateTransitionUtil.compareNotInAndSet(reference, INVALID_TRANSITIONS.get(desiredState), desiredState);
boolean expected = !INVALID_TRANSITIONS.get(desiredState).contains(currentState);
assertEquals(expected, actual);
}
}

@Test
public void testCompareAndSetWithBarrierTimeout() {
String currentState = STATES.get(RANDOM.nextInt(STATES.size()));
String desiredState = STATES.get(RANDOM.nextInt(STATES.size()));
AtomicReference<String> reference = new AtomicReference<>(currentState);

final CountDownLatch barrier = new CountDownLatch(1);
Duration timeout = Duration.ofMillis(RANDOM.nextInt(1000));
boolean transitionResult = StateTransitionUtil.compareAndSetWithBarrier(reference, currentState, desiredState, barrier, timeout);

assertFalse(transitionResult);
}

@Test
public void testCompareAndSetWithBarrierAlreadyCompleted() {
String currentState = STATES.get(RANDOM.nextInt(STATES.size()));
String desiredState = STATES.get(RANDOM.nextInt(STATES.size()));
AtomicReference<String> reference = new AtomicReference<>(currentState);

final CountDownLatch barrier = new CountDownLatch(0);
Duration timeout = Duration.ofMillis(RANDOM.nextInt(1000));
boolean transitionResult = StateTransitionUtil.compareAndSetWithBarrier(reference, currentState, desiredState, barrier, timeout);

assertTrue(transitionResult);
}

@Test
public void testCompareAndSetWithBarrierCompletionWithStateChange() {
String currentState = STATES.get(RANDOM.nextInt(STATES.size()));
String desiredState = STATES.get(RANDOM.nextInt(STATES.size()));
AtomicReference<String> reference = new AtomicReference<>(currentState);

final CountDownLatch barrier = new CountDownLatch(1);
Duration timeout = Duration.ofMillis(RANDOM.nextInt(1000));

CompletableFuture.runAsync(() -> {
try {
Thread.sleep(timeout.toMillis());
} catch (InterruptedException e) {

}

for (String state : STATES) {
if (!currentState.equals(state)) {
reference.set(state);
break;
}
}
barrier.countDown();
});

boolean transitionResult = StateTransitionUtil.compareAndSetWithBarrier(reference, currentState, desiredState, barrier, Duration.ofMillis(-1));
assertFalse(transitionResult);
}

@Test
public void testCompareAndSetWithBarrierCompletion() {
String currentState = STATES.get(RANDOM.nextInt(STATES.size()));
String desiredState = STATES.get(RANDOM.nextInt(STATES.size()));
AtomicReference<String> reference = new AtomicReference<>(currentState);

final CountDownLatch barrier = new CountDownLatch(1);
Duration timeout = Duration.ofMillis(RANDOM.nextInt(1000));

CompletableFuture.runAsync(() -> {
try {
Thread.sleep(timeout.toMillis());
} catch (InterruptedException e) {

}

barrier.countDown();
});

boolean transitionResult = StateTransitionUtil.compareAndSetWithBarrier(reference, currentState, desiredState, barrier, Duration.ofMillis(-1));
assertTrue(transitionResult);
}
}
Loading