DBZ-4277 Multi-batch dispatch to EventHubs
This commit is contained in:
parent
4cd5b092e9
commit
c57f793387
@ -125,26 +125,37 @@ public void handleBatch(List<ChangeEvent<Object, Object>> records,
|
||||
op.setMaximumSizeInBytes(maxBatchSize);
|
||||
}
|
||||
|
||||
for (int i = 0; i < records.size();) {
|
||||
int added = 0;
|
||||
int start = i;
|
||||
LOGGER.trace("Emitting events starting from index {}", start);
|
||||
|
||||
EventDataBatch batch = producer.createBatch(op);
|
||||
|
||||
// this loop just adds records to the batch
|
||||
for (ChangeEvent<Object, Object> record : records) {
|
||||
// this loop adds as many records to the batch as possible
|
||||
for (; i < records.size();) {
|
||||
ChangeEvent<Object, Object> record = records.get(i);
|
||||
LOGGER.trace("Received record '{}'", record.value());
|
||||
if (null == record.value()) {
|
||||
continue;
|
||||
}
|
||||
EventData eventData = null;
|
||||
|
||||
EventData eventData = null;
|
||||
if (record.value() instanceof String) {
|
||||
eventData = new EventData((String) record.value());
|
||||
}
|
||||
else if (record.value() instanceof byte[]) {
|
||||
eventData = new EventData(getBytes(record.value()));
|
||||
}
|
||||
|
||||
try {
|
||||
if (!batch.tryAdd(eventData)) {
|
||||
throw new DebeziumException("Event data was too large to fit in the batch");
|
||||
// reached the maximum allowed size for the batch
|
||||
LOGGER.trace("Maximum batch reached, dispatching {} events.", added);
|
||||
break;
|
||||
}
|
||||
added++;
|
||||
i++;
|
||||
}
|
||||
catch (IllegalArgumentException e) {
|
||||
// thrown by tryAdd if event data is null
|
||||
@ -160,7 +171,9 @@ else if (record.value() instanceof byte[]) {
|
||||
}
|
||||
}
|
||||
|
||||
if (added > 0) {
|
||||
try {
|
||||
LOGGER.trace("Sending batch of {} events to Event Hubs", added);
|
||||
producer.send(batch);
|
||||
LOGGER.trace("Sent record batch to Event Hubs");
|
||||
}
|
||||
@ -168,8 +181,10 @@ else if (record.value() instanceof byte[]) {
|
||||
throw new DebeziumException(e);
|
||||
}
|
||||
|
||||
// this loop commits each record
|
||||
for (ChangeEvent<Object, Object> record : records) {
|
||||
// this loop commits each record submitted in the event hubs batch
|
||||
LOGGER.trace("Marking records at index {} to {} as processed", start, start + added);
|
||||
for (int j = start; j < start + added; ++j) {
|
||||
ChangeEvent<Object, Object> record = records.get(j);
|
||||
try {
|
||||
committer.markProcessed(record);
|
||||
LOGGER.trace("Record marked processed");
|
||||
@ -178,6 +193,8 @@ else if (record.value() instanceof byte[]) {
|
||||
throw new DebeziumException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
committer.markBatchFinished();
|
||||
LOGGER.trace("Batch marked finished");
|
||||
|
Loading…
Reference in New Issue
Block a user