Browse Source
This PR main refactoring relates to : 1. serializers/deserializers used in clients - unified in a Deserializers class 2. logic for configuring ClusterResourceListeners moved to ClientUtils 3. misc refactoring of the new async consumer in preparation for upcoming Request Managers Reviewers: Jun Rao <junrao@gmail.com>pull/14171/head
Lianet Magrans
1 year ago
committed by
GitHub
21 changed files with 338 additions and 305 deletions
@ -0,0 +1,78 @@
@@ -0,0 +1,78 @@
|
||||
/* |
||||
* 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.ConsumerConfig; |
||||
import org.apache.kafka.common.KafkaException; |
||||
import org.apache.kafka.common.errors.InterruptException; |
||||
import org.apache.kafka.common.serialization.Deserializer; |
||||
import org.apache.kafka.common.utils.Utils; |
||||
|
||||
import java.util.Collections; |
||||
import java.util.Objects; |
||||
import java.util.concurrent.atomic.AtomicReference; |
||||
|
||||
public class Deserializers<K, V> implements AutoCloseable { |
||||
|
||||
public final Deserializer<K> keyDeserializer; |
||||
public final Deserializer<V> valueDeserializer; |
||||
|
||||
public Deserializers(Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer) { |
||||
this.keyDeserializer = Objects.requireNonNull(keyDeserializer, "Key deserializer provided to Deserializers should not be null"); |
||||
this.valueDeserializer = Objects.requireNonNull(valueDeserializer, "Value deserializer provided to Deserializers should not be null"); |
||||
} |
||||
|
||||
public Deserializers(ConsumerConfig config) { |
||||
this(config, null, null); |
||||
} |
||||
|
||||
@SuppressWarnings("unchecked") |
||||
public Deserializers(ConsumerConfig config, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer) { |
||||
String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG); |
||||
|
||||
if (keyDeserializer == null) { |
||||
this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); |
||||
this.keyDeserializer.configure(config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)), true); |
||||
} else { |
||||
config.ignore(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG); |
||||
this.keyDeserializer = keyDeserializer; |
||||
} |
||||
|
||||
if (valueDeserializer == null) { |
||||
this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class); |
||||
this.valueDeserializer.configure(config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)), false); |
||||
} else { |
||||
config.ignore(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG); |
||||
this.valueDeserializer = valueDeserializer; |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public void close() { |
||||
AtomicReference<Throwable> firstException = new AtomicReference<>(); |
||||
Utils.closeQuietly(keyDeserializer, "key deserializer", firstException); |
||||
Utils.closeQuietly(valueDeserializer, "value deserializer", firstException); |
||||
Throwable exception = firstException.get(); |
||||
|
||||
if (exception != null) { |
||||
if (exception instanceof InterruptException) { |
||||
throw (InterruptException) exception; |
||||
} |
||||
throw new KafkaException("Failed to close deserializers", exception); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,49 @@
@@ -0,0 +1,49 @@
|
||||
/* |
||||
* 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 java.util.ArrayList; |
||||
import java.util.Collections; |
||||
import java.util.List; |
||||
import java.util.Optional; |
||||
|
||||
/** |
||||
* {@code RequestManagers} provides a means to pass around the set of {@link RequestManager} instances in the system. |
||||
* This allows callers to both use the specific {@link RequestManager} instance, or to iterate over the list via |
||||
* the {@link #entries()} method. |
||||
*/ |
||||
public class RequestManagers { |
||||
|
||||
public final Optional<CoordinatorRequestManager> coordinatorRequestManager; |
||||
public final Optional<CommitRequestManager> commitRequestManager; |
||||
private final List<Optional<? extends RequestManager>> entries; |
||||
|
||||
public RequestManagers(Optional<CoordinatorRequestManager> coordinatorRequestManager, |
||||
Optional<CommitRequestManager> commitRequestManager) { |
||||
this.coordinatorRequestManager = coordinatorRequestManager; |
||||
this.commitRequestManager = commitRequestManager; |
||||
|
||||
List<Optional<? extends RequestManager>> list = new ArrayList<>(); |
||||
list.add(coordinatorRequestManager); |
||||
list.add(commitRequestManager); |
||||
entries = Collections.unmodifiableList(list); |
||||
} |
||||
|
||||
public List<Optional<? extends RequestManager>> entries() { |
||||
return entries; |
||||
} |
||||
} |
Loading…
Reference in new issue