forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Moved entries filtering from consumer to dispatcher (apache#4329)
* Moved entries filtering from consumer to dispatcher * Optimize when all messages were filtered out * Fixed pending adds adding and re-added check for older consumers * Fixed reusing of thread local from different thread * Pass the redelivery tracker to consumer.sendMessages()
- Loading branch information
Showing
15 changed files
with
407 additions
and
225 deletions.
There are no files selected for viewing
96 changes: 96 additions & 0 deletions
96
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,96 @@ | ||
/** | ||
* 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.broker.service; | ||
|
||
import io.netty.buffer.ByteBuf; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
|
||
import org.apache.bookkeeper.mledger.Entry; | ||
import org.apache.bookkeeper.mledger.impl.PositionImpl; | ||
import org.apache.pulsar.common.api.Commands; | ||
import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; | ||
import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; | ||
|
||
public abstract class AbstractBaseDispatcher { | ||
|
||
protected final Subscription subscription; | ||
|
||
protected AbstractBaseDispatcher(Subscription subscription) { | ||
this.subscription = subscription; | ||
} | ||
|
||
/** | ||
* Filter messages that are being sent to a consumers. | ||
* <p> | ||
* Messages can be filtered out for multiple reasons: | ||
* <ul> | ||
* <li>Checksum or metadata corrupted | ||
* <li>Message is an internal marker | ||
* <li>Message is not meant to be delivered immediately | ||
* </ul> | ||
* | ||
* @param entries | ||
* a list of entries as read from storage | ||
* | ||
* @param batchSizes | ||
* an array where the batch size for each entry (the number of messages within an entry) is stored. This | ||
* array needs to be of at least the same size as the entries list | ||
* | ||
* @param sendMessageInfo | ||
* an object where the total size in messages and bytes will be returned back to the caller | ||
* @param subscription | ||
* the subscription object | ||
*/ | ||
public void filterEntriesForConsumer(List<Entry> entries, EntryBatchSizes batchSizes, SendMessageInfo sendMessageInfo) { | ||
int totalMessages = 0; | ||
long totalBytes = 0; | ||
|
||
for (int i = 0, entriesSize = entries.size(); i < entriesSize; i++) { | ||
Entry entry = entries.get(i); | ||
ByteBuf metadataAndPayload = entry.getDataBuffer(); | ||
PositionImpl pos = (PositionImpl) entry.getPosition(); | ||
|
||
MessageMetadata msgMetadata = Commands.peekMessageMetadata(metadataAndPayload, subscription.toString(), -1); | ||
|
||
try { | ||
if (msgMetadata == null) { | ||
// Message metadata was corrupted | ||
entries.set(i, null); | ||
entry.release(); | ||
subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual, | ||
Collections.emptyMap()); | ||
continue; | ||
} | ||
|
||
int batchSize = msgMetadata.getNumMessagesInBatch(); | ||
totalMessages += batchSize; | ||
totalBytes += metadataAndPayload.readableBytes(); | ||
batchSizes.setBatchSize(i, batchSize); | ||
} finally { | ||
msgMetadata.recycle(); | ||
} | ||
} | ||
|
||
sendMessageInfo.setTotalMessages(totalMessages); | ||
sendMessageInfo.setTotalBytes(totalBytes); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.