Skip to content

Commit

Permalink
[hotfix][cep] Enabled distributed cache for CEP functions
Browse files Browse the repository at this point in the history
  • Loading branch information
dawidwys committed Nov 29, 2018
1 parent b2b4bdc commit 9a747ab
Show file tree
Hide file tree
Showing 2 changed files with 9 additions and 12 deletions.
Expand Up @@ -103,6 +103,11 @@ public ClassLoader getUserCodeClassLoader() {
return runtimeContext.getUserCodeClassLoader();
}

@Override
public DistributedCache getDistributedCache() {
return runtimeContext.getDistributedCache();
}

// -----------------------------------------------------------------------------------
// Unsupported operations
// -----------------------------------------------------------------------------------
Expand Down Expand Up @@ -159,11 +164,6 @@ public <T, C> C getBroadcastVariableWithInitializer(
throw new UnsupportedOperationException("Broadcast variables are not supported.");
}

@Override
public DistributedCache getDistributedCache() {
throw new UnsupportedOperationException("Distributed cache is not supported.");
}

@Override
public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
throw new UnsupportedOperationException("State is not supported.");
Expand Down
Expand Up @@ -20,6 +20,7 @@

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.common.cache.DistributedCache;
import org.apache.flink.api.common.functions.AggregateFunction;
import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
import org.apache.flink.api.common.functions.FoldFunction;
Expand Down Expand Up @@ -134,6 +135,7 @@ private void verifyRuntimeContext(final RichFunction function) {
final String taskNameWithSubtask = "barfoo";
final ExecutionConfig executionConfig = mock(ExecutionConfig.class);
final ClassLoader userCodeClassLoader = mock(ClassLoader.class);
final DistributedCache distributedCache = mock(DistributedCache.class);

RuntimeContext mockedRuntimeContext = mock(RuntimeContext.class);

Expand All @@ -145,6 +147,7 @@ private void verifyRuntimeContext(final RichFunction function) {
when(mockedRuntimeContext.getTaskNameWithSubtasks()).thenReturn(taskNameWithSubtask);
when(mockedRuntimeContext.getExecutionConfig()).thenReturn(executionConfig);
when(mockedRuntimeContext.getUserCodeClassLoader()).thenReturn(userCodeClassLoader);
when(mockedRuntimeContext.getDistributedCache()).thenReturn(distributedCache);

function.setRuntimeContext(mockedRuntimeContext);

Expand All @@ -159,13 +162,7 @@ private void verifyRuntimeContext(final RichFunction function) {
assertEquals(taskNameWithSubtask, runtimeContext.getTaskNameWithSubtasks());
assertEquals(executionConfig, runtimeContext.getExecutionConfig());
assertEquals(userCodeClassLoader, runtimeContext.getUserCodeClassLoader());

try {
runtimeContext.getDistributedCache();
fail("Expected getDistributedCached to fail with unsupported operation exception.");
} catch (UnsupportedOperationException e) {
// expected
}
assertEquals(distributedCache, runtimeContext.getDistributedCache());

try {
runtimeContext.getState(new ValueStateDescriptor<>("foobar", Integer.class, 42));
Expand Down

0 comments on commit 9a747ab

Please sign in to comment.