pulsar-client 1.13.1 → 1.13.2
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
- package/index.d.ts +7 -0
- package/package.json +2 -1
- package/pkg/linux/download-cpp-client.sh +5 -5
- package/src/Client.cc +13 -0
- package/src/Message.cc +10 -1
- package/src/Message.h +1 -0
- package/src/ProducerConfig.cc +16 -0
- package/src/ReaderConfig.cc +2 -1
- package/tests/end_to_end.test.js +115 -0
package/index.d.ts
CHANGED
|
@@ -33,6 +33,7 @@ export interface ClientConfig {
|
|
|
33
33
|
listenerName?: string;
|
|
34
34
|
log?: (level: LogLevel, file: string, line: number, message: string) => void;
|
|
35
35
|
logLevel?: LogLevel;
|
|
36
|
+
connectionTimeoutMs?: number;
|
|
36
37
|
}
|
|
37
38
|
|
|
38
39
|
export class Client {
|
|
@@ -66,6 +67,7 @@ export interface ProducerConfig {
|
|
|
66
67
|
chunkingEnabled?: boolean;
|
|
67
68
|
schema?: SchemaInfo;
|
|
68
69
|
accessMode?: ProducerAccessMode;
|
|
70
|
+
batchingType?: ProducerBatchType;
|
|
69
71
|
}
|
|
70
72
|
|
|
71
73
|
export class Producer {
|
|
@@ -162,6 +164,7 @@ export class Message {
|
|
|
162
164
|
getEventTimestamp(): number;
|
|
163
165
|
getRedeliveryCount(): number;
|
|
164
166
|
getPartitionKey(): string;
|
|
167
|
+
getOrderingKey(): string;
|
|
165
168
|
}
|
|
166
169
|
|
|
167
170
|
export class MessageId {
|
|
@@ -270,6 +273,10 @@ export type CompressionType =
|
|
|
270
273
|
'ZSTD' |
|
|
271
274
|
'SNAPPY';
|
|
272
275
|
|
|
276
|
+
export type ProducerBatchType =
|
|
277
|
+
'DefaultBatching' |
|
|
278
|
+
'KeyBasedBatching';
|
|
279
|
+
|
|
273
280
|
export type ProducerCryptoFailureAction =
|
|
274
281
|
'FAIL' |
|
|
275
282
|
'SEND';
|
package/package.json
CHANGED
|
@@ -1,6 +1,6 @@
|
|
|
1
1
|
{
|
|
2
2
|
"name": "pulsar-client",
|
|
3
|
-
"version": "1.13.
|
|
3
|
+
"version": "1.13.2",
|
|
4
4
|
"description": "Pulsar Node.js client",
|
|
5
5
|
"main": "index.js",
|
|
6
6
|
"types": "index.d.ts",
|
|
@@ -35,6 +35,7 @@
|
|
|
35
35
|
"node": ">=12.3.0"
|
|
36
36
|
},
|
|
37
37
|
"devDependencies": {
|
|
38
|
+
"@definitelytyped/utils": "^0.0.168",
|
|
38
39
|
"@seadub/clang-format-lint": "0.0.2",
|
|
39
40
|
"@types/node": "^14.14.25",
|
|
40
41
|
"clang-format": "^1.4.0",
|
|
@@ -23,14 +23,14 @@ set -e -x
|
|
|
23
23
|
ROOT_DIR=`cd $(dirname $0) && cd ../../ && pwd`
|
|
24
24
|
source $ROOT_DIR/pulsar-client-cpp.txt
|
|
25
25
|
|
|
26
|
-
if [ $USER != "root" ]; then
|
|
26
|
+
if [ "$USER" != "root" -a "$USER" != "" ]; then
|
|
27
27
|
SUDO="sudo"
|
|
28
28
|
fi
|
|
29
29
|
|
|
30
30
|
# Get the flavor of Linux
|
|
31
31
|
export $(cat /etc/*-release | grep "^ID=")
|
|
32
32
|
UNAME_ARCH=$(uname -m)
|
|
33
|
-
if [ $UNAME_ARCH == 'aarch64' ]; then
|
|
33
|
+
if [ "$UNAME_ARCH" == 'aarch64' ]; then
|
|
34
34
|
PLATFORM=arm64
|
|
35
35
|
else
|
|
36
36
|
PLATFORM=x86_64
|
|
@@ -42,18 +42,18 @@ rm -rf $ROOT_DIR/pkg/linux/tmp
|
|
|
42
42
|
mkdir $ROOT_DIR/pkg/linux/tmp
|
|
43
43
|
cd $ROOT_DIR/pkg/linux/tmp
|
|
44
44
|
|
|
45
|
-
if [ $ID == 'ubuntu' -o $ID == 'debian' ]; then
|
|
45
|
+
if [ "$ID" == 'ubuntu' -o "$ID" == 'debian' ]; then
|
|
46
46
|
curl -L -O ${CPP_CLIENT_BASE_URL}/deb-${PLATFORM}/apache-pulsar-client-dev.deb
|
|
47
47
|
$SUDO ar x apache-pulsar-client-dev.deb
|
|
48
48
|
$SUDO tar -xvf data.tar.xz
|
|
49
49
|
cp -r usr/* $ROOT_DIR/pkg/linux/pulsar-cpp/
|
|
50
50
|
|
|
51
|
-
elif [ $ID == 'alpine' ]; then
|
|
51
|
+
elif [ "$ID" == 'alpine' ]; then
|
|
52
52
|
curl -L -O ${CPP_CLIENT_BASE_URL}/apk-${PLATFORM}/${UNAME_ARCH}/apache-pulsar-client-dev-${CPP_CLIENT_VERSION}-r0.apk
|
|
53
53
|
$SUDO tar -xvf apache-pulsar-client-dev-${CPP_CLIENT_VERSION}-r0.apk
|
|
54
54
|
cp -r usr/* $ROOT_DIR/pkg/linux/pulsar-cpp/
|
|
55
55
|
|
|
56
|
-
elif [ $ID == '"centos"' -o $ID == '"rocky"' ]; then
|
|
56
|
+
elif [ "$ID" == '"centos"' -o "$ID" == '"rocky"' ]; then
|
|
57
57
|
curl -L -O ${CPP_CLIENT_BASE_URL}/rpm-${PLATFORM}/${UNAME_ARCH}/apache-pulsar-client-devel-${CPP_CLIENT_VERSION}-1.${UNAME_ARCH}.rpm
|
|
58
58
|
$SUDO rpm -i --prefix=$ROOT_DIR/pkg/linux/pulsar-cpp apache-pulsar-client-devel-${CPP_CLIENT_VERSION}-1.${UNAME_ARCH}.rpm --nodeps --force
|
|
59
59
|
|
package/src/Client.cc
CHANGED
|
@@ -26,6 +26,7 @@
|
|
|
26
26
|
#include <pulsar/c/client.h>
|
|
27
27
|
#include <pulsar/c/client_configuration.h>
|
|
28
28
|
#include <pulsar/c/result.h>
|
|
29
|
+
#include "pulsar/ClientConfiguration.h"
|
|
29
30
|
|
|
30
31
|
static const std::string CFG_SERVICE_URL = "serviceUrl";
|
|
31
32
|
static const std::string CFG_AUTH = "authentication";
|
|
@@ -42,9 +43,14 @@ static const std::string CFG_STATS_INTERVAL = "statsIntervalInSeconds";
|
|
|
42
43
|
static const std::string CFG_LOG = "log";
|
|
43
44
|
static const std::string CFG_LOG_LEVEL = "logLevel";
|
|
44
45
|
static const std::string CFG_LISTENER_NAME = "listenerName";
|
|
46
|
+
static const std::string CFG_CONNECTION_TIMEOUT = "connectionTimeoutMs";
|
|
45
47
|
|
|
46
48
|
LogCallback *Client::logCallback = nullptr;
|
|
47
49
|
|
|
50
|
+
struct _pulsar_client_configuration {
|
|
51
|
+
pulsar::ClientConfiguration conf;
|
|
52
|
+
};
|
|
53
|
+
|
|
48
54
|
void Client::SetLogHandler(const Napi::CallbackInfo &info) {
|
|
49
55
|
Napi::Env env = info.Env();
|
|
50
56
|
Napi::HandleScope scope(env);
|
|
@@ -157,6 +163,13 @@ Client::Client(const Napi::CallbackInfo &info) : Napi::ObjectWrap<Client>(info)
|
|
|
157
163
|
}
|
|
158
164
|
}
|
|
159
165
|
|
|
166
|
+
if (clientConfig.Has(CFG_CONNECTION_TIMEOUT) && clientConfig.Get(CFG_CONNECTION_TIMEOUT).IsNumber()) {
|
|
167
|
+
int32_t connectionTimeoutMs = clientConfig.Get(CFG_CONNECTION_TIMEOUT).ToNumber().Int32Value();
|
|
168
|
+
if (connectionTimeoutMs > 0) {
|
|
169
|
+
cClientConfig.get()->conf.setConnectionTimeout(connectionTimeoutMs);
|
|
170
|
+
}
|
|
171
|
+
}
|
|
172
|
+
|
|
160
173
|
if (clientConfig.Has(CFG_LISTENER_THREADS) && clientConfig.Get(CFG_LISTENER_THREADS).IsNumber()) {
|
|
161
174
|
int32_t messageListenerThreads = clientConfig.Get(CFG_LISTENER_THREADS).ToNumber().Int32Value();
|
|
162
175
|
if (messageListenerThreads > 0) {
|
package/src/Message.cc
CHANGED
|
@@ -45,7 +45,8 @@ Napi::Object Message::Init(Napi::Env env, Napi::Object exports) {
|
|
|
45
45
|
InstanceMethod("getPublishTimestamp", &Message::GetPublishTimestamp),
|
|
46
46
|
InstanceMethod("getEventTimestamp", &Message::GetEventTimestamp),
|
|
47
47
|
InstanceMethod("getRedeliveryCount", &Message::GetRedeliveryCount),
|
|
48
|
-
InstanceMethod("getPartitionKey", &Message::GetPartitionKey)
|
|
48
|
+
InstanceMethod("getPartitionKey", &Message::GetPartitionKey),
|
|
49
|
+
InstanceMethod("getOrderingKey", &Message::GetOrderingKey)});
|
|
49
50
|
|
|
50
51
|
constructor = Napi::Persistent(func);
|
|
51
52
|
constructor.SuppressDestruct();
|
|
@@ -138,6 +139,14 @@ Napi::Value Message::GetPartitionKey(const Napi::CallbackInfo &info) {
|
|
|
138
139
|
return Napi::String::New(env, pulsar_message_get_partitionKey(this->cMessage.get()));
|
|
139
140
|
}
|
|
140
141
|
|
|
142
|
+
Napi::Value Message::GetOrderingKey(const Napi::CallbackInfo &info) {
|
|
143
|
+
Napi::Env env = info.Env();
|
|
144
|
+
if (!ValidateCMessage(env)) {
|
|
145
|
+
return env.Null();
|
|
146
|
+
}
|
|
147
|
+
return Napi::String::New(env, pulsar_message_get_orderingKey(this->cMessage.get()));
|
|
148
|
+
}
|
|
149
|
+
|
|
141
150
|
bool Message::ValidateCMessage(Napi::Env env) {
|
|
142
151
|
if (this->cMessage.get()) {
|
|
143
152
|
return true;
|
package/src/Message.h
CHANGED
|
@@ -44,6 +44,7 @@ class Message : public Napi::ObjectWrap<Message> {
|
|
|
44
44
|
Napi::Value GetPublishTimestamp(const Napi::CallbackInfo &info);
|
|
45
45
|
Napi::Value GetEventTimestamp(const Napi::CallbackInfo &info);
|
|
46
46
|
Napi::Value GetPartitionKey(const Napi::CallbackInfo &info);
|
|
47
|
+
Napi::Value GetOrderingKey(const Napi::CallbackInfo &info);
|
|
47
48
|
Napi::Value GetRedeliveryCount(const Napi::CallbackInfo &info);
|
|
48
49
|
bool ValidateCMessage(Napi::Env env);
|
|
49
50
|
|
package/src/ProducerConfig.cc
CHANGED
|
@@ -19,6 +19,7 @@
|
|
|
19
19
|
#include "SchemaInfo.h"
|
|
20
20
|
#include "ProducerConfig.h"
|
|
21
21
|
#include <map>
|
|
22
|
+
#include "pulsar/ProducerConfiguration.h"
|
|
22
23
|
|
|
23
24
|
static const std::string CFG_TOPIC = "topic";
|
|
24
25
|
static const std::string CFG_PRODUCER_NAME = "producerName";
|
|
@@ -40,6 +41,11 @@ static const std::string CFG_ENCRYPTION_KEY = "encryptionKey";
|
|
|
40
41
|
static const std::string CFG_CRYPTO_FAILURE_ACTION = "cryptoFailureAction";
|
|
41
42
|
static const std::string CFG_CHUNK_ENABLED = "chunkingEnabled";
|
|
42
43
|
static const std::string CFG_ACCESS_MODE = "accessMode";
|
|
44
|
+
static const std::string CFG_BATCHING_TYPE = "batchingType";
|
|
45
|
+
|
|
46
|
+
struct _pulsar_producer_configuration {
|
|
47
|
+
pulsar::ProducerConfiguration conf;
|
|
48
|
+
};
|
|
43
49
|
|
|
44
50
|
static const std::map<std::string, pulsar_partitions_routing_mode> MESSAGE_ROUTING_MODE = {
|
|
45
51
|
{"UseSinglePartition", pulsar_UseSinglePartition},
|
|
@@ -71,6 +77,11 @@ static std::map<std::string, pulsar_producer_access_mode> PRODUCER_ACCESS_MODE =
|
|
|
71
77
|
{"ExclusiveWithFencing", pulsar_ProducerAccessModeExclusiveWithFencing},
|
|
72
78
|
};
|
|
73
79
|
|
|
80
|
+
static std::map<std::string, pulsar::ProducerConfiguration::BatchingType> PRODUCER_BATCHING_TYPE = {
|
|
81
|
+
{"DefaultBatching", pulsar::ProducerConfiguration::DefaultBatching},
|
|
82
|
+
{"KeyBasedBatching", pulsar::ProducerConfiguration::KeyBasedBatching},
|
|
83
|
+
};
|
|
84
|
+
|
|
74
85
|
ProducerConfig::ProducerConfig(const Napi::Object& producerConfig) : topic("") {
|
|
75
86
|
this->cProducerConfig = std::shared_ptr<pulsar_producer_configuration_t>(
|
|
76
87
|
pulsar_producer_configuration_create(), pulsar_producer_configuration_free);
|
|
@@ -208,6 +219,11 @@ ProducerConfig::ProducerConfig(const Napi::Object& producerConfig) : topic("") {
|
|
|
208
219
|
pulsar_producer_configuration_set_access_mode(this->cProducerConfig.get(),
|
|
209
220
|
PRODUCER_ACCESS_MODE.at(accessMode));
|
|
210
221
|
}
|
|
222
|
+
|
|
223
|
+
std::string batchingType = producerConfig.Get(CFG_BATCHING_TYPE).ToString().Utf8Value();
|
|
224
|
+
if (PRODUCER_BATCHING_TYPE.count(batchingType)) {
|
|
225
|
+
this->cProducerConfig.get()->conf.setBatchingType(PRODUCER_BATCHING_TYPE.at(batchingType));
|
|
226
|
+
}
|
|
211
227
|
}
|
|
212
228
|
|
|
213
229
|
ProducerConfig::~ProducerConfig() {}
|
package/src/ReaderConfig.cc
CHANGED
|
@@ -72,7 +72,8 @@ ReaderConfig::ReaderConfig(const Napi::Object &readerConfig, pulsar_reader_liste
|
|
|
72
72
|
std::string subscriptionRolePrefix =
|
|
73
73
|
readerConfig.Get(CFG_SUBSCRIPTION_ROLE_PREFIX).ToString().Utf8Value();
|
|
74
74
|
if (!subscriptionRolePrefix.empty())
|
|
75
|
-
|
|
75
|
+
pulsar_reader_configuration_set_subscription_role_prefix(this->cReaderConfig.get(),
|
|
76
|
+
subscriptionRolePrefix.c_str());
|
|
76
77
|
}
|
|
77
78
|
|
|
78
79
|
if (readerConfig.Has(CFG_READ_COMPACTED) && readerConfig.Get(CFG_READ_COMPACTED).IsBoolean()) {
|
package/tests/end_to_end.test.js
CHANGED
|
@@ -32,6 +32,7 @@ const Pulsar = require('../index');
|
|
|
32
32
|
serviceUrl,
|
|
33
33
|
tlsTrustCertsFilePath: `${__dirname}/certificate/server.crt`,
|
|
34
34
|
operationTimeoutSeconds: 30,
|
|
35
|
+
connectionTimeoutMs: 20000,
|
|
35
36
|
listenerName,
|
|
36
37
|
});
|
|
37
38
|
|
|
@@ -1329,4 +1330,118 @@ const Pulsar = require('../index');
|
|
|
1329
1330
|
await client.close();
|
|
1330
1331
|
});
|
|
1331
1332
|
});
|
|
1333
|
+
describe('KeyBasedBatchingTest', () => {
|
|
1334
|
+
let client;
|
|
1335
|
+
let producer;
|
|
1336
|
+
let consumer;
|
|
1337
|
+
let topicName;
|
|
1338
|
+
|
|
1339
|
+
beforeAll(async () => {
|
|
1340
|
+
client = new Pulsar.Client({
|
|
1341
|
+
serviceUrl: 'pulsar://localhost:6650',
|
|
1342
|
+
});
|
|
1343
|
+
});
|
|
1344
|
+
|
|
1345
|
+
afterAll(async () => {
|
|
1346
|
+
await client.close();
|
|
1347
|
+
});
|
|
1348
|
+
|
|
1349
|
+
beforeEach(async () => {
|
|
1350
|
+
topicName = `KeyBasedBatchingTest-${Date.now()}`;
|
|
1351
|
+
});
|
|
1352
|
+
|
|
1353
|
+
afterEach(async () => {
|
|
1354
|
+
if (producer) await producer.close();
|
|
1355
|
+
if (consumer) await consumer.close();
|
|
1356
|
+
});
|
|
1357
|
+
|
|
1358
|
+
const initProducer = async (maxMessages) => {
|
|
1359
|
+
producer = await client.createProducer({
|
|
1360
|
+
topic: topicName,
|
|
1361
|
+
batchingEnabled: true,
|
|
1362
|
+
batchingMaxMessages: maxMessages,
|
|
1363
|
+
batchingType: 'KeyBasedBatching',
|
|
1364
|
+
batchingMaxPublishDelayMs: 3600 * 1000,
|
|
1365
|
+
});
|
|
1366
|
+
};
|
|
1367
|
+
|
|
1368
|
+
const initConsumer = async () => {
|
|
1369
|
+
consumer = await client.subscribe({
|
|
1370
|
+
topic: topicName,
|
|
1371
|
+
subscription: 'SubscriptionName',
|
|
1372
|
+
subscriptionType: 'Exclusive',
|
|
1373
|
+
});
|
|
1374
|
+
};
|
|
1375
|
+
|
|
1376
|
+
const receiveAndAck = async () => {
|
|
1377
|
+
const msg = await consumer.receive();
|
|
1378
|
+
await consumer.acknowledge(msg);
|
|
1379
|
+
return msg;
|
|
1380
|
+
};
|
|
1381
|
+
|
|
1382
|
+
test('testSequenceId', async () => {
|
|
1383
|
+
await initProducer(6);
|
|
1384
|
+
await initConsumer();
|
|
1385
|
+
|
|
1386
|
+
// 0. Send 6 messages, use different keys and order
|
|
1387
|
+
producer.send({ data: Buffer.from('0'), partitionKey: 'A' });
|
|
1388
|
+
producer.send({ data: Buffer.from('1'), partitionKey: 'B' });
|
|
1389
|
+
producer.send({ data: Buffer.from('2'), partitionKey: 'C' });
|
|
1390
|
+
producer.send({ data: Buffer.from('3'), partitionKey: 'B' });
|
|
1391
|
+
producer.send({ data: Buffer.from('4'), partitionKey: 'C' });
|
|
1392
|
+
producer.send({ data: Buffer.from('5'), partitionKey: 'A' });
|
|
1393
|
+
await producer.flush();
|
|
1394
|
+
|
|
1395
|
+
// 1. Receive all messages
|
|
1396
|
+
const received = [];
|
|
1397
|
+
for (let i = 0; i < 6; i += 1) {
|
|
1398
|
+
const msg = await receiveAndAck();
|
|
1399
|
+
received.push({
|
|
1400
|
+
key: msg.getPartitionKey().toString(),
|
|
1401
|
+
value: msg.getData().toString(),
|
|
1402
|
+
});
|
|
1403
|
+
}
|
|
1404
|
+
|
|
1405
|
+
// 2. Verify message order (based on key dictionary order)
|
|
1406
|
+
const expected = [
|
|
1407
|
+
{ key: 'B', value: '1' },
|
|
1408
|
+
{ key: 'B', value: '3' },
|
|
1409
|
+
{ key: 'C', value: '2' },
|
|
1410
|
+
{ key: 'C', value: '4' },
|
|
1411
|
+
{ key: 'A', value: '0' },
|
|
1412
|
+
{ key: 'A', value: '5' },
|
|
1413
|
+
];
|
|
1414
|
+
|
|
1415
|
+
expect(received).toEqual(expected);
|
|
1416
|
+
});
|
|
1417
|
+
|
|
1418
|
+
test('testOrderingKeyPriority', async () => {
|
|
1419
|
+
await initProducer(3);
|
|
1420
|
+
await initConsumer();
|
|
1421
|
+
|
|
1422
|
+
// 1. Send 3 messages to verify orderingKey takes precedence over partitionKey
|
|
1423
|
+
producer.send({
|
|
1424
|
+
data: Buffer.from('0'),
|
|
1425
|
+
orderingKey: 'A',
|
|
1426
|
+
partitionKey: 'B',
|
|
1427
|
+
});
|
|
1428
|
+
producer.send({ data: Buffer.from('2'), orderingKey: 'B' });
|
|
1429
|
+
producer.send({ data: Buffer.from('1'), orderingKey: 'A' });
|
|
1430
|
+
await producer.flush();
|
|
1431
|
+
|
|
1432
|
+
// 2. Receive messages and verify their order and keys
|
|
1433
|
+
const msg1 = await receiveAndAck();
|
|
1434
|
+
expect(msg1.getData().toString()).toBe('2');
|
|
1435
|
+
expect(msg1.getOrderingKey().toString()).toBe('B');
|
|
1436
|
+
|
|
1437
|
+
const msg2 = await receiveAndAck();
|
|
1438
|
+
expect(msg2.getData().toString()).toBe('0');
|
|
1439
|
+
expect(msg2.getOrderingKey()).toBe('A');
|
|
1440
|
+
expect(msg2.getPartitionKey()).toBe('B');
|
|
1441
|
+
|
|
1442
|
+
const msg3 = await receiveAndAck();
|
|
1443
|
+
expect(msg3.getData().toString()).toBe('1');
|
|
1444
|
+
expect(msg3.getOrderingKey().toString()).toBe('A');
|
|
1445
|
+
});
|
|
1446
|
+
});
|
|
1332
1447
|
})();
|