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
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ public class PulsarFunctionE2ESecurityTest {
private static final String ADMIN_SUBJECT = "superUser";
private static final String ANONYMOUS_ROLE = "anonymousUser";

private static final Logger log = LoggerFactory.getLogger(PulsarFunctionE2ETest.class);
private static final Logger log = LoggerFactory.getLogger(PulsarFunctionE2ESecurityTest.class);
private String adminToken;
private String brokerServiceUrl;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1565,6 +1565,157 @@ public void testFunctionAutomaticSubCleanup() throws Exception {
assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1);
}

@Test
public void testMessageAsInputJavaNativeFunction() throws Exception {
final String namespacePortion = "io";
final String replNamespace = tenant + "/" + namespacePortion;
final String sourceTopic = "persistent://" + replNamespace + "/my-topic";
final String sinkTopic = "persistent://" + replNamespace + "/output";
final String functionName = "PulsarFunction-test";
final String subscriptionName = "test-sub";
admin.namespaces().createNamespace(replNamespace);
Set<String> clusters = Sets.newHashSet(Lists.newArrayList("use"));
admin.namespaces().setNamespaceReplicationClusters(replNamespace, clusters);

// create a producer that creates a topic at broker
Producer<String> producer = pulsarClient.newProducer(Schema.STRING).topic(sourceTopic).create();
Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING).topic(sinkTopic).subscriptionName("sub").subscribe();

String jarFilePathUrl = Utils.FILE + ":" + getClass().getClassLoader().getResource("pulsar-functions-api-examples.jar").getFile();
FunctionConfig functionConfig = new FunctionConfig();
functionConfig.setTenant(tenant);
functionConfig.setNamespace(namespacePortion);
functionConfig.setName(functionName);
functionConfig.setParallelism(1);
functionConfig.setInputs(Collections.singleton(sourceTopic));
functionConfig.setClassName("org.apache.pulsar.functions.api.examples.MessageInputJavaNativeFunction");
functionConfig.setOutput(sinkTopic);
functionConfig.setCleanupSubscription(false);
functionConfig.setRuntime(FunctionConfig.Runtime.JAVA);
functionConfig.setSubName(subscriptionName);

admin.functions().createFunctionWithUrl(functionConfig, jarFilePathUrl);
retryStrategically((test) -> {
try {
return admin.functions().getFunction(tenant, namespacePortion, functionName).getCleanupSubscription();
} catch (PulsarAdminException e) {
return false;
}
}, 5, 150);
assertFalse(admin.functions().getFunction(tenant, namespacePortion, functionName).getCleanupSubscription());

retryStrategically((test) -> {
try {
return admin.topics().getStats(sourceTopic).subscriptions.size() == 1;
} catch (PulsarAdminException e) {
return false;
}
}, 5, 150);
// validate pulsar source consumer has started on the topic
assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1);

int totalMsgs = 10;
for (int i = 0; i < totalMsgs; i++) {
String data = "val_" + i;
String key = "key_" + i;
producer.newMessage().key(key).value(data).send();
}
retryStrategically((test) -> {
try {
SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName);
return subStats.unackedMessages == 0 && subStats.msgThroughputOut == totalMsgs;
} catch (PulsarAdminException e) {
return false;
}
}, 5, 200);

for (int i = 0; i < totalMsgs; i++) {
String data = "val_" + i;
String key = "key_" + i;
Message<String> msg = consumer.receive(5, TimeUnit.SECONDS);
assertEquals(key + "-" + data, msg.getValue());
assertEquals(key, msg.getKey());
}

// delete functions
admin.functions().deleteFunction(tenant, namespacePortion, functionName);
}

@Test
public void testMessageAsInputFunction() throws Exception {
final String namespacePortion = "io";
final String replNamespace = tenant + "/" + namespacePortion;
final String sourceTopic = "persistent://" + replNamespace + "/my-topic";
final String sinkTopic = "persistent://" + replNamespace + "/output";
final String functionName = "PulsarFunction-test";
final String subscriptionName = "test-sub";
admin.namespaces().createNamespace(replNamespace);
Set<String> clusters = Sets.newHashSet(Lists.newArrayList("use"));
admin.namespaces().setNamespaceReplicationClusters(replNamespace, clusters);

// create a producer that creates a topic at broker
Producer<String> producer = pulsarClient.newProducer(Schema.STRING).topic(sourceTopic).create();
Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING).topic(sinkTopic).subscriptionName("sub").subscribe();

String jarFilePathUrl = Utils.FILE + ":" + getClass().getClassLoader().getResource("pulsar-functions-api-examples.jar").getFile();
FunctionConfig functionConfig = new FunctionConfig();
functionConfig.setTenant(tenant);
functionConfig.setNamespace(namespacePortion);
functionConfig.setName(functionName);
functionConfig.setParallelism(1);
functionConfig.setInputs(Collections.singleton(sourceTopic));
functionConfig.setClassName("org.apache.pulsar.functions.api.examples.MessageInputFunction");
functionConfig.setOutput(sinkTopic);
functionConfig.setCleanupSubscription(false);
functionConfig.setRuntime(FunctionConfig.Runtime.JAVA);
functionConfig.setSubName(subscriptionName);

admin.functions().createFunctionWithUrl(functionConfig, jarFilePathUrl);
retryStrategically((test) -> {
try {
return admin.functions().getFunction(tenant, namespacePortion, functionName).getCleanupSubscription();
} catch (PulsarAdminException e) {
return false;
}
}, 5, 150);
assertFalse(admin.functions().getFunction(tenant, namespacePortion, functionName).getCleanupSubscription());

retryStrategically((test) -> {
try {
return admin.topics().getStats(sourceTopic).subscriptions.size() == 1;
} catch (PulsarAdminException e) {
return false;
}
}, 5, 150);
// validate pulsar source consumer has started on the topic
assertEquals(admin.topics().getStats(sourceTopic).subscriptions.size(), 1);

int totalMsgs = 10;
for (int i = 0; i < totalMsgs; i++) {
String data = "val_" + i;
String key = "key_" + i;
producer.newMessage().key(key).value(data).send();
}
retryStrategically((test) -> {
try {
SubscriptionStats subStats = admin.topics().getStats(sourceTopic).subscriptions.get(subscriptionName);
return subStats.unackedMessages == 0 && subStats.msgThroughputOut == totalMsgs;
} catch (PulsarAdminException e) {
return false;
}
}, 5, 200);

for (int i = 0; i < totalMsgs; i++) {
String data = "val_" + i;
String key = "key_" + i;
Message<String> msg = consumer.receive(5, TimeUnit.SECONDS);
assertEquals(key + "-" + data, msg.getValue());
assertEquals(key, msg.getKey());
}

// delete functions
admin.functions().deleteFunction(tenant, namespacePortion, functionName);
}

public static String getPrometheusMetrics(int metricsPort) throws IOException {
StringBuilder result = new StringBuilder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@

import org.apache.pulsar.functions.api.Function;
import org.apache.pulsar.functions.api.Record;
import org.apache.pulsar.functions.source.PulsarRecord;

import java.util.Map;

Expand All @@ -39,8 +40,9 @@ public class JavaInstance implements AutoCloseable {
private final ContextImpl context;
private Function function;
private java.util.function.Function javaUtilFunction;
private boolean isMessageInput;

public JavaInstance(ContextImpl contextImpl, Object userClassObject) {
public JavaInstance(ContextImpl contextImpl, Object userClassObject, boolean isMessageInput) {

this.context = contextImpl;

Expand All @@ -50,14 +52,21 @@ public JavaInstance(ContextImpl contextImpl, Object userClassObject) {
} else {
this.javaUtilFunction = (java.util.function.Function) userClassObject;
}
this.isMessageInput = isMessageInput;
}

public JavaExecutionResult handleMessage(Record<?> record, Object input) {
public JavaExecutionResult handleMessage(Record<?> record) {
if (context != null) {
context.setCurrentMessageContext(record);
}
JavaExecutionResult executionResult = new JavaExecutionResult();
Object input;
try {
if (isMessageInput) {
input = ((PulsarRecord) record).getMessage();
} else {
input = record.getValue();
}
Object output;
if (function != null) {
output = function.process(input, context);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@
import org.apache.pulsar.functions.sink.PulsarSink;
import org.apache.pulsar.functions.sink.PulsarSinkConfig;
import org.apache.pulsar.functions.sink.PulsarSinkDisable;
import org.apache.pulsar.functions.source.PulsarRecord;
import org.apache.pulsar.functions.source.PulsarSource;
import org.apache.pulsar.functions.source.PulsarSourceConfig;
import org.apache.pulsar.functions.utils.Reflections;
Expand Down Expand Up @@ -166,6 +167,8 @@ public JavaInstanceRunnable(InstanceConfig instanceConfig,
// metrics collection especially in threaded mode
// In process mode the JavaInstanceMain will declare a CollectorRegistry and pass it down
this.collectorRegistry = collectorRegistry;


}

/**
Expand All @@ -191,6 +194,9 @@ JavaInstance setupJavaInstance(ContextImpl contextImpl) throws Exception {
throw new RuntimeException("User class must either be Function or java.util.Function");
}

// check if input is Message<T>
boolean isMessageInput = FunctionCommon.isFunctionInputMessage(object.getClass());

// start the state table
setupStateTable();
// start the output producer
Expand All @@ -200,7 +206,7 @@ JavaInstance setupJavaInstance(ContextImpl contextImpl) throws Exception {
// start any log topic handler
setupLogHandler();

return new JavaInstance(contextImpl, object);
return new JavaInstance(contextImpl, object, isMessageInput);
}

ContextImpl setupContext() {
Expand Down Expand Up @@ -254,7 +260,7 @@ public void run() {
stats.processTimeStart();

// process the message
result = javaInstance.handleMessage(currentRecord, currentRecord.getValue());
result = javaInstance.handleMessage(currentRecord);

// register end time
stats.processTimeEnd();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ public class PulsarRecord<T> implements RecordWithEncryptionContext<T> {
private final String topicName;
private final int partition;

@Getter
private final Message<T> message;

private final Runnable failFunction;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,9 @@ public class JavaInstanceTest {
public void testLambda() {
JavaInstance instance = new JavaInstance(
mock(ContextImpl.class),
(Function<String, String>) (input, context) -> input + "-lambda");
(Function<String, String>) (input, context) -> input + "-lambda", false);
String testString = "ABC123";
JavaExecutionResult result = instance.handleMessage(mock(Record.class), testString);
JavaExecutionResult result = instance.handleMessage(() -> testString);
assertNotNull(result.getResult());
assertEquals(new String(testString + "-lambda"), result.getResult());
instance.close();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/**
* 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.pulsar.functions.api.examples;

import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.functions.api.Context;
import org.apache.pulsar.functions.api.Function;


public class MessageInputFunction implements Function<Message<String>, String> {
Copy link
Member

Choose a reason for hiding this comment

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

Can you add an integration test for this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@sijie as mentioned in the description of the PR, once everyone is on board with this approach I will add the tests for this

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@sijie I have added the tests


@Override
public String process(Message<String> input, Context context) throws Exception {
return input.getKey() + "-" + input.getValue();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/**
* 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.pulsar.functions.api.examples;

import org.apache.pulsar.client.api.Message;

public class MessageInputJavaNativeFunction implements java.util.function.Function<Message<String>, String> {
@Override
public String apply(Message<String> input) {
return input.getKey() + "-" + input.getValue();
}
}
Loading