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

Update producerconfigs #5

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -45,7 +45,7 @@ public class RoundRobinPartitionMessageRouterImpl extends MessageRouterBase {
private volatile int partitionIndex = 0;

private final int startPtnIdx;
private final boolean isBatchingEnabled;
protected boolean isBatchingEnabled;
private final long partitionSwitchMs;

private final Clock clock;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@
import lombok.Data;
import lombok.EqualsAndHashCode;
import lombok.NoArgsConstructor;
import org.apache.pulsar.client.api.CompressionType;
import org.apache.pulsar.client.api.HashingScheme;
import org.apache.pulsar.client.api.MessageRoutingMode;

/**
* Configuration of the producer inside the function.
Expand All @@ -38,4 +41,60 @@ public class ProducerConfig {
private Boolean useThreadLocalProducers;
private CryptoConfig cryptoConfig;
private String batchBuilder;

public Boolean batchingDisabled;
public Boolean chunkingEnabled;
public Boolean blockIfQueueFullDisabled;
public CompressionType compressionType;
public HashingScheme hashingScheme;
public MessageRoutingMode messageRoutingMode;
public Long batchingMaxPublishDelay;
public Boolean getBatchingEnabled(){
return !this.getBatchingDisabled();
}
public Boolean getBlockIfQueueFullEnabled() { return !this.getBlockIfQueueFullDisabled();}
public ProducerConfig merge(ProducerConfig newConfig){
ProducerConfig mergedConfig = new ProducerConfig();
if(newConfig == null) {
mergedConfig = this;
}
else {
if(newConfig.getBatchingDisabled() != null){
mergedConfig.setBatchingDisabled(newConfig.getBatchingDisabled());
} else {
mergedConfig.setBatchingDisabled(this.getBatchingDisabled());
}
if(newConfig.getChunkingEnabled() != null){
mergedConfig.setChunkingEnabled(newConfig.getChunkingEnabled());
} else {
mergedConfig.setChunkingEnabled(this.getChunkingEnabled());
}
if(newConfig.getBlockIfQueueFullDisabled() != null){
mergedConfig.setBlockIfQueueFullDisabled(newConfig.getBlockIfQueueFullDisabled());
} else {
mergedConfig.setBlockIfQueueFullDisabled(this.getBlockIfQueueFullDisabled());
}
if(newConfig.getCompressionType() != null){
mergedConfig.setCompressionType(newConfig.getCompressionType());
} else {
mergedConfig.setCompressionType(this.getCompressionType());
}
if(newConfig.getHashingScheme() != null){
mergedConfig.setHashingScheme(newConfig.getHashingScheme());
} else {
mergedConfig.setHashingScheme(this.getHashingScheme());
}
if (newConfig.getMessageRoutingMode() != null) {
mergedConfig.setMessageRoutingMode(newConfig.getMessageRoutingMode());
} else {
mergedConfig.setMessageRoutingMode(this.getMessageRoutingMode());
}
if(newConfig.getBatchingMaxPublishDelay() != null){
mergedConfig.setBatchingMaxPublishDelay(newConfig.getBatchingMaxPublishDelay());
} else {
mergedConfig.setBatchingMaxPublishDelay(this.getBatchingMaxPublishDelay());
}
}
return mergedConfig;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@
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.functions.utils.functions.ProducerConfigFromProtobufConverter;
import org.apache.pulsar.io.core.SinkContext;
import org.apache.pulsar.io.core.SourceContext;
import org.slf4j.Logger;
Expand Down Expand Up @@ -480,27 +481,54 @@ private <O> Producer<O> getProducer(String pulsarName, String topicName, Schema<

if (producer == null) {

Producer<O> newProducer = ((ProducerBuilderImpl<O>) pulsar.getProducerBuilder().clone())
.schema(schema)
.blockIfQueueFull(true)
.enableBatching(true)
.batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS)
.compressionType(CompressionType.LZ4)
.hashingScheme(HashingScheme.Murmur3_32Hash) //
.messageRoutingMode(MessageRoutingMode.CustomPartition)
.messageRouter(FunctionResultRouter.of())
// set send timeout to be infinity to prevent potential deadlock with consumer
// that might happen when consumer is blocked due to unacked messages
.sendTimeout(0, TimeUnit.SECONDS)
.topic(topicName)
.properties(InstanceUtils.getProperties(componentType,
FunctionCommon.getFullyQualifiedName(
this.config.getFunctionDetails().getTenant(),
this.config.getFunctionDetails().getNamespace(),
this.config.getFunctionDetails().getName()),
this.config.getInstanceId()))
.create();

Producer<O> newProducer = null;

if (this.config.getFunctionDetails() != null && this.config.getFunctionDetails().getSink() != null &&
this.config.getFunctionDetails().getSink().getProducerSpec() != null){
Function.ProducerSpec producerSpec = this.config.getFunctionDetails().getSink().getProducerSpec();
ProducerConfigFromProtobufConverter converter = new ProducerConfigFromProtobufConverter(producerSpec);
newProducer = ((ProducerBuilderImpl<O>) pulsar.getProducerBuilder().clone())
.schema(schema)
.blockIfQueueFull(converter.getBlockIfQueueFullEnabled())
.enableBatching(converter.getBatchingEnabled())
.batchingMaxPublishDelay(converter.getBatchingMaxPublishDelay(), TimeUnit.MILLISECONDS)
.compressionType(converter.getCompressionType())
.hashingScheme(converter.getHashingScheme()) //
.messageRoutingMode(converter.getMessageRoutingMode())
.messageRouter(FunctionResultRouter.of(converter.getBatchingEnabled()))
// set send timeout to be infinity to prevent potential deadlock with consumer
// that might happen when consumer is blocked due to unacked messages
.sendTimeout(0, TimeUnit.SECONDS)
.topic(topicName)
.properties(InstanceUtils.getProperties(componentType,
FunctionCommon.getFullyQualifiedName(
this.config.getFunctionDetails().getTenant(),
this.config.getFunctionDetails().getNamespace(),
this.config.getFunctionDetails().getName()),
this.config.getInstanceId()))
.create();
} else {
newProducer = ((ProducerBuilderImpl<O>) pulsar.getProducerBuilder().clone())
.schema(schema)
.blockIfQueueFull(true)
.enableBatching(true)
.batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS)
.compressionType(CompressionType.LZ4)
.hashingScheme(HashingScheme.Murmur3_32Hash) //
.messageRoutingMode(MessageRoutingMode.CustomPartition)
.messageRouter(FunctionResultRouter.of(true))
// set send timeout to be infinity to prevent potential deadlock with consumer
// that might happen when consumer is blocked due to unacked messages
.sendTimeout(0, TimeUnit.SECONDS)
.topic(topicName)
.properties(InstanceUtils.getProperties(componentType,
FunctionCommon.getFullyQualifiedName(
this.config.getFunctionDetails().getTenant(),
this.config.getFunctionDetails().getNamespace(),
this.config.getFunctionDetails().getName()),
this.config.getInstanceId()))
.create();
}
if (pulsar.getTlPublishProducers() != null) {
pulsar.getTlPublishProducers().get().put(topicName, newProducer);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,16 +26,17 @@
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.TopicMetadata;
import org.apache.pulsar.client.impl.RoundRobinPartitionMessageRouterImpl;
import org.apache.pulsar.common.functions.ProducerConfig;

/**
* Router for routing function results.
*/
public class FunctionResultRouter extends RoundRobinPartitionMessageRouterImpl {

private static final FunctionResultRouter INSTANCE = new FunctionResultRouter();
private static final FunctionRouterClock clock = FunctionRouterClock.of();

public FunctionResultRouter() {
this(Math.abs(ThreadLocalRandom.current().nextInt()), Clock.systemUTC());
public FunctionResultRouter(boolean isBatchingEnabled) {
this(clock.getPtnIdx(), clock.getTime(), isBatchingEnabled);
}

@VisibleForTesting
Expand All @@ -48,8 +49,17 @@ public FunctionResultRouter(int startPtnIdx, Clock clock) {
clock);
}

public static FunctionResultRouter of() {
return INSTANCE;
public FunctionResultRouter(int startPtnIdx, Clock clock, boolean isBatchingEnabled) {
super(
HashingScheme.Murmur3_32Hash,
startPtnIdx,
isBatchingEnabled,
1,
clock);
}

public static FunctionResultRouter of(boolean isBatchingEnabled) {
return new FunctionResultRouter(isBatchingEnabled);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/**
* 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 lombok.Getter;

import java.time.Clock;
import java.util.concurrent.ThreadLocalRandom;

public class FunctionRouterClock {
private static final FunctionRouterClock INSTANCE = new FunctionRouterClock();

@Getter
private final int PtnIdx = Math.abs(ThreadLocalRandom.current().nextInt());

@Getter
private final Clock time = Clock.systemUTC();

public static FunctionRouterClock of(){
return INSTANCE;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@
import org.apache.pulsar.functions.source.batch.BatchSourceExecutor;
import org.apache.pulsar.functions.utils.CryptoUtils;
import org.apache.pulsar.functions.utils.FunctionCommon;
import org.apache.pulsar.functions.utils.functions.ProducerConfigFromProtobufConverter;
import org.apache.pulsar.io.core.Sink;
import org.apache.pulsar.io.core.Source;
import org.slf4j.Logger;
Expand Down Expand Up @@ -744,13 +745,10 @@ private void setupOutput(ContextImpl contextImpl) throws Exception {
pulsarSinkConfig.setSchemaProperties(sinkSpec.getSchemaPropertiesMap());

if (this.instanceConfig.getFunctionDetails().getSink().getProducerSpec() != null) {
org.apache.pulsar.functions.proto.Function.ProducerSpec conf = this.instanceConfig.getFunctionDetails().getSink().getProducerSpec();
ProducerConfig.ProducerConfigBuilder builder = ProducerConfig.builder()
.maxPendingMessages(conf.getMaxPendingMessages())
.maxPendingMessagesAcrossPartitions(conf.getMaxPendingMessagesAcrossPartitions())
.useThreadLocalProducers(conf.getUseThreadLocalProducers())
.cryptoConfig(CryptoUtils.convertFromSpec(conf.getCryptoSpec()));
pulsarSinkConfig.setProducerConfig(builder.build());
org.apache.pulsar.functions.proto.Function.ProducerSpec spec = this.instanceConfig.getFunctionDetails().getSink().getProducerSpec();
ProducerConfigFromProtobufConverter converter = new ProducerConfigFromProtobufConverter(spec);
ProducerConfig producerConfig = converter.getProducerConfig();
pulsarSinkConfig.setProducerConfig(producerConfig);
}

object = new PulsarSink(this.client, pulsarSinkConfig, this.properties, this.stats, this.functionClassLoader);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,10 +56,13 @@ public PulsarCluster(PulsarClient client, PulsarAdmin adminClient, ProducerSpec
this.client = client;
this.adminClient = adminClient;
this.topicSchema = new TopicSchema(client);
this.producerBuilder = (ProducerBuilderImpl<?>) client.newProducer().blockIfQueueFull(true).enableBatching(true)
.batchingMaxPublishDelay(1, TimeUnit.MILLISECONDS);
boolean useThreadLocalProducers = false;
if (producerSpec != null) {
this.producerBuilder = (ProducerBuilderImpl<?>) client.newProducer()
.blockIfQueueFull(!producerSpec.getBlockIfQueueFullDisabled())
.enableBatching(!producerSpec.getBatchingDisabled())
.batchingMaxPublishDelay(producerSpec.getBatchingMaxPublishDelay() == 0L ? 1L :
producerSpec.getBatchingMaxPublishDelay(), TimeUnit.MILLISECONDS);
if (producerSpec.getMaxPendingMessages() != 0) {
this.producerBuilder.maxPendingMessages(producerSpec.getMaxPendingMessages());
}
Expand All @@ -75,6 +78,12 @@ public PulsarCluster(PulsarClient client, PulsarAdmin adminClient, ProducerSpec
}
useThreadLocalProducers = producerSpec.getUseThreadLocalProducers();
}
else {
this.producerBuilder = (ProducerBuilderImpl<?>) client.newProducer()
.blockIfQueueFull(true)
.enableBatching(true)
.batchingMaxPublishDelay(1, TimeUnit.MILLISECONDS);
}
if (useThreadLocalProducers) {
tlPublishProducers = new ThreadLocal<>();
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,13 +108,6 @@ protected PulsarSinkProcessorBase(Schema schema, Crypto crypto) {
public Producer<T> createProducer(PulsarClient client, String topic, String producerName, Schema<T> schema)
throws PulsarClientException {
ProducerBuilder<T> builder = client.newProducer(schema)
.blockIfQueueFull(true)
.enableBatching(true)
.batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS)
.compressionType(CompressionType.LZ4)
.hashingScheme(HashingScheme.Murmur3_32Hash) //
.messageRoutingMode(MessageRoutingMode.CustomPartition)
.messageRouter(FunctionResultRouter.of())
// set send timeout to be infinity to prevent potential deadlock with consumer
// that might happen when consumer is blocked due to unacked messages
.sendTimeout(0, TimeUnit.SECONDS)
Expand All @@ -124,10 +117,48 @@ public Producer<T> createProducer(PulsarClient client, String topic, String prod
}
if (pulsarSinkConfig.getProducerConfig() != null) {
ProducerConfig producerConfig = pulsarSinkConfig.getProducerConfig();
if (producerConfig.getMaxPendingMessages() != 0) {
if(producerConfig.getBatchingEnabled() != null){
builder.enableBatching(producerConfig.getBatchingEnabled());
builder.messageRouter(FunctionResultRouter.of(producerConfig.getBatchingEnabled()));
} else {
builder.enableBatching(true);
builder.messageRouter(FunctionResultRouter.of(true));
}
if(producerConfig.getChunkingEnabled() != null){
builder.enableChunking(producerConfig.getChunkingEnabled());
} else {
builder.enableChunking(false);
}
if(producerConfig.getBlockIfQueueFullEnabled() != null){
builder.blockIfQueueFull(producerConfig.getBlockIfQueueFullEnabled());
} else {
builder.blockIfQueueFull(true);
}
if(producerConfig.getBatchingMaxPublishDelay() != null){
builder.batchingMaxPublishDelay(producerConfig.getBatchingMaxPublishDelay(), TimeUnit.MILLISECONDS);
} else {
builder.batchingMaxPublishDelay(10L, TimeUnit.MILLISECONDS);
}
if(producerConfig.getCompressionType() != null){
builder.compressionType(producerConfig.getCompressionType());
} else {
builder.compressionType(CompressionType.LZ4);
}
if(producerConfig.getHashingScheme() != null){
builder.hashingScheme(producerConfig.getHashingScheme());
} else {
builder.hashingScheme(HashingScheme.Murmur3_32Hash);
}
if(producerConfig.getMessageRoutingMode() != null){
builder.messageRoutingMode(producerConfig.getMessageRoutingMode());
} else {
builder.messageRoutingMode(MessageRoutingMode.CustomPartition);
}
if (producerConfig.getMaxPendingMessages() != null && producerConfig.getMaxPendingMessages() != 0) {
builder.maxPendingMessages(producerConfig.getMaxPendingMessages());
}
if (producerConfig.getMaxPendingMessagesAcrossPartitions() != 0) {
if (producerConfig.getMaxPendingMessagesAcrossPartitions() != null &&
producerConfig.getMaxPendingMessagesAcrossPartitions() != 0) {
builder.maxPendingMessagesAcrossPartitions(producerConfig.getMaxPendingMessagesAcrossPartitions());
}
if (producerConfig.getCryptoConfig() != null) {
Expand All @@ -144,6 +175,14 @@ public Producer<T> createProducer(PulsarClient client, String topic, String prod
builder.batcherBuilder(BatcherBuilder.DEFAULT);
}
}
} else {
builder.blockIfQueueFull(true)
.enableBatching(true)
.batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS)
.compressionType(CompressionType.LZ4)
.hashingScheme(HashingScheme.Murmur3_32Hash) //
.messageRoutingMode(MessageRoutingMode.CustomPartition)
.messageRouter(FunctionResultRouter.of(true));
}
return builder.properties(properties).create();
}
Expand Down
Loading