Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

SinkContext: ability to seek/pause/resume consumer for a topic #10498

Merged
merged 4 commits into from
May 18, 2021
Merged
Show file tree
Hide file tree
Changes from 2 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
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.gson.Gson;
import com.google.gson.reflect.TypeToken;
import io.prometheus.client.CollectorRegistry;
import io.prometheus.client.Summary;
import java.nio.ByteBuffer;
import java.util.Arrays;
Expand All @@ -40,6 +39,7 @@
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.api.CompressionType;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.ConsumerBuilder;
import org.apache.pulsar.client.api.HashingScheme;
import org.apache.pulsar.client.api.MessageId;
Expand All @@ -51,7 +51,9 @@
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.api.TypedMessageBuilder;
import org.apache.pulsar.client.impl.ProducerBuilderImpl;
import org.apache.pulsar.client.impl.TopicMessageIdImpl;
import org.apache.pulsar.common.functions.ExternalPulsarConfig;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.functions.api.Context;
import org.apache.pulsar.functions.api.Record;
Expand All @@ -68,8 +70,8 @@
import org.apache.pulsar.functions.secretsprovider.SecretsProvider;
import org.apache.pulsar.functions.utils.FunctionCommon;
import org.apache.pulsar.functions.utils.ProducerConfigUtils;
import org.apache.pulsar.io.core.ExtendedSourceContext;
import org.apache.pulsar.io.core.SinkContext;
import org.apache.pulsar.io.core.SourceContext;
import org.slf4j.Logger;

import static com.google.common.base.Preconditions.checkState;
Expand All @@ -79,7 +81,7 @@
* This class implements the Context interface exposed to the user.
*/
@ToString
class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable {
class ContextImpl implements Context, SinkContext, ExtendedSourceContext, AutoCloseable {
private InstanceConfig config;
private Logger logger;

Expand Down Expand Up @@ -113,6 +115,8 @@ class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable

private boolean exposePulsarAdminClientEnabled;

private java.util.function.Function<String, Optional<Consumer<?>>> getConsumerFunc;

static {
// add label to indicate user metric
userMetricsLabelNames = Arrays.copyOf(ComponentStatsManager.metricsLabelNames, ComponentStatsManager.metricsLabelNames.length + 1);
Expand Down Expand Up @@ -706,4 +710,43 @@ public void close() {
logger.warn("Failed to close producers", e);
}
}

@Override
public void seek(String topic, int partition, MessageId messageId) throws PulsarClientException {
Consumer<?> consumer = getConsumer(topic);
final MessageId msgId;
if (partition == 0) {
msgId = messageId;
} else {
TopicName topicName = TopicName.get(topic);
msgId = new TopicMessageIdImpl(
topicName.getPartition(partition).toString(), topicName.toString(), messageId);
}
consumer.seek(msgId);
}

@Override
public void pause(String topic) throws PulsarClientException {
getConsumer(topic).pause();
}

@Override
public void resume(String topic) throws PulsarClientException {
getConsumer(topic).resume();
}

@Override
public void setConsumerGetter(java.util.function.Function<String, Optional<Consumer<?>>> getConsumerFunc) {
this.getConsumerFunc = getConsumerFunc;
}

private Consumer<?> getConsumer(String topic) throws PulsarClientException {
if (getConsumerFunc == null) {
throw new PulsarClientException("Getting consumer is not supported");
}
return getConsumerFunc
.apply(topic)
.orElseThrow(() -> new PulsarClientException("Consumer for topic " + topic + " is not found"));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -26,12 +26,16 @@
import org.apache.pulsar.client.api.MessageListener;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.Reflections;
import org.apache.pulsar.io.core.ExtendedSourceContext;
import org.apache.pulsar.io.core.SourceContext;

import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.TreeMap;

import static com.google.common.base.Preconditions.checkArgument;
Expand All @@ -41,7 +45,7 @@ public class MultiConsumerPulsarSource<T> extends PushPulsarSource<T> implements

private final MultiConsumerPulsarSourceConfig pulsarSourceConfig;
private final ClassLoader functionClassLoader;
private List<Consumer<T>> inputConsumers = new LinkedList<>();
private final Map<TopicName, Consumer<T>> inputConsumers = new HashMap<>();

public MultiConsumerPulsarSource(PulsarClient pulsarClient,
MultiConsumerPulsarSourceConfig pulsarSourceConfig,
Expand Down Expand Up @@ -69,7 +73,18 @@ public void open(Map<String, Object> config, SourceContext sourceContext) throws
cb.messageListener(this);

Consumer<T> consumer = cb.subscribeAsync().join();
inputConsumers.add(consumer);
inputConsumers.put(TopicName.get(topic), consumer);
}
if (sourceContext instanceof ExtendedSourceContext) {
dlg99 marked this conversation as resolved.
Show resolved Hide resolved
((ExtendedSourceContext) sourceContext).setConsumerGetter(topicName -> {
try {
TopicName req = TopicName.get(topicName);
return Optional.ofNullable(inputConsumers.get(req));
} catch (Exception e) {
log.warn("Failed to get TopicName for {}", topicName, e);
return Optional.empty();
}
});
}
}

Expand All @@ -81,7 +96,7 @@ public void received(Consumer<T> consumer, Message<T> message) {
@Override
public void close() throws Exception {
if (inputConsumers != null ) {
inputConsumers.forEach(consumer -> {
inputConsumers.values().forEach(consumer -> {
try {
consumer.close();
} catch (PulsarClientException e) {
Expand Down Expand Up @@ -109,7 +124,7 @@ private Map<String, PulsarSourceConsumerConfig<T>> setupConsumerConfigs() throws

@VisibleForTesting
List<Consumer<T>> getInputConsumers() {
return inputConsumers;
return new LinkedList<>(inputConsumers.values());
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -24,11 +24,14 @@
import org.apache.pulsar.client.api.ConsumerBuilder;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.Reflections;
import org.apache.pulsar.functions.api.Record;
import org.apache.pulsar.io.core.ExtendedSourceContext;
import org.apache.pulsar.io.core.SourceContext;

import java.util.Map;
import java.util.Optional;

import static com.google.common.base.Preconditions.checkArgument;

Expand Down Expand Up @@ -71,6 +74,20 @@ public void open(Map<String, Object> config, SourceContext sourceContext) throws

ConsumerBuilder<T> cb = createConsumeBuilder(topic, pulsarSourceConsumerConfig);
consumer = cb.subscribeAsync().join();
if (sourceContext instanceof ExtendedSourceContext) {
((ExtendedSourceContext) sourceContext).setConsumerGetter(topicName -> {
try {
TopicName src = TopicName.get(topic);
if (src.equals(TopicName.get(topicName))) {
return Optional.of(consumer);
}
} catch (Exception e) {
eolivelli marked this conversation as resolved.
Show resolved Hide resolved
log.warn("Failed to get TopicName for {}", topicName, e);
return Optional.empty();
}
return Optional.empty();
});
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,10 @@
import java.util.concurrent.CompletableFuture;

import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.api.TypedMessageBuilder;
Expand All @@ -55,6 +58,7 @@
import org.apache.pulsar.functions.proto.Function.FunctionDetails;
import org.apache.pulsar.functions.secretsprovider.EnvironmentBasedSecretsProvider;
import org.apache.pulsar.io.core.SinkContext;
import org.mockito.Mockito;
import org.slf4j.Logger;
import org.testng.Assert;
import org.testng.annotations.BeforeMethod;
Expand Down Expand Up @@ -199,4 +203,66 @@ FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(),
pulsarAdmin);
context.getPulsarAdmin();
}

@Test
public void testUnsupportedExtendedSinkContext(){
config.setExposePulsarAdminClientEnabled(false);
context = new ContextImpl(
config,
logger,
client,
new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0],
FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(),
pulsarAdmin);
try {
context.seek("z", 0, Mockito.mock(MessageId.class));
Assert.fail("Expected exception");
} catch (PulsarClientException e) {
// pass
}
try {
context.pause("z");
Assert.fail("Expected exception");
} catch (PulsarClientException e) {
// pass
}
try {
context.resume("z");
Assert.fail("Expected exception");
} catch (PulsarClientException e) {
// pass
}
}

@Test
public void testExtendedSinkContext() throws PulsarClientException {
config.setExposePulsarAdminClientEnabled(false);
context = new ContextImpl(
config,
logger,
client,
new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0],
FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(),
pulsarAdmin);
Consumer<?> mockConsumer = Mockito.mock(Consumer.class);
context.setConsumerGetter(topic ->
topic.equalsIgnoreCase("z") ? Optional.of(mockConsumer) : Optional.empty());

context.seek("z", 0, Mockito.mock(MessageId.class));
Mockito.verify(mockConsumer, Mockito.times(1)).seek(any(MessageId.class));

context.pause("z");
Mockito.verify(mockConsumer, Mockito.times(1)).pause();

context.resume("z");
Mockito.verify(mockConsumer, Mockito.times(1)).resume();

try {
context.resume("unknown");
Assert.fail("Expected exception");
} catch (PulsarClientException e) {
// pass
}

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.functions.source;

import static com.google.common.base.Preconditions.checkArgument;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
Expand All @@ -30,8 +31,12 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiFunction;
import java.util.function.Function;

import lombok.Cleanup;
import lombok.Getter;
Expand All @@ -50,13 +55,20 @@
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.common.functions.ConsumerConfig;
import org.apache.pulsar.common.functions.FunctionConfig;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.Reflections;
import org.apache.pulsar.functions.api.Record;
import org.apache.pulsar.functions.api.SerDe;
import org.apache.pulsar.io.core.ExtendedSourceContext;
import org.apache.pulsar.io.core.SourceContext;
import org.junit.Assert;
import org.mockito.ArgumentMatcher;
import static org.testng.Assert.assertSame;

import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

Expand Down Expand Up @@ -331,4 +343,40 @@ public void testPreserveOriginalSchema(PulsarSourceConfig pulsarSourceConfig) th
Record<GenericRecord> pushed = pulsarSource.read();
assertSame(pushed.getSchema(), schema);
}

@Test(dataProvider = "sourceImpls")
public void testSetConsumerGetter(PulsarSourceConfig pulsarSourceConfig) throws Exception {
PulsarSource<GenericRecord> pulsarSource = getPulsarSource(pulsarSourceConfig);

AtomicReference<Function<String, Optional<Consumer<?>>>> getConsumerFunc = new AtomicReference<>();

ExtendedSourceContext ctxMock = Mockito.mock(ExtendedSourceContext.class);
Mockito.doAnswer(new Answer<Void>() {
public Void answer(InvocationOnMock invocation) {
getConsumerFunc.set(invocation.getArgument(0));
return null;
}
}).when(ctxMock).setConsumerGetter(any());

pulsarSource.open(new HashMap<>(), ctxMock);

Mockito.verify(ctxMock, Mockito.times(1)).setConsumerGetter(any());
Assert.assertFalse(getConsumerFunc.get().apply("UnknownTopic").isPresent());

if (pulsarSourceConfig instanceof SingleConsumerPulsarSourceConfig) {
SingleConsumerPulsarSourceConfig cfg = (SingleConsumerPulsarSourceConfig) pulsarSourceConfig;
Assert.assertTrue(getConsumerFunc.get().apply(cfg.getTopic()).isPresent());
return;
}

if (pulsarSourceConfig instanceof MultiConsumerPulsarSourceConfig) {
MultiConsumerPulsarSourceConfig cfg = (MultiConsumerPulsarSourceConfig) pulsarSourceConfig;
cfg.getTopicSchema().forEach((topic, conf) -> {
Assert.assertTrue(getConsumerFunc.get().apply(topic).isPresent());
});
return;
}

fail("Unknown config type");
}
}
Loading