Skip to content

Commit

Permalink
refactoring functions to use source interface (apache#1649)
Browse files Browse the repository at this point in the history
* refactoring functions to use source interface

* addressing comments
  • Loading branch information
jerrypeng authored and sijie committed Apr 26, 2018
1 parent eaae9ea commit 4cfa83a
Show file tree
Hide file tree
Showing 22 changed files with 506 additions and 213 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.functions.api.Function;
import org.apache.pulsar.functions.instance.PulsarSource;
import org.apache.pulsar.functions.utils.FunctionConfig;
import org.apache.pulsar.functions.api.SerDe;
import org.apache.pulsar.functions.api.utils.DefaultSerDe;
Expand All @@ -54,6 +55,7 @@
import org.apache.pulsar.functions.shaded.io.netty.buffer.ByteBuf;
import org.apache.pulsar.functions.shaded.io.netty.buffer.ByteBufUtil;
import org.apache.pulsar.functions.shaded.io.netty.buffer.Unpooled;
import org.apache.pulsar.functions.shaded.proto.Function.ConnectorDetails;
import org.apache.pulsar.functions.shaded.proto.Function.FunctionDetails;
import org.apache.pulsar.functions.utils.Reflections;
import org.apache.pulsar.functions.utils.Utils;
Expand Down Expand Up @@ -837,6 +839,10 @@ private FunctionDetails convert(FunctionConfig functionConfig)
if (functionConfig.getInputs() != null) {
functionDetailsBuilder.setTenant(functionConfig.getTenant());
}
functionDetailsBuilder.setSource(
ConnectorDetails.newBuilder()
.setClassName(PulsarSource.class.getName())
.build());
if (functionConfig.getNamespace() != null) {
functionDetailsBuilder.setNamespace(functionConfig.getNamespace());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,12 @@ public interface Source<T> extends AutoCloseable {
* @param config initialization config
* @throws Exception IO type exceptions when opening a connector
*/
void open(final Map<String, String> config) throws Exception;
void open(final Map<String, Object> config) throws Exception;

/**
* Reads the next message from source, if one exists, and returns. This call should be non-blocking.
* If source does not have any new messages, return null immediately.
* @return next message from source or null, if no new messages are available.
* Reads the next message from source.
* If source does not have any new messages, this call should block.
* @return next message from source. The return result should never be null
* @throws Exception
*/
Record<T> read() throws Exception;
Expand Down
6 changes: 6 additions & 0 deletions pulsar-functions/instance/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,12 @@
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.pulsar</groupId>
<artifactId>pulsar-connect-core</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.bookkeeper</groupId>
<artifactId>stream-storage-java-client</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/**
* 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.instance;

import org.apache.pulsar.functions.api.SerDe;
import org.apache.pulsar.functions.api.utils.DefaultSerDe;
import org.apache.pulsar.functions.utils.Reflections;

public class InstanceUtils {
public static SerDe initializeSerDe(String serdeClassName, ClassLoader clsLoader,
Class<?> type) {
if (null == serdeClassName || serdeClassName.isEmpty()) {
return null;
} else if (serdeClassName.equals(DefaultSerDe.class.getName())) {
return initializeDefaultSerDe(type);
} else {
return Reflections.createInstance(
serdeClassName,
SerDe.class,
clsLoader);
}
}

public static SerDe initializeDefaultSerDe(Class<?> type) {
if (!DefaultSerDe.IsSupportedType(type)) {
throw new RuntimeException("Default Serializer does not support " + type);
}
return new DefaultSerDe(type);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@
import lombok.AccessLevel;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.connect.core.Source;
import org.apache.pulsar.functions.api.Function;
import org.apache.pulsar.functions.proto.InstanceCommunication;

Expand All @@ -48,11 +48,16 @@ public class JavaInstance implements AutoCloseable {
public JavaInstance(InstanceConfig config, Object userClassObject,
ClassLoader clsLoader,
PulsarClient pulsarClient,
Consumer inputConsumer) {
Source source) {
// TODO: cache logger instances by functions?
Logger instanceLog = LoggerFactory.getLogger("function-" + config.getFunctionDetails().getName());

this.context = new ContextImpl(config, instanceLog, pulsarClient, clsLoader, inputConsumer);
if (source instanceof PulsarSource) {
this.context = new ContextImpl(config, instanceLog, pulsarClient, clsLoader,
((PulsarSource) source).getInputConsumer());
} else {
this.context = null;
}

// create the functions
if (userClassObject instanceof Function) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicBoolean;

import lombok.AccessLevel;
import lombok.Getter;
Expand All @@ -49,8 +50,10 @@
import org.apache.logging.log4j.core.config.Configuration;
import org.apache.logging.log4j.core.config.LoggerConfig;
import org.apache.pulsar.client.api.MessageBuilder;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.connect.core.Record;
import org.apache.pulsar.functions.api.Function;
import org.apache.pulsar.functions.api.utils.DefaultSerDe;
import org.apache.pulsar.functions.instance.processors.MessageProcessor;
Expand Down Expand Up @@ -88,7 +91,7 @@ public class JavaInstanceRunnable implements AutoCloseable, Runnable {
@Getter
private Exception failureException;
private JavaInstance javaInstance;
private volatile boolean running = true;
private AtomicBoolean running = new AtomicBoolean(true);

@Getter(AccessLevel.PACKAGE)
private Map<String, SerDe> inputSerDe;
Expand All @@ -101,6 +104,8 @@ public class JavaInstanceRunnable implements AutoCloseable, Runnable {
// function stats
private final FunctionStats stats;

private Record currentRecord;

public JavaInstanceRunnable(InstanceConfig instanceConfig,
FunctionCacheManager fnCache,
String jarFile,
Expand Down Expand Up @@ -159,7 +164,7 @@ JavaInstance setupJavaInstance() throws Exception {
// start any log topic handler
setupLogHandler();

return new JavaInstance(instanceConfig, object, clsLoader, client, processor.getInputConsumer());
return new JavaInstance(instanceConfig, object, clsLoader, client, processor.getSource());
}

/**
Expand All @@ -169,9 +174,11 @@ JavaInstance setupJavaInstance() throws Exception {
public void run() {
try {
javaInstance = setupJavaInstance();
while (running) {
while (running.get()) {

currentRecord = processor.recieveMessage();

InputMessage currentMessage = processor.recieveMessage();
processor.postReceiveMessage(currentRecord);

// state object is per function, because we need to have the ability to know what updates
// are made in this function and ensure we only acknowledge after the state is persisted.
Expand All @@ -184,20 +191,20 @@ public void run() {
}

// process the message
Object input;
try {
input = currentMessage.getInputSerDe().deserialize(currentMessage.getActualMessage().getData());
} catch (Exception ex) {
stats.incrementDeserializationExceptions(currentMessage.getTopicName());
throw ex;
}
long processAt = System.currentTimeMillis();
stats.incrementProcessed(processAt);
addLogTopicHandler();
JavaExecutionResult result = javaInstance.handleMessage(
currentMessage.getActualMessage().getMessageId(),
currentMessage.getTopicName(),
input);
JavaExecutionResult result;
MessageId messageId = null;
String topicName = null;

if (currentRecord instanceof PulsarRecord) {
PulsarRecord pulsarRecord = (PulsarRecord) currentRecord;
messageId = pulsarRecord.getMessageId();
topicName = pulsarRecord.getTopicName();
}
result = javaInstance.handleMessage(messageId, topicName, currentRecord.getValue());

removeLogTopicHandler();

long doneProcessing = System.currentTimeMillis();
Expand All @@ -209,18 +216,25 @@ public void run() {
try {
completableFuture.join();
} catch (Exception e) {
log.error("Failed to flush the state updates of message {}", currentMessage, e);
throw e;
log.error("Failed to flush the state updates of message {}", currentRecord, e);
currentRecord.fail();
}
}
processResult(currentMessage, result, processAt, doneProcessing);
try {
processResult(currentRecord, result, processAt, doneProcessing);
} catch (Exception e) {
log.warn("Failed to process result of message {}", currentRecord, e);
currentRecord.fail();
}
}

javaInstance.close();
} catch (Exception ex) {
log.info("Uncaught exception in Java Instance", ex);
failureException = ex;
throw new RuntimeException(ex);
log.error("Uncaught exception in Java Instance", ex);
if (running.get()) {
failureException = ex;
throw new RuntimeException(ex);
}
} finally {
close();
}
}

Expand Down Expand Up @@ -286,15 +300,15 @@ private void setupStateTable() throws Exception {
this.stateTable = result(storageClient.openTable(tableName));
}

private void processResult(InputMessage msg,
private void processResult(Record srcRecord,
JavaExecutionResult result,
long startTime, long endTime) throws Exception {
if (result.getUserException() != null) {
log.info("Encountered user exception when processing message {}", msg, result.getUserException());
log.info("Encountered user exception when processing message {}", srcRecord, result.getUserException());
stats.incrementUserExceptions(result.getUserException());
throw result.getUserException();
this.currentRecord.fail();
} else if (result.getSystemException() != null) {
log.info("Encountered system exception when processing message {}", msg, result.getSystemException());
log.info("Encountered system exception when processing message {}", srcRecord, result.getSystemException());
stats.incrementSystemExceptions(result.getSystemException());
throw result.getSystemException();
} else {
Expand All @@ -308,35 +322,47 @@ private void processResult(InputMessage msg,
throw ex;
}
if (output != null) {
sendOutputMessage(msg, output);
sendOutputMessage(srcRecord, output);
} else {
processor.sendOutputMessage(msg, null);
processor.sendOutputMessage(srcRecord, null);
}
} else {
// the function doesn't produce any result or the user doesn't want the result.
processor.sendOutputMessage(msg, null);
processor.sendOutputMessage(srcRecord, null);
}
}
}

private void sendOutputMessage(InputMessage srcMsg,
private void sendOutputMessage(Record srcRecord,
byte[] output) throws Exception {
MessageBuilder msgBuilder = MessageBuilder.create()
.setContent(output)
.setProperty("__pfn_input_topic__", srcMsg.getTopicName())
.setProperty("__pfn_input_msg_id__", new String(Base64.getEncoder().encode(srcMsg.getActualMessage().getMessageId().toByteArray())));

processor.sendOutputMessage(srcMsg, msgBuilder);
MessageBuilder msgBuilder = MessageBuilder.create();
if (srcRecord instanceof PulsarRecord) {
PulsarRecord pulsarMessage = (PulsarRecord) srcRecord;
msgBuilder
.setContent(output)
.setProperty("__pfn_input_topic__", pulsarMessage.getTopicName())
.setProperty("__pfn_input_msg_id__", new String(Base64.getEncoder().encode(pulsarMessage.getMessageId().toByteArray())));
}

processor.sendOutputMessage(srcRecord, msgBuilder);
}

/**
* Stop java instance runnable
*/
public void stop() {
this.running.set(false);
}

@Override
public void close() {
if (!running) {
if (!running.get()) {
return;
}
running = false;

processor.close();
javaInstance.close();

// kill the state table
if (null != stateTable) {
Expand Down Expand Up @@ -401,39 +427,12 @@ private static void addSystemMetrics(String metricName, double value, InstanceCo
bldr.putMetrics(metricName, digest);
}

private static SerDe initializeSerDe(String serdeClassName, ClassLoader clsLoader,
Class<?>[] typeArgs, boolean inputArgs) {
if (null == serdeClassName || serdeClassName.isEmpty()) {
return null;
} else if (serdeClassName.equals(DefaultSerDe.class.getName())) {
return initializeDefaultSerDe(typeArgs, inputArgs);
} else {
return Reflections.createInstance(
serdeClassName,
SerDe.class,
clsLoader);
}
}

private static SerDe initializeDefaultSerDe(Class<?>[] typeArgs, boolean inputArgs) {
if (inputArgs) {
if (!DefaultSerDe.IsSupportedType(typeArgs[0])) {
throw new RuntimeException("Default Serializer does not support " + typeArgs[0]);
}
return new DefaultSerDe(typeArgs[0]);
} else {
if (!DefaultSerDe.IsSupportedType(typeArgs[1])) {
throw new RuntimeException("Default Serializer does not support " + typeArgs[1]);
}
return new DefaultSerDe(typeArgs[1]);
}
}

private void setupSerDe(Class<?>[] typeArgs, ClassLoader clsLoader) {

this.inputSerDe = new HashMap<>();
instanceConfig.getFunctionDetails().getCustomSerdeInputsMap().forEach((k, v) -> this.inputSerDe.put(k, initializeSerDe(v, clsLoader, typeArgs, true)));
instanceConfig.getFunctionDetails().getCustomSerdeInputsMap().forEach((k, v) -> this.inputSerDe.put(k, InstanceUtils.initializeSerDe(v, clsLoader, typeArgs[0])));
for (String topicName : instanceConfig.getFunctionDetails().getInputsList()) {
this.inputSerDe.put(topicName, initializeDefaultSerDe(typeArgs, true));
this.inputSerDe.put(topicName, InstanceUtils.initializeDefaultSerDe(typeArgs[0]));
}

if (Void.class.equals(typeArgs[0])) {
Expand All @@ -458,9 +457,9 @@ private void setupSerDe(Class<?>[] typeArgs, ClassLoader clsLoader) {
if (instanceConfig.getFunctionDetails().getOutputSerdeClassName() == null
|| instanceConfig.getFunctionDetails().getOutputSerdeClassName().isEmpty()
|| instanceConfig.getFunctionDetails().getOutputSerdeClassName().equals(DefaultSerDe.class.getName())) {
outputSerDe = initializeDefaultSerDe(typeArgs, false);
outputSerDe = InstanceUtils.initializeDefaultSerDe(typeArgs[1]);
} else {
this.outputSerDe = initializeSerDe(instanceConfig.getFunctionDetails().getOutputSerdeClassName(), clsLoader, typeArgs, false);
this.outputSerDe = InstanceUtils.initializeSerDe(instanceConfig.getFunctionDetails().getOutputSerdeClassName(), clsLoader, typeArgs[1]);
}
Class<?>[] outputSerdeTypeArgs = TypeResolver.resolveRawArguments(SerDe.class, outputSerDe.getClass());
if (outputSerDe.getClass().getName().equals(DefaultSerDe.class.getName())) {
Expand Down
Loading

0 comments on commit 4cfa83a

Please sign in to comment.