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.
Master Issue: #4 This is the request Fetch implementation. Basic logic: For each Fetch request from a partition, create and maintain a NonDurableCursor to read from backed PersistentTopic. ** changes ** - Add basic code implementation - Add unit tests.
- Loading branch information
Showing
13 changed files
with
945 additions
and
61 deletions.
There are no files selected for viewing
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
429 changes: 398 additions & 31 deletions
429
src/main/java/io/streamnative/kop/KafkaRequestHandler.java
Large diffs are not rendered by default.
Oops, something went wrong.
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
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
99 changes: 99 additions & 0 deletions
99
src/main/java/io/streamnative/kop/KafkaTopicConsumerManager.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,99 @@ | ||
/** | ||
* Licensed 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 io.streamnative.kop; | ||
|
||
import io.streamnative.kop.utils.MessageIdUtils; | ||
import io.streamnative.kop.utils.ReflectionUtils; | ||
import java.lang.reflect.Method; | ||
import java.util.UUID; | ||
import java.util.concurrent.CompletableFuture; | ||
import lombok.Getter; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.bookkeeper.mledger.ManagedCursor; | ||
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; | ||
import org.apache.bookkeeper.mledger.impl.PositionImpl; | ||
import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap; | ||
import org.apache.commons.codec.digest.DigestUtils; | ||
import org.apache.pulsar.broker.service.persistent.PersistentTopic; | ||
|
||
/** | ||
* KafkaTopicConsumerManager manages a topic and its related offset cursor. | ||
* Each cursor is trying to track the read from a consumer client. | ||
*/ | ||
@Slf4j | ||
public class KafkaTopicConsumerManager { | ||
|
||
private final PersistentTopic topic; | ||
@Getter | ||
private final ConcurrentLongHashMap<CompletableFuture<ManagedCursor>> consumers; | ||
|
||
KafkaTopicConsumerManager(PersistentTopic topic) { | ||
this.topic = topic; | ||
this.consumers = new ConcurrentLongHashMap<>(); | ||
} | ||
|
||
public CompletableFuture<ManagedCursor> remove(long offset) { | ||
CompletableFuture<ManagedCursor> cursor = consumers.remove(offset); | ||
if (cursor != null) { | ||
if (log.isDebugEnabled()) { | ||
log.debug("Get cursor for offset: {} in cache", offset); | ||
} | ||
return cursor; | ||
} | ||
|
||
// handle null remove. | ||
cursor = new CompletableFuture<>(); | ||
CompletableFuture<ManagedCursor> oldCursor = consumers.putIfAbsent(offset, cursor); | ||
if (oldCursor != null) { | ||
// added by other thread while creating. | ||
return remove(offset); | ||
} | ||
|
||
String cursorName = "kop-consumer-cursor-" + topic.getName() + "-" + offset + "-" | ||
+ DigestUtils.sha1Hex(UUID.randomUUID().toString()).substring(0, 10); | ||
|
||
PositionImpl position = MessageIdUtils.getPosition(offset); | ||
|
||
try { | ||
// get previous position, because NonDurableCursor is read from next position. | ||
ManagedLedgerImpl ledger = (ManagedLedgerImpl) topic.getManagedLedger(); | ||
Method getPreviousPosition = ReflectionUtils | ||
.setMethodAccessible(ledger, "getPreviousPosition", PositionImpl.class); | ||
PositionImpl previous = (PositionImpl) getPreviousPosition.invoke(ledger, position); | ||
|
||
if (log.isDebugEnabled()) { | ||
log.debug("Create cursor {} for offset: {}. position: {}, previousPosition: {}", | ||
cursorName, offset, position, previous); | ||
} | ||
|
||
cursor.complete(topic.getManagedLedger().newNonDurableCursor(previous, cursorName)); | ||
} catch (Exception e) { | ||
log.error("Failed create nonDurable cursor for topic {} position: {}.", topic, position, e); | ||
cursor.completeExceptionally(e); | ||
} | ||
|
||
return remove(offset); | ||
} | ||
|
||
|
||
// once entry read complete, add new offset back. | ||
public void add(long offset, CompletableFuture<ManagedCursor> cursor) { | ||
CompletableFuture<ManagedCursor> oldCursor = consumers.putIfAbsent(offset, cursor); | ||
|
||
if (log.isDebugEnabled()) { | ||
log.debug("Add cursor {} for offset: {}. oldCursor: {}", cursor.join().getName(), offset, oldCursor); | ||
} | ||
} | ||
|
||
} |
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,58 @@ | ||
/** | ||
* Licensed 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 io.streamnative.kop; | ||
|
||
import java.util.concurrent.CompletableFuture; | ||
import lombok.Getter; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.bookkeeper.util.collections.ConcurrentOpenHashMap; | ||
import org.apache.pulsar.broker.service.BrokerService; | ||
import org.apache.pulsar.broker.service.persistent.PersistentTopic; | ||
|
||
/** | ||
* KafkaTopicManager manages a Map of topic to KafkaTopicConsumerManager. | ||
* For each topic, there is a KafkaTopicConsumerManager, which manages a topic and its related offset cursor. | ||
*/ | ||
@Slf4j | ||
public class KafkaTopicManager { | ||
|
||
private final BrokerService service; | ||
@Getter | ||
private final ConcurrentOpenHashMap<String, CompletableFuture<KafkaTopicConsumerManager>> topics; | ||
|
||
KafkaTopicManager(BrokerService service) { | ||
this.service = service; | ||
topics = new ConcurrentOpenHashMap<>(); | ||
} | ||
|
||
public CompletableFuture<KafkaTopicConsumerManager> getTopicConsumerManager(String topicName) { | ||
return topics.computeIfAbsent( | ||
topicName, | ||
t -> service | ||
.getTopic(topicName, true) | ||
.thenApply(t2 -> { | ||
if (log.isDebugEnabled()) { | ||
log.debug("Call getTopicConsumerManager for {}, and create KafkaTopicConsumerManager.", | ||
topicName); | ||
} | ||
return new KafkaTopicConsumerManager((PersistentTopic) t2.get()); | ||
}) | ||
.exceptionally(ex -> { | ||
log.error("Failed to getTopicConsumerManager {}. exception:", | ||
topicName, ex); | ||
return null; | ||
}) | ||
); | ||
} | ||
} |
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.