Skip to content

Commit fad925f

Browse files
[improve][fn] Implement pip 401: Support set batching configurations for Pulsar Functions&Sources (#23860)
1 parent 114aaf0 commit fad925f

File tree

17 files changed

+674
-11
lines changed

17 files changed

+674
-11
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,41 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.pulsar.common.functions;
20+
21+
import lombok.AllArgsConstructor;
22+
import lombok.Builder;
23+
import lombok.Data;
24+
import lombok.EqualsAndHashCode;
25+
import lombok.NoArgsConstructor;
26+
27+
@Data
28+
@Builder
29+
@NoArgsConstructor
30+
@AllArgsConstructor
31+
@EqualsAndHashCode
32+
public class BatchingConfig {
33+
@Builder.Default
34+
private boolean enabled = true;
35+
@Builder.Default
36+
private Integer batchingMaxPublishDelayMs = 10;
37+
private Integer roundRobinRouterBatchingPartitionSwitchFrequency;
38+
private Integer batchingMaxMessages;
39+
private Integer batchingMaxBytes;
40+
private String batchBuilder;
41+
}

pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/functions/ProducerConfig.java

+1
Original file line numberDiff line numberDiff line change
@@ -40,4 +40,5 @@ public class ProducerConfig {
4040
private CryptoConfig cryptoConfig;
4141
private String batchBuilder;
4242
private CompressionType compressionType;
43+
private BatchingConfig batchingConfig;
4344
}

pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java

+2
Original file line numberDiff line numberDiff line change
@@ -101,6 +101,7 @@
101101
import org.apache.pulsar.functions.source.SingleConsumerPulsarSource;
102102
import org.apache.pulsar.functions.source.SingleConsumerPulsarSourceConfig;
103103
import org.apache.pulsar.functions.source.batch.BatchSourceExecutor;
104+
import org.apache.pulsar.functions.utils.BatchingUtils;
104105
import org.apache.pulsar.functions.utils.CryptoUtils;
105106
import org.apache.pulsar.functions.utils.FunctionCommon;
106107
import org.apache.pulsar.functions.utils.io.ConnectorUtils;
@@ -1050,6 +1051,7 @@ private void setupOutput(ContextImpl contextImpl) throws Exception {
10501051
.batchBuilder(conf.getBatchBuilder())
10511052
.useThreadLocalProducers(conf.getUseThreadLocalProducers())
10521053
.cryptoConfig(CryptoUtils.convertFromSpec(conf.getCryptoSpec()))
1054+
.batchingConfig(BatchingUtils.convertFromSpec(conf.getBatchingSpec()))
10531055
.compressionType(FunctionCommon.convertFromFunctionDetailsCompressionType(
10541056
conf.getCompressionType()));
10551057
pulsarSinkConfig.setProducerConfig(builder.build());

pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerBuilderFactory.java

+29
Original file line numberDiff line numberDiff line change
@@ -116,6 +116,35 @@ public <T> ProducerBuilder<T> createProducerBuilder(String topic, Schema<T> sche
116116
builder.batcherBuilder(BatcherBuilder.DEFAULT);
117117
}
118118
}
119+
if (producerConfig.getBatchingConfig() != null) {
120+
builder.enableBatching(producerConfig.getBatchingConfig().isEnabled());
121+
if (producerConfig.getBatchingConfig().getBatchingMaxPublishDelayMs() != null
122+
&& producerConfig.getBatchingConfig().getBatchingMaxPublishDelayMs() > 0) {
123+
builder.batchingMaxPublishDelay(producerConfig.getBatchingConfig().getBatchingMaxPublishDelayMs(),
124+
TimeUnit.MILLISECONDS);
125+
}
126+
if (producerConfig.getBatchingConfig().getRoundRobinRouterBatchingPartitionSwitchFrequency() != null
127+
&& producerConfig.getBatchingConfig().getRoundRobinRouterBatchingPartitionSwitchFrequency()
128+
> 0) {
129+
builder.roundRobinRouterBatchingPartitionSwitchFrequency(
130+
producerConfig.getBatchingConfig().getRoundRobinRouterBatchingPartitionSwitchFrequency());
131+
}
132+
if (producerConfig.getBatchingConfig().getBatchingMaxMessages() != null
133+
&& producerConfig.getBatchingConfig().getBatchingMaxMessages() > 0) {
134+
builder.batchingMaxMessages(producerConfig.getBatchingConfig().getBatchingMaxMessages());
135+
}
136+
if (producerConfig.getBatchingConfig().getBatchingMaxBytes() != null
137+
&& producerConfig.getBatchingConfig().getBatchingMaxBytes() > 0) {
138+
builder.batchingMaxBytes(producerConfig.getBatchingConfig().getBatchingMaxBytes());
139+
}
140+
if (producerConfig.getBatchingConfig().getBatchBuilder() != null) {
141+
if (producerConfig.getBatchingConfig().getBatchBuilder().equals("KEY_BASED")) {
142+
builder.batcherBuilder(BatcherBuilder.KEY_BASED);
143+
} else {
144+
builder.batcherBuilder(BatcherBuilder.DEFAULT);
145+
}
146+
}
147+
}
119148
}
120149
return builder;
121150
}

pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerBuilderFactoryTest.java

+60
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import static org.mockito.ArgumentMatchers.anyString;
2626
import static org.mockito.Mockito.doReturn;
2727
import static org.mockito.Mockito.mock;
28+
import static org.mockito.Mockito.times;
2829
import static org.mockito.Mockito.verify;
2930
import static org.mockito.Mockito.verifyNoMoreInteractions;
3031
import static org.testng.Assert.assertEquals;
@@ -41,6 +42,7 @@
4142
import org.apache.pulsar.client.api.ProducerCryptoFailureAction;
4243
import org.apache.pulsar.client.api.PulsarClient;
4344
import org.apache.pulsar.client.api.Schema;
45+
import org.apache.pulsar.common.functions.BatchingConfig;
4446
import org.apache.pulsar.common.functions.CryptoConfig;
4547
import org.apache.pulsar.common.functions.ProducerConfig;
4648
import org.mockito.internal.util.MockUtil;
@@ -139,6 +141,62 @@ public void testCreateProducerBuilderWithAdvancedProducerConfig() {
139141
cryptoConfig.setCryptoKeyReaderConfig(Map.of("key", "value"));
140142
cryptoConfig.setCryptoKeyReaderClassName(TestCryptoKeyReader.class.getName());
141143
producerConfig.setCryptoConfig(cryptoConfig);
144+
BatchingConfig batchingConfig = new BatchingConfig();
145+
batchingConfig.setEnabled(true);
146+
batchingConfig.setBatchingMaxPublishDelayMs(20);
147+
batchingConfig.setBatchingMaxMessages(100);
148+
batchingConfig.setBatchingMaxBytes(-1);
149+
producerConfig.setBatchingConfig(batchingConfig);
150+
ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, producerConfig, null, null);
151+
builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName");
152+
153+
verify(pulsarClient).newProducer(Schema.STRING);
154+
verify(producerBuilder).blockIfQueueFull(true);
155+
// enableBatching will be called twice here:
156+
// the first time is called by default to keep the backward compability
157+
// the second call is called when the producerConfig and producerConfig.batchingConfig are not null
158+
verify(producerBuilder, times(2)).enableBatching(true);
159+
verify(producerBuilder).batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS);
160+
verify(producerBuilder).hashingScheme(HashingScheme.Murmur3_32Hash);
161+
verify(producerBuilder).messageRoutingMode(MessageRoutingMode.CustomPartition);
162+
verify(producerBuilder).messageRouter(FunctionResultRouter.of());
163+
verify(producerBuilder).sendTimeout(0, TimeUnit.SECONDS);
164+
verify(producerBuilder).topic("topic");
165+
verify(producerBuilder).producerName("producerName");
166+
167+
verify(producerBuilder).compressionType(CompressionType.SNAPPY);
168+
verify(producerBuilder).batcherBuilder(BatcherBuilder.KEY_BASED);
169+
verify(producerBuilder).maxPendingMessages(5000);
170+
verify(producerBuilder).maxPendingMessagesAcrossPartitions(50000);
171+
TestCryptoKeyReader lastInstance = TestCryptoKeyReader.LAST_INSTANCE;
172+
assertNotNull(lastInstance);
173+
assertEquals(lastInstance.configs, cryptoConfig.getCryptoKeyReaderConfig());
174+
verify(producerBuilder).cryptoKeyReader(lastInstance);
175+
verify(producerBuilder).cryptoFailureAction(ProducerCryptoFailureAction.FAIL);
176+
verify(producerBuilder).addEncryptionKey("key1");
177+
verify(producerBuilder).addEncryptionKey("key2");
178+
verify(producerBuilder).batchingMaxPublishDelay(20, TimeUnit.MILLISECONDS);
179+
verify(producerBuilder).batchingMaxMessages(100);
180+
verifyNoMoreInteractions(producerBuilder);
181+
}
182+
183+
@Test
184+
public void testCreateProducerBuilderWithBatchingDisabled() {
185+
ProducerConfig producerConfig = new ProducerConfig();
186+
producerConfig.setBatchBuilder("KEY_BASED");
187+
producerConfig.setCompressionType(CompressionType.SNAPPY);
188+
producerConfig.setMaxPendingMessages(5000);
189+
producerConfig.setMaxPendingMessagesAcrossPartitions(50000);
190+
CryptoConfig cryptoConfig = new CryptoConfig();
191+
cryptoConfig.setProducerCryptoFailureAction(ProducerCryptoFailureAction.FAIL);
192+
cryptoConfig.setEncryptionKeys(new String[]{"key1", "key2"});
193+
cryptoConfig.setCryptoKeyReaderConfig(Map.of("key", "value"));
194+
cryptoConfig.setCryptoKeyReaderClassName(TestCryptoKeyReader.class.getName());
195+
producerConfig.setCryptoConfig(cryptoConfig);
196+
BatchingConfig batchingConfig = new BatchingConfig();
197+
batchingConfig.setEnabled(false);
198+
batchingConfig.setBatchingMaxPublishDelayMs(0);
199+
producerConfig.setBatchingConfig(batchingConfig);
142200
ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, producerConfig, null, null);
143201
builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName");
144202
verifyCommon();
@@ -153,12 +211,14 @@ public void testCreateProducerBuilderWithAdvancedProducerConfig() {
153211
verify(producerBuilder).cryptoFailureAction(ProducerCryptoFailureAction.FAIL);
154212
verify(producerBuilder).addEncryptionKey("key1");
155213
verify(producerBuilder).addEncryptionKey("key2");
214+
verify(producerBuilder).enableBatching(false);
156215
verifyNoMoreInteractions(producerBuilder);
157216
}
158217

159218
public static class TestCryptoKeyReader implements CryptoKeyReader {
160219
static TestCryptoKeyReader LAST_INSTANCE;
161220
Map<String, Object> configs;
221+
162222
public TestCryptoKeyReader(Map<String, Object> configs) {
163223
this.configs = configs;
164224
assert LAST_INSTANCE == null;

pulsar-functions/proto/src/main/proto/Function.proto

+10
Original file line numberDiff line numberDiff line change
@@ -121,6 +121,7 @@ message ProducerSpec {
121121
CryptoSpec cryptoSpec = 4;
122122
string batchBuilder = 5;
123123
CompressionType compressionType = 6;
124+
BatchingSpec batchingSpec = 7;
124125
}
125126

126127
message CryptoSpec {
@@ -147,6 +148,15 @@ message CryptoSpec {
147148
FailureAction consumerCryptoFailureAction = 5;
148149
}
149150

151+
message BatchingSpec {
152+
bool enabled = 1;
153+
int32 batchingMaxPublishDelayMs = 2;
154+
int32 roundRobinRouterBatchingPartitionSwitchFrequency = 3;
155+
int32 batchingMaxMessages = 4;
156+
int32 batchingMaxBytes = 5;
157+
string batchBuilder = 6;
158+
}
159+
150160
message SourceSpec {
151161
string className = 1;
152162
// map in json format
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,86 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.pulsar.functions.utils;
20+
21+
import org.apache.pulsar.common.functions.BatchingConfig;
22+
import org.apache.pulsar.functions.proto.Function;
23+
24+
public final class BatchingUtils {
25+
public static Function.BatchingSpec convert(BatchingConfig config) {
26+
if (config == null) {
27+
return null;
28+
}
29+
30+
Function.BatchingSpec.Builder builder = Function.BatchingSpec.newBuilder()
31+
.setEnabled(config.isEnabled());
32+
33+
if (config.getBatchingMaxPublishDelayMs() != null && config.getBatchingMaxPublishDelayMs() > 0) {
34+
builder.setBatchingMaxPublishDelayMs(config.getBatchingMaxPublishDelayMs());
35+
}
36+
if (config.getRoundRobinRouterBatchingPartitionSwitchFrequency() != null
37+
&& config.getRoundRobinRouterBatchingPartitionSwitchFrequency() > 0) {
38+
builder.setRoundRobinRouterBatchingPartitionSwitchFrequency(
39+
config.getRoundRobinRouterBatchingPartitionSwitchFrequency());
40+
}
41+
if (config.getBatchingMaxMessages() != null && config.getBatchingMaxMessages() > 0) {
42+
builder.setBatchingMaxMessages(config.getBatchingMaxMessages());
43+
}
44+
if (config.getBatchingMaxBytes() != null && config.getBatchingMaxBytes() > 0) {
45+
builder.setBatchingMaxBytes(config.getBatchingMaxBytes());
46+
}
47+
if (config.getBatchBuilder() != null && !config.getBatchBuilder().isEmpty()) {
48+
builder.setBatchBuilder(config.getBatchBuilder());
49+
}
50+
51+
return builder.build();
52+
}
53+
54+
public static BatchingConfig convertFromSpec(Function.BatchingSpec spec) {
55+
// to keep the backward compatibility, when batchingSpec is null or empty
56+
// the batching is enabled by default, and the default max publish delay is 10ms
57+
if (spec == null || spec.toString().equals("")) {
58+
return BatchingConfig.builder()
59+
.enabled(true)
60+
.batchingMaxPublishDelayMs(10)
61+
.build();
62+
}
63+
64+
BatchingConfig.BatchingConfigBuilder builder = BatchingConfig.builder()
65+
.enabled(spec.getEnabled());
66+
67+
if (spec.getBatchingMaxPublishDelayMs() > 0) {
68+
builder.batchingMaxPublishDelayMs(spec.getBatchingMaxPublishDelayMs());
69+
}
70+
if (spec.getRoundRobinRouterBatchingPartitionSwitchFrequency() > 0) {
71+
builder.roundRobinRouterBatchingPartitionSwitchFrequency(
72+
spec.getRoundRobinRouterBatchingPartitionSwitchFrequency());
73+
}
74+
if (spec.getBatchingMaxMessages() > 0) {
75+
builder.batchingMaxMessages(spec.getBatchingMaxMessages());
76+
}
77+
if (spec.getBatchingMaxBytes() > 0) {
78+
builder.batchingMaxBytes(spec.getBatchingMaxBytes());
79+
}
80+
if (spec.getBatchBuilder() != null && !spec.getBatchBuilder().isEmpty()) {
81+
builder.batchBuilder(spec.getBatchBuilder());
82+
}
83+
84+
return builder.build();
85+
}
86+
}

pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java

+6
Original file line numberDiff line numberDiff line change
@@ -524,6 +524,9 @@ public static Function.ProducerSpec convertProducerConfigToProducerSpec(Producer
524524
if (producerConf.getBatchBuilder() != null) {
525525
builder.setBatchBuilder(producerConf.getBatchBuilder());
526526
}
527+
if (producerConf.getBatchingConfig() != null) {
528+
builder.setBatchingSpec(BatchingUtils.convert(producerConf.getBatchingConfig()));
529+
}
527530
if (producerConf.getCompressionType() != null) {
528531
builder.setCompressionType(convertFromCompressionType(producerConf.getCompressionType()));
529532
} else {
@@ -546,6 +549,9 @@ public static ProducerConfig convertProducerSpecToProducerConfig(Function.Produc
546549
if (spec.getBatchBuilder() != null) {
547550
producerConfig.setBatchBuilder(spec.getBatchBuilder());
548551
}
552+
if (spec.hasBatchingSpec()) {
553+
producerConfig.setBatchingConfig(BatchingUtils.convertFromSpec(spec.getBatchingSpec()));
554+
}
549555
producerConfig.setUseThreadLocalProducers(spec.getUseThreadLocalProducers());
550556
producerConfig.setCompressionType(convertFromFunctionDetailsCompressionType(spec.getCompressionType()));
551557
return producerConfig;
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,70 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.pulsar.functions.utils;
20+
21+
import static org.testng.Assert.*;
22+
import org.apache.pulsar.common.functions.BatchingConfig;
23+
import org.apache.pulsar.functions.proto.Function;
24+
import org.testng.annotations.Test;
25+
26+
public class BatchingUtilsTest {
27+
28+
@Test
29+
public void testConvert() {
30+
BatchingConfig config = BatchingConfig.builder()
31+
.enabled(true)
32+
.batchingMaxPublishDelayMs(30)
33+
.roundRobinRouterBatchingPartitionSwitchFrequency(10)
34+
.batchingMaxMessages(1000)
35+
.batchBuilder("DEFAULT")
36+
.build();
37+
Function.BatchingSpec spec = BatchingUtils.convert(config);
38+
assertEquals(spec.getEnabled(), true);
39+
assertEquals(spec.getBatchingMaxPublishDelayMs(), 30);
40+
assertEquals(spec.getRoundRobinRouterBatchingPartitionSwitchFrequency(), 10);
41+
assertEquals(spec.getBatchingMaxMessages(), 1000);
42+
assertEquals(spec.getBatchingMaxBytes(), 0);
43+
assertEquals(spec.getBatchBuilder(), "DEFAULT");
44+
}
45+
46+
@Test
47+
public void testConvertFromSpec() {
48+
Function.BatchingSpec spec = Function.BatchingSpec.newBuilder()
49+
.setEnabled(true)
50+
.setBatchingMaxPublishDelayMs(30)
51+
.setRoundRobinRouterBatchingPartitionSwitchFrequency(10)
52+
.setBatchingMaxMessages(1000)
53+
.setBatchBuilder("DEFAULT")
54+
.build();
55+
BatchingConfig config = BatchingUtils.convertFromSpec(spec);
56+
assertEquals(config.isEnabled(), true);
57+
assertEquals(config.getBatchingMaxPublishDelayMs().intValue(), 30);
58+
assertEquals(config.getRoundRobinRouterBatchingPartitionSwitchFrequency().intValue(), 10);
59+
assertEquals(config.getBatchingMaxMessages().intValue(), 1000);
60+
assertEquals(config.getBatchingMaxBytes(), null);
61+
assertEquals(config.getBatchBuilder(), "DEFAULT");
62+
}
63+
64+
@Test
65+
public void testConvertFromSpecFromNull() {
66+
BatchingConfig config = BatchingUtils.convertFromSpec(null);
67+
assertTrue(config.isEnabled());
68+
assertEquals(config.getBatchingMaxPublishDelayMs().intValue(), 10);
69+
}
70+
}

0 commit comments

Comments
 (0)