forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 0
/
BatchMessageContainerImpl.java
240 lines (217 loc) · 9.79 KB
/
BatchMessageContainerImpl.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
/**
* 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.client.impl;
import com.google.common.collect.Lists;
import io.netty.buffer.ByteBuf;
import io.netty.util.ReferenceCountUtil;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import lombok.Getter;
import lombok.Setter;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.api.proto.CompressionType;
import org.apache.pulsar.common.api.proto.MessageMetadata;
import org.apache.pulsar.common.protocol.ByteBufPair;
import org.apache.pulsar.common.protocol.Commands;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Default batch message container
*
* incoming single messages:
* (k1, v1), (k2, v1), (k3, v1), (k1, v2), (k2, v2), (k3, v2), (k1, v3), (k2, v3), (k3, v3)
*
* batched into single batch message:
* [(k1, v1), (k2, v1), (k3, v1), (k1, v2), (k2, v2), (k3, v2), (k1, v3), (k2, v3), (k3, v3)]
*/
class BatchMessageContainerImpl extends AbstractBatchMessageContainer {
private MessageMetadata messageMetadata = new MessageMetadata();
// sequence id for this batch which will be persisted as a single entry by broker
@Getter
@Setter
private long lowestSequenceId = -1L;
@Getter
@Setter
private long highestSequenceId = -1L;
private ByteBuf batchedMessageMetadataAndPayload;
private List<MessageImpl<?>> messages = Lists.newArrayList();
protected SendCallback previousCallback = null;
// keep track of callbacks for individual messages being published in a batch
protected SendCallback firstCallback;
public BatchMessageContainerImpl() {
}
public BatchMessageContainerImpl(ProducerImpl<?> producer) {
this();
setProducer(producer);
}
@Override
public boolean add(MessageImpl<?> msg, SendCallback callback) {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] add message to batch, num messages in batch so far {}", topicName, producerName,
numMessagesInBatch);
}
if (++numMessagesInBatch == 1) {
try {
// some properties are common amongst the different messages in the batch, hence we just pick it up from
// the first message
messageMetadata.setSequenceId(msg.getSequenceId());
lowestSequenceId = Commands.initBatchMessageMetadata(messageMetadata, msg.getMessageBuilder());
this.firstCallback = callback;
batchedMessageMetadataAndPayload = PulsarByteBufAllocator.DEFAULT
.buffer(Math.min(maxBatchSize, ClientCnx.getMaxMessageSize()));
if (msg.getMessageBuilder().hasTxnidMostBits() && currentTxnidMostBits == -1) {
currentTxnidMostBits = msg.getMessageBuilder().getTxnidMostBits();
}
if (msg.getMessageBuilder().hasTxnidLeastBits() && currentTxnidLeastBits == -1) {
currentTxnidLeastBits = msg.getMessageBuilder().getTxnidLeastBits();
}
} catch (Throwable e) {
log.error("construct first message failed, exception is ", e);
discard(new PulsarClientException(e));
return false;
}
}
if (previousCallback != null) {
previousCallback.addCallback(msg, callback);
}
previousCallback = callback;
currentBatchSizeBytes += msg.getDataBuffer().readableBytes();
messages.add(msg);
if (lowestSequenceId == -1L) {
lowestSequenceId = msg.getSequenceId();
messageMetadata.setSequenceId(lowestSequenceId);
}
highestSequenceId = msg.getSequenceId();
ProducerImpl.LAST_SEQ_ID_PUSHED_UPDATER.getAndUpdate(producer, prev -> Math.max(prev, msg.getSequenceId()));
return isBatchFull();
}
private ByteBuf getCompressedBatchMetadataAndPayload() {
int batchWriteIndex = batchedMessageMetadataAndPayload.writerIndex();
int batchReadIndex = batchedMessageMetadataAndPayload.readerIndex();
for (int i = 0, n = messages.size(); i < n; i++) {
MessageImpl<?> msg = messages.get(i);
msg.getDataBuffer().markReaderIndex();
try {
batchedMessageMetadataAndPayload = Commands.serializeSingleMessageInBatchWithPayload(msg.getMessageBuilder(),
msg.getDataBuffer(), batchedMessageMetadataAndPayload);
} catch (Throwable th) {
// serializing batch message can corrupt the index of message and batch-message. Reset the index so,
// next iteration doesn't send corrupt message to broker.
for (int j = 0; j <= i; j++) {
MessageImpl<?> previousMsg = messages.get(j);
previousMsg.getDataBuffer().resetReaderIndex();
}
batchedMessageMetadataAndPayload.writerIndex(batchWriteIndex);
batchedMessageMetadataAndPayload.readerIndex(batchReadIndex);
throw new RuntimeException(th);
}
}
int uncompressedSize = batchedMessageMetadataAndPayload.readableBytes();
ByteBuf compressedPayload = compressor.encode(batchedMessageMetadataAndPayload);
batchedMessageMetadataAndPayload.release();
if (compressionType != CompressionType.NONE) {
messageMetadata.setCompression(compressionType);
messageMetadata.setUncompressedSize(uncompressedSize);
}
// Update the current max batch size using the uncompressed size, which is what we need in any case to
// accumulate the batch content
maxBatchSize = Math.max(maxBatchSize, uncompressedSize);
return compressedPayload;
}
@Override
public void clear() {
messages = Lists.newArrayList();
firstCallback = null;
previousCallback = null;
messageMetadata.clear();
numMessagesInBatch = 0;
currentBatchSizeBytes = 0;
lowestSequenceId = -1L;
highestSequenceId = -1L;
batchedMessageMetadataAndPayload = null;
currentTxnidMostBits = -1L;
currentTxnidLeastBits = -1L;
}
@Override
public boolean isEmpty() {
return messages.isEmpty();
}
@Override
public void discard(Exception ex) {
try {
// Need to protect ourselves from any exception being thrown in the future handler from the application
if (firstCallback != null) {
firstCallback.sendComplete(ex);
}
if (batchedMessageMetadataAndPayload != null) {
ReferenceCountUtil.safeRelease(batchedMessageMetadataAndPayload);
batchedMessageMetadataAndPayload = null;
}
} catch (Throwable t) {
log.warn("[{}] [{}] Got exception while completing the callback for msg {}:", topicName, producerName,
lowestSequenceId, t);
}
clear();
}
@Override
public boolean isMultiBatches() {
return false;
}
@Override
public OpSendMsg createOpSendMsg() throws IOException {
ByteBuf encryptedPayload = producer.encryptMessage(messageMetadata, getCompressedBatchMetadataAndPayload());
if (encryptedPayload.readableBytes() > ClientCnx.getMaxMessageSize()) {
producer.semaphoreRelease(messages.size());
discard(new PulsarClientException.InvalidMessageException(
"Message size is bigger than " + ClientCnx.getMaxMessageSize() + " bytes"));
return null;
}
messageMetadata.setNumMessagesInBatch(numMessagesInBatch);
messageMetadata.setSequenceId(lowestSequenceId);
messageMetadata.setHighestSequenceId(highestSequenceId);
if (currentTxnidMostBits != -1) {
messageMetadata.setTxnidMostBits(currentTxnidMostBits);
}
if (currentTxnidLeastBits != -1) {
messageMetadata.setTxnidLeastBits(currentTxnidLeastBits);
}
ByteBufPair cmd = producer.sendMessage(producer.producerId, messageMetadata.getSequenceId(),
messageMetadata.getHighestSequenceId(), numMessagesInBatch, messageMetadata, encryptedPayload);
OpSendMsg op = OpSendMsg.create(messages, cmd, messageMetadata.getSequenceId(),
messageMetadata.getHighestSequenceId(), firstCallback);
op.setNumMessagesInBatch(numMessagesInBatch);
op.setBatchSizeByte(currentBatchSizeBytes);
lowestSequenceId = -1L;
return op;
}
@Override
public boolean hasSameSchema(MessageImpl<?> msg) {
if (numMessagesInBatch == 0) {
return true;
}
if (!messageMetadata.hasSchemaVersion()) {
return msg.getSchemaVersion() == null;
}
return Arrays.equals(msg.getSchemaVersion(), messageMetadata.getSchemaVersion());
}
private static final Logger log = LoggerFactory.getLogger(BatchMessageContainerImpl.class);
}