mirror of https://github.com/apache/kafka.git
KAFKA-14247; Define event handler interface and events (#12663)
Add some initial interfaces to kick off https://cwiki.apache.org/confluence/display/KAFKA/Proposal%3A+Consumer+Threading+Model+Refactor. We introduce an `EventHandler` interface and a new consumer implementation which demonstrates how it will be used. Subsequent PRs will continue to flesh out the implementation. Reviewers: Guozhang Wang wangguoz@gmail.com, Jason Gustafson <jason@confluent.io>
This commit is contained in:
parent
c1f23b6c9a
commit
c3690a3b4a
|
|
@ -0,0 +1,60 @@
|
||||||
|
/*
|
||||||
|
* 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.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.EventHandler;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class interfaces the KafkaConsumer and the background thread. It allows the caller to enqueue {@link ApplicationEvent}
|
||||||
|
* to be consumed by the background thread and poll {@linkBackgroundEvent} produced by the background thread.
|
||||||
|
*/
|
||||||
|
public class DefaultEventHandler implements EventHandler {
|
||||||
|
private final BlockingQueue<ApplicationEvent> applicationEvents;
|
||||||
|
private final BlockingQueue<BackgroundEvent> backgroundEvents;
|
||||||
|
|
||||||
|
public DefaultEventHandler() {
|
||||||
|
this.applicationEvents = new LinkedBlockingQueue<>();
|
||||||
|
this.backgroundEvents = new LinkedBlockingQueue<>();
|
||||||
|
// TODO: a concreted implementation of how requests are being consumed, and how responses are being produced.
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Optional<BackgroundEvent> poll() {
|
||||||
|
return Optional.ofNullable(backgroundEvents.poll());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isEmpty() {
|
||||||
|
return backgroundEvents.isEmpty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean add(ApplicationEvent event) {
|
||||||
|
try {
|
||||||
|
return applicationEvents.add(event);
|
||||||
|
} catch (IllegalStateException e) {
|
||||||
|
// swallow the capacity restriction exception
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,126 @@
|
||||||
|
/*
|
||||||
|
* 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.kafka.clients.consumer.internals;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.Consumer;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.events.EventHandler;
|
||||||
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This prototype consumer uses the EventHandler to process application events so that the network IO can be processed in a background thread. Visit
|
||||||
|
* <a href="https://cwiki.apache.org/confluence/display/KAFKA/Proposal%3A+Consumer+Threading+Model+Refactor" >this document</a>
|
||||||
|
* for detail implementation.
|
||||||
|
*/
|
||||||
|
public abstract class PrototypeAsyncConsumer<K, V> implements Consumer<K, V> {
|
||||||
|
private final EventHandler eventHandler;
|
||||||
|
private final Time time;
|
||||||
|
|
||||||
|
public PrototypeAsyncConsumer(final Time time, final EventHandler eventHandler) {
|
||||||
|
this.time = time;
|
||||||
|
this.eventHandler = eventHandler;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* poll implementation using {@link EventHandler}.
|
||||||
|
* 1. Poll for background events. If there's a fetch response event, process the record and return it. If it is
|
||||||
|
* another type of event, process it.
|
||||||
|
* 2. Send fetches if needed.
|
||||||
|
* If the timeout expires, return an empty ConsumerRecord.
|
||||||
|
* @param timeout timeout of the poll loop
|
||||||
|
* @return ConsumerRecord. It can be empty if time timeout expires.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public ConsumerRecords<K, V> poll(final Duration timeout) {
|
||||||
|
try {
|
||||||
|
do {
|
||||||
|
if (!eventHandler.isEmpty()) {
|
||||||
|
Optional<BackgroundEvent> backgroundEvent = eventHandler.poll();
|
||||||
|
// processEvent() may process 3 types of event:
|
||||||
|
// 1. Errors
|
||||||
|
// 2. Callback Invocation
|
||||||
|
// 3. Fetch responses
|
||||||
|
// Errors will be handled or rethrown.
|
||||||
|
// Callback invocation will trigger callback function execution, which is blocking until completion.
|
||||||
|
// Successful fetch responses will be added to the completedFetches in the fetcher, which will then
|
||||||
|
// be processed in the collectFetches().
|
||||||
|
backgroundEvent.ifPresent(event -> processEvent(event, timeout));
|
||||||
|
}
|
||||||
|
// The idea here is to have the background thread sending fetches autonomously, and the fetcher
|
||||||
|
// uses the poll loop to retrieve successful fetchResponse and process them on the polling thread.
|
||||||
|
final Fetch<K, V> fetch = collectFetches();
|
||||||
|
if (!fetch.isEmpty()) {
|
||||||
|
return processFetchResults(fetch);
|
||||||
|
}
|
||||||
|
// We will wait for retryBackoffMs
|
||||||
|
} while (time.timer(timeout).notExpired());
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
return ConsumerRecords.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract void processEvent(BackgroundEvent backgroundEvent, Duration timeout);
|
||||||
|
abstract ConsumerRecords<K, V> processFetchResults(Fetch<K, V> fetch);
|
||||||
|
abstract Fetch<K, V> collectFetches();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method sends a commit event to the EventHandler and return.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void commitAsync() {
|
||||||
|
ApplicationEvent commitEvent = new CommitApplicationEvent();
|
||||||
|
eventHandler.add(commitEvent);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method sends a commit event to the EventHandler and waits for the event to finish.
|
||||||
|
* @param timeout max wait time for the blocking operation.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void commitSync(Duration timeout) {
|
||||||
|
CommitApplicationEvent commitEvent = new CommitApplicationEvent();
|
||||||
|
eventHandler.add(commitEvent);
|
||||||
|
|
||||||
|
CompletableFuture<Void> commitFuture = commitEvent.commitFuture;
|
||||||
|
try {
|
||||||
|
commitFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS);
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
throw new org.apache.kafka.common.errors.TimeoutException("timeout");
|
||||||
|
} catch (Exception e) {
|
||||||
|
// handle exception here
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A stubbed ApplicationEvent for demonstration purpose
|
||||||
|
*/
|
||||||
|
private class CommitApplicationEvent extends ApplicationEvent {
|
||||||
|
// this is the stubbed commitAsyncEvents
|
||||||
|
CompletableFuture<Void> commitFuture = new CompletableFuture<>();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,23 @@
|
||||||
|
/*
|
||||||
|
* 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.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is the abstract definition of the events created by the KafkaConsumer API
|
||||||
|
*/
|
||||||
|
abstract public class ApplicationEvent {
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,23 @@
|
||||||
|
/*
|
||||||
|
* 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.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is the abstract definition of the events created by the background thread.
|
||||||
|
*/
|
||||||
|
abstract public class BackgroundEvent {
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,46 @@
|
||||||
|
/*
|
||||||
|
* 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.kafka.clients.consumer.internals.events;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class interfaces with the KafkaConsumer and the background thread. It allows the caller to enqueue events via
|
||||||
|
* the {@code add()} method and to retrieve events via the {@code poll()} method.
|
||||||
|
*/
|
||||||
|
public interface EventHandler {
|
||||||
|
/**
|
||||||
|
* Retrieves and removes a {@link BackgroundEvent}. Returns an empty Optional instance if there is nothing.
|
||||||
|
* @return an Optional of {@link BackgroundEvent} if the value is present. Otherwise, an empty Optional.
|
||||||
|
*/
|
||||||
|
Optional<BackgroundEvent> poll();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check whether there are pending {@code BackgroundEvent} await to be consumed.
|
||||||
|
* @return true if there are no pending event
|
||||||
|
*/
|
||||||
|
boolean isEmpty();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add an {@link ApplicationEvent} to the handler. The method returns true upon successful add; otherwise returns
|
||||||
|
* false.
|
||||||
|
* @param event An {@link ApplicationEvent} created by the polling thread.
|
||||||
|
* @return true upon successful add.
|
||||||
|
*/
|
||||||
|
boolean add(ApplicationEvent event);
|
||||||
|
}
|
||||||
Loading…
Reference in New Issue