Add km module kafka

This commit is contained in:
leewei
2023-02-14 16:27:47 +08:00
parent 229140f067
commit 0b8160a714
4039 changed files with 718112 additions and 46204 deletions

1
clients/.gitignore vendored Normal file
View File

@@ -0,0 +1 @@
/bin/

View File

@@ -0,0 +1,56 @@
/*
* 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;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKey;
import org.apache.kafka.common.protocol.ApiKeys;
/**
* Represents the min version and max version of an api key.
*
* NOTE: This class is intended for INTERNAL usage only within Kafka.
*/
public class ApiVersion {
public final short apiKey;
public final short minVersion;
public final short maxVersion;
public ApiVersion(ApiKeys apiKey) {
this(apiKey.id, apiKey.oldestVersion(), apiKey.latestVersion());
}
public ApiVersion(short apiKey, short minVersion, short maxVersion) {
this.apiKey = apiKey;
this.minVersion = minVersion;
this.maxVersion = maxVersion;
}
public ApiVersion(ApiVersionsResponseKey apiVersionsResponseKey) {
this.apiKey = apiVersionsResponseKey.apiKey();
this.minVersion = apiVersionsResponseKey.minVersion();
this.maxVersion = apiVersionsResponseKey.maxVersion();
}
@Override
public String toString() {
return "ApiVersion(" +
"apiKey=" + apiKey +
", minVersion=" + minVersion +
", maxVersion= " + maxVersion +
")";
}
}

View File

@@ -0,0 +1,66 @@
/*
* 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;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.requests.ProduceRequest;
import java.util.HashMap;
import java.util.Map;
/**
* Maintains node api versions for access outside of NetworkClient (which is where the information is derived).
* The pattern is akin to the use of {@link Metadata} for topic metadata.
*
* NOTE: This class is intended for INTERNAL usage only within Kafka.
*/
public class ApiVersions {
private final Map<String, NodeApiVersions> nodeApiVersions = new HashMap<>();
private byte maxUsableProduceMagic = RecordBatch.CURRENT_MAGIC_VALUE;
public synchronized void update(String nodeId, NodeApiVersions nodeApiVersions) {
this.nodeApiVersions.put(nodeId, nodeApiVersions);
this.maxUsableProduceMagic = computeMaxUsableProduceMagic();
}
public synchronized void remove(String nodeId) {
this.nodeApiVersions.remove(nodeId);
this.maxUsableProduceMagic = computeMaxUsableProduceMagic();
}
public synchronized NodeApiVersions get(String nodeId) {
return this.nodeApiVersions.get(nodeId);
}
private byte computeMaxUsableProduceMagic() {
// use a magic version which is supported by all brokers to reduce the chance that
// we will need to convert the messages when they are ready to be sent.
byte maxUsableMagic = RecordBatch.CURRENT_MAGIC_VALUE;
for (NodeApiVersions versions : this.nodeApiVersions.values()) {
byte nodeMaxUsableMagic = ProduceRequest.requiredMagicForVersion(versions.latestUsableVersion(ApiKeys.PRODUCE));
maxUsableMagic = (byte) Math.min(nodeMaxUsableMagic, maxUsableMagic);
}
return maxUsableMagic;
}
public synchronized byte maxUsableProduceMagic() {
return maxUsableProduceMagic;
}
}

View File

@@ -0,0 +1,41 @@
/*
* 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;
import java.util.Locale;
public enum ClientDnsLookup {
DEFAULT("default"),
USE_ALL_DNS_IPS("use_all_dns_ips"),
RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY("resolve_canonical_bootstrap_servers_only");
private String clientDnsLookup;
ClientDnsLookup(String clientDnsLookup) {
this.clientDnsLookup = clientDnsLookup;
}
@Override
public String toString() {
return clientDnsLookup;
}
public static ClientDnsLookup forConfig(String config) {
return ClientDnsLookup.valueOf(config.toUpperCase(Locale.ROOT));
}
}

View File

@@ -0,0 +1,119 @@
/*
* 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;
import org.apache.kafka.common.message.RequestHeaderData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.RequestHeader;
/**
* A request being sent to the server. This holds both the network send as well as the client-level metadata.
*/
public final class ClientRequest {
private final String destination;
private final AbstractRequest.Builder<?> requestBuilder;
private final int correlationId;
private final String clientId;
private final long createdTimeMs;
private final boolean expectResponse;
private final int requestTimeoutMs;
private final RequestCompletionHandler callback;
/**
* @param destination The brokerId to send the request to
* @param requestBuilder The builder for the request to make
* @param correlationId The correlation id for this client request
* @param clientId The client ID to use for the header
* @param createdTimeMs The unix timestamp in milliseconds for the time at which this request was created.
* @param expectResponse Should we expect a response message or is this request complete once it is sent?
* @param callback A callback to execute when the response has been received (or null if no callback is necessary)
*/
public ClientRequest(String destination,
AbstractRequest.Builder<?> requestBuilder,
int correlationId,
String clientId,
long createdTimeMs,
boolean expectResponse,
int requestTimeoutMs,
RequestCompletionHandler callback) {
this.destination = destination;
this.requestBuilder = requestBuilder;
this.correlationId = correlationId;
this.clientId = clientId;
this.createdTimeMs = createdTimeMs;
this.expectResponse = expectResponse;
this.requestTimeoutMs = requestTimeoutMs;
this.callback = callback;
}
@Override
public String toString() {
return "ClientRequest(expectResponse=" + expectResponse +
", callback=" + callback +
", destination=" + destination +
", correlationId=" + correlationId +
", clientId=" + clientId +
", createdTimeMs=" + createdTimeMs +
", requestBuilder=" + requestBuilder +
")";
}
public boolean expectResponse() {
return expectResponse;
}
public ApiKeys apiKey() {
return requestBuilder.apiKey();
}
public RequestHeader makeHeader(short version) {
short requestApiKey = requestBuilder.apiKey().id;
return new RequestHeader(
new RequestHeaderData().
setRequestApiKey(requestApiKey).
setRequestApiVersion(version).
setClientId(clientId).
setCorrelationId(correlationId),
ApiKeys.forId(requestApiKey).requestHeaderVersion(version));
}
public AbstractRequest.Builder<?> requestBuilder() {
return requestBuilder;
}
public String destination() {
return destination;
}
public RequestCompletionHandler callback() {
return callback;
}
public long createdTimeMs() {
return createdTimeMs;
}
public int correlationId() {
return correlationId;
}
public int requestTimeoutMs() {
return requestTimeoutMs;
}
}

View File

@@ -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;
import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.requests.AbstractResponse;
import org.apache.kafka.common.requests.RequestHeader;
/**
* A response from the server. Contains both the body of the response as well as the correlated request
* metadata that was originally sent.
*/
public class ClientResponse {
private final RequestHeader requestHeader;
private final RequestCompletionHandler callback;
private final String destination;
private final long receivedTimeMs;
private final long latencyMs;
private final boolean disconnected;
private final UnsupportedVersionException versionMismatch;
private final AuthenticationException authenticationException;
private final AbstractResponse responseBody;
/**
* @param requestHeader The header of the corresponding request
* @param callback The callback to be invoked
* @param createdTimeMs The unix timestamp when the corresponding request was created
* @param destination The node the corresponding request was sent to
* @param receivedTimeMs The unix timestamp when this response was received
* @param disconnected Whether the client disconnected before fully reading a response
* @param versionMismatch Whether there was a version mismatch that prevented sending the request.
* @param responseBody The response contents (or null) if we disconnected, no response was expected,
* or if there was a version mismatch.
*/
public ClientResponse(RequestHeader requestHeader,
RequestCompletionHandler callback,
String destination,
long createdTimeMs,
long receivedTimeMs,
boolean disconnected,
UnsupportedVersionException versionMismatch,
AuthenticationException authenticationException,
AbstractResponse responseBody) {
this.requestHeader = requestHeader;
this.callback = callback;
this.destination = destination;
this.receivedTimeMs = receivedTimeMs;
this.latencyMs = receivedTimeMs - createdTimeMs;
this.disconnected = disconnected;
this.versionMismatch = versionMismatch;
this.authenticationException = authenticationException;
this.responseBody = responseBody;
}
public long receivedTimeMs() {
return receivedTimeMs;
}
public boolean wasDisconnected() {
return disconnected;
}
public UnsupportedVersionException versionMismatch() {
return versionMismatch;
}
public AuthenticationException authenticationException() {
return authenticationException;
}
public RequestHeader requestHeader() {
return requestHeader;
}
public String destination() {
return destination;
}
public AbstractResponse responseBody() {
return responseBody;
}
public boolean hasResponse() {
return responseBody != null;
}
public long requestLatencyMs() {
return latencyMs;
}
public void onComplete() {
if (callback != null)
callback.onComplete(this);
}
@Override
public String toString() {
return "ClientResponse(receivedTimeMs=" + receivedTimeMs +
", latencyMs=" +
latencyMs +
", disconnected=" +
disconnected +
", requestHeader=" +
requestHeader +
", responseBody=" +
responseBody +
")";
}
}

View File

@@ -0,0 +1,131 @@
/*
* 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;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.network.ChannelBuilder;
import org.apache.kafka.common.network.ChannelBuilders;
import org.apache.kafka.common.security.JaasContext;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import static org.apache.kafka.common.utils.Utils.getHost;
import static org.apache.kafka.common.utils.Utils.getPort;
public final class ClientUtils {
private static final Logger log = LoggerFactory.getLogger(ClientUtils.class);
private ClientUtils() {
}
public static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls, String clientDnsLookupConfig) {
return parseAndValidateAddresses(urls, ClientDnsLookup.forConfig(clientDnsLookupConfig));
}
public static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls, ClientDnsLookup clientDnsLookup) {
List<InetSocketAddress> addresses = new ArrayList<>();
for (String url : urls) {
if (url != null && !url.isEmpty()) {
try {
String host = getHost(url);
Integer port = getPort(url);
if (host == null || port == null)
throw new ConfigException("Invalid url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
if (clientDnsLookup == ClientDnsLookup.RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY) {
InetAddress[] inetAddresses = InetAddress.getAllByName(host);
for (InetAddress inetAddress : inetAddresses) {
String resolvedCanonicalName = inetAddress.getCanonicalHostName();
InetSocketAddress address = new InetSocketAddress(resolvedCanonicalName, port);
if (address.isUnresolved()) {
log.warn("Couldn't resolve server {} from {} as DNS resolution of the canonical hostname {} failed for {}", url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, resolvedCanonicalName, host);
} else {
addresses.add(address);
}
}
} else {
InetSocketAddress address = new InetSocketAddress(host, port);
if (address.isUnresolved()) {
log.warn("Couldn't resolve server {} from {} as DNS resolution failed for {}", url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, host);
} else {
addresses.add(address);
}
}
} catch (IllegalArgumentException e) {
throw new ConfigException("Invalid port in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
} catch (UnknownHostException e) {
throw new ConfigException("Unknown host in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
}
}
}
if (addresses.isEmpty())
throw new ConfigException("No resolvable bootstrap urls given in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
return addresses;
}
/**
* Create a new channel builder from the provided configuration.
*
* @param config client configs
* @param time the time implementation
* @param logContext the logging context
*
* @return configured ChannelBuilder based on the configs.
*/
public static ChannelBuilder createChannelBuilder(AbstractConfig config, Time time, LogContext logContext) {
SecurityProtocol securityProtocol = SecurityProtocol.forName(config.getString(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG));
String clientSaslMechanism = config.getString(SaslConfigs.SASL_MECHANISM);
return ChannelBuilders.clientChannelBuilder(securityProtocol, JaasContext.Type.CLIENT, config, null,
clientSaslMechanism, time, true, logContext);
}
static List<InetAddress> resolve(String host, ClientDnsLookup clientDnsLookup) throws UnknownHostException {
InetAddress[] addresses = InetAddress.getAllByName(host);
if (ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup) {
return filterPreferredAddresses(addresses);
} else {
return Collections.singletonList(addresses[0]);
}
}
static List<InetAddress> filterPreferredAddresses(InetAddress[] allAddresses) {
List<InetAddress> preferredAddresses = new ArrayList<>();
Class<? extends InetAddress> clazz = null;
for (InetAddress address : allAddresses) {
if (clazz == null) {
clazz = address.getClass();
}
if (clazz.isInstance(address)) {
preferredAddresses.add(address);
}
}
return preferredAddresses;
}
}

View File

@@ -0,0 +1,427 @@
/*
* 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;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.utils.LogContext;
import org.slf4j.Logger;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* The state of our connection to each node in the cluster.
*
*/
final class ClusterConnectionStates {
private final long reconnectBackoffInitMs;
private final long reconnectBackoffMaxMs;
private final static int RECONNECT_BACKOFF_EXP_BASE = 2;
private final double reconnectBackoffMaxExp;
private final Map<String, NodeConnectionState> nodeState;
private final Logger log;
public ClusterConnectionStates(long reconnectBackoffMs, long reconnectBackoffMaxMs, LogContext logContext) {
this.log = logContext.logger(ClusterConnectionStates.class);
this.reconnectBackoffInitMs = reconnectBackoffMs;
this.reconnectBackoffMaxMs = reconnectBackoffMaxMs;
this.reconnectBackoffMaxExp = Math.log(this.reconnectBackoffMaxMs / (double) Math.max(reconnectBackoffMs, 1)) / Math.log(RECONNECT_BACKOFF_EXP_BASE);
this.nodeState = new HashMap<>();
}
/**
* Return true iff we can currently initiate a new connection. This will be the case if we are not
* connected and haven't been connected for at least the minimum reconnection backoff period.
* @param id the connection id to check
* @param now the current time in ms
* @return true if we can initiate a new connection
*/
public boolean canConnect(String id, long now) {
NodeConnectionState state = nodeState.get(id);
if (state == null)
return true;
else
return state.state.isDisconnected() &&
now - state.lastConnectAttemptMs >= state.reconnectBackoffMs;
}
/**
* Return true if we are disconnected from the given node and can't re-establish a connection yet.
* @param id the connection to check
* @param now the current time in ms
*/
public boolean isBlackedOut(String id, long now) {
NodeConnectionState state = nodeState.get(id);
return state != null
&& state.state.isDisconnected()
&& now - state.lastConnectAttemptMs < state.reconnectBackoffMs;
}
/**
* Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
* disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
* connections.
* @param id the connection to check
* @param now the current time in ms
*/
public long connectionDelay(String id, long now) {
NodeConnectionState state = nodeState.get(id);
if (state == null) return 0;
if (state.state.isDisconnected()) {
long timeWaited = now - state.lastConnectAttemptMs;
return Math.max(state.reconnectBackoffMs - timeWaited, 0);
} else {
// When connecting or connected, we should be able to delay indefinitely since other events (connection or
// data acked) will cause a wakeup once data can be sent.
return Long.MAX_VALUE;
}
}
/**
* Return true if a specific connection establishment is currently underway
* @param id The id of the node to check
*/
public boolean isConnecting(String id) {
NodeConnectionState state = nodeState.get(id);
return state != null && state.state == ConnectionState.CONNECTING;
}
/**
* Check whether a connection is either being established or awaiting API version information.
* @param id The id of the node to check
* @return true if the node is either connecting or has connected and is awaiting API versions, false otherwise
*/
public boolean isPreparingConnection(String id) {
NodeConnectionState state = nodeState.get(id);
return state != null &&
(state.state == ConnectionState.CONNECTING || state.state == ConnectionState.CHECKING_API_VERSIONS);
}
/**
* Enter the connecting state for the given connection, moving to a new resolved address if necessary.
* @param id the id of the connection
* @param now the current time in ms
* @param host the host of the connection, to be resolved internally if needed
* @param clientDnsLookup the mode of DNS lookup to use when resolving the {@code host}
*/
public void connecting(String id, long now, String host, ClientDnsLookup clientDnsLookup) {
NodeConnectionState connectionState = nodeState.get(id);
if (connectionState != null && connectionState.host().equals(host)) {
connectionState.lastConnectAttemptMs = now;
connectionState.state = ConnectionState.CONNECTING;
// Move to next resolved address, or if addresses are exhausted, mark node to be re-resolved
connectionState.moveToNextAddress();
return;
} else if (connectionState != null) {
log.info("Hostname for node {} changed from {} to {}.", id, connectionState.host(), host);
}
// Create a new NodeConnectionState if nodeState does not already contain one
// for the specified id or if the hostname associated with the node id changed.
nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, now,
this.reconnectBackoffInitMs, host, clientDnsLookup));
}
/**
* Returns a resolved address for the given connection, resolving it if necessary.
* @param id the id of the connection
* @throws UnknownHostException if the address was not resolvable
*/
public InetAddress currentAddress(String id) throws UnknownHostException {
return nodeState(id).currentAddress();
}
/**
* Enter the disconnected state for the given node.
* @param id the connection we have disconnected
* @param now the current time in ms
*/
public void disconnected(String id, long now) {
NodeConnectionState nodeState = nodeState(id);
nodeState.state = ConnectionState.DISCONNECTED;
nodeState.lastConnectAttemptMs = now;
updateReconnectBackoff(nodeState);
}
/**
* Indicate that the connection is throttled until the specified deadline.
* @param id the connection to be throttled
* @param throttleUntilTimeMs the throttle deadline in milliseconds
*/
public void throttle(String id, long throttleUntilTimeMs) {
NodeConnectionState state = nodeState.get(id);
// The throttle deadline should never regress.
if (state != null && state.throttleUntilTimeMs < throttleUntilTimeMs) {
state.throttleUntilTimeMs = throttleUntilTimeMs;
}
}
/**
* Return the remaining throttling delay in milliseconds if throttling is in progress. Return 0, otherwise.
* @param id the connection to check
* @param now the current time in ms
*/
public long throttleDelayMs(String id, long now) {
NodeConnectionState state = nodeState.get(id);
if (state != null && state.throttleUntilTimeMs > now) {
return state.throttleUntilTimeMs - now;
} else {
return 0;
}
}
/**
* Return the number of milliseconds to wait, based on the connection state and the throttle time, before
* attempting to send data. If the connection has been established but being throttled, return throttle delay.
* Otherwise, return connection delay.
* @param id the connection to check
* @param now the current time in ms
*/
public long pollDelayMs(String id, long now) {
long throttleDelayMs = throttleDelayMs(id, now);
if (isConnected(id) && throttleDelayMs > 0) {
return throttleDelayMs;
} else {
return connectionDelay(id, now);
}
}
/**
* Enter the checking_api_versions state for the given node.
* @param id the connection identifier
*/
public void checkingApiVersions(String id) {
NodeConnectionState nodeState = nodeState(id);
nodeState.state = ConnectionState.CHECKING_API_VERSIONS;
}
/**
* Enter the ready state for the given node.
* @param id the connection identifier
*/
public void ready(String id) {
NodeConnectionState nodeState = nodeState(id);
nodeState.state = ConnectionState.READY;
nodeState.authenticationException = null;
resetReconnectBackoff(nodeState);
}
/**
* Enter the authentication failed state for the given node.
* @param id the connection identifier
* @param now the current time in ms
* @param exception the authentication exception
*/
public void authenticationFailed(String id, long now, AuthenticationException exception) {
NodeConnectionState nodeState = nodeState(id);
nodeState.authenticationException = exception;
nodeState.state = ConnectionState.AUTHENTICATION_FAILED;
nodeState.lastConnectAttemptMs = now;
updateReconnectBackoff(nodeState);
}
/**
* Return true if the connection is in the READY state and currently not throttled.
*
* @param id the connection identifier
* @param now the current time in ms
*/
public boolean isReady(String id, long now) {
return isReady(nodeState.get(id), now);
}
private boolean isReady(NodeConnectionState state, long now) {
return state != null && state.state == ConnectionState.READY && state.throttleUntilTimeMs <= now;
}
/**
* Return true if there is at least one node with connection in the READY state and not throttled. Returns false
* otherwise.
*
* @param now the current time in ms
*/
public boolean hasReadyNodes(long now) {
for (Map.Entry<String, NodeConnectionState> entry : nodeState.entrySet()) {
if (isReady(entry.getValue(), now)) {
return true;
}
}
return false;
}
/**
* Return true if the connection has been established
* @param id The id of the node to check
*/
public boolean isConnected(String id) {
NodeConnectionState state = nodeState.get(id);
return state != null && state.state.isConnected();
}
/**
* Return true if the connection has been disconnected
* @param id The id of the node to check
*/
public boolean isDisconnected(String id) {
NodeConnectionState state = nodeState.get(id);
return state != null && state.state.isDisconnected();
}
/**
* Return authentication exception if an authentication error occurred
* @param id The id of the node to check
*/
public AuthenticationException authenticationException(String id) {
NodeConnectionState state = nodeState.get(id);
return state != null ? state.authenticationException : null;
}
/**
* Resets the failure count for a node and sets the reconnect backoff to the base
* value configured via reconnect.backoff.ms
*
* @param nodeState The node state object to update
*/
private void resetReconnectBackoff(NodeConnectionState nodeState) {
nodeState.failedAttempts = 0;
nodeState.reconnectBackoffMs = this.reconnectBackoffInitMs;
}
/**
* Update the node reconnect backoff exponentially.
* The delay is reconnect.backoff.ms * 2**(failures - 1) * (+/- 20% random jitter)
* Up to a (pre-jitter) maximum of reconnect.backoff.max.ms
*
* @param nodeState The node state object to update
*/
private void updateReconnectBackoff(NodeConnectionState nodeState) {
if (this.reconnectBackoffMaxMs > this.reconnectBackoffInitMs) {
nodeState.failedAttempts += 1;
double backoffExp = Math.min(nodeState.failedAttempts - 1, this.reconnectBackoffMaxExp);
double backoffFactor = Math.pow(RECONNECT_BACKOFF_EXP_BASE, backoffExp);
long reconnectBackoffMs = (long) (this.reconnectBackoffInitMs * backoffFactor);
// Actual backoff is randomized to avoid connection storms.
double randomFactor = ThreadLocalRandom.current().nextDouble(0.8, 1.2);
nodeState.reconnectBackoffMs = (long) (randomFactor * reconnectBackoffMs);
}
}
/**
* Remove the given node from the tracked connection states. The main difference between this and `disconnected`
* is the impact on `connectionDelay`: it will be 0 after this call whereas `reconnectBackoffMs` will be taken
* into account after `disconnected` is called.
*
* @param id the connection to remove
*/
public void remove(String id) {
nodeState.remove(id);
}
/**
* Get the state of a given connection.
* @param id the id of the connection
* @return the state of our connection
*/
public ConnectionState connectionState(String id) {
return nodeState(id).state;
}
/**
* Get the state of a given node.
* @param id the connection to fetch the state for
*/
private NodeConnectionState nodeState(String id) {
NodeConnectionState state = this.nodeState.get(id);
if (state == null)
throw new IllegalStateException("No entry found for connection " + id);
return state;
}
/**
* The state of our connection to a node.
*/
private static class NodeConnectionState {
ConnectionState state;
AuthenticationException authenticationException;
long lastConnectAttemptMs;
long failedAttempts;
long reconnectBackoffMs;
// Connection is being throttled if current time < throttleUntilTimeMs.
long throttleUntilTimeMs;
private List<InetAddress> addresses;
private int addressIndex;
private final String host;
private final ClientDnsLookup clientDnsLookup;
private NodeConnectionState(ConnectionState state, long lastConnectAttempt, long reconnectBackoffMs,
String host, ClientDnsLookup clientDnsLookup) {
this.state = state;
this.addresses = Collections.emptyList();
this.addressIndex = -1;
this.authenticationException = null;
this.lastConnectAttemptMs = lastConnectAttempt;
this.failedAttempts = 0;
this.reconnectBackoffMs = reconnectBackoffMs;
this.throttleUntilTimeMs = 0;
this.host = host;
this.clientDnsLookup = clientDnsLookup;
}
public String host() {
return host;
}
/**
* Fetches the current selected IP address for this node, resolving {@link #host()} if necessary.
* @return the selected address
* @throws UnknownHostException if resolving {@link #host()} fails
*/
private InetAddress currentAddress() throws UnknownHostException {
if (addresses.isEmpty()) {
// (Re-)initialize list
addresses = ClientUtils.resolve(host, clientDnsLookup);
addressIndex = 0;
}
return addresses.get(addressIndex);
}
/**
* Jumps to the next available resolved address for this node. If no other addresses are available, marks the
* list to be refreshed on the next {@link #currentAddress()} call.
*/
private void moveToNextAddress() {
if (addresses.isEmpty())
return; // Avoid div0. List will initialize on next currentAddress() call
addressIndex = (addressIndex + 1) % addresses.size();
if (addressIndex == 0)
addresses = Collections.emptyList(); // Exhausted list. Re-resolve on next currentAddress() call
}
public String toString() {
return "NodeState(" + state + ", " + lastConnectAttemptMs + ", " + failedAttempts + ", " + throttleUntilTimeMs + ")";
}
}
}

View File

@@ -0,0 +1,168 @@
/*
* 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;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.utils.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashMap;
import java.util.Map;
/**
* Configurations shared by Kafka client applications: producer, consumer, connect, etc.
*/
public class CommonClientConfigs {
private static final Logger log = LoggerFactory.getLogger(CommonClientConfigs.class);
/*
* NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
*/
public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
public static final String BOOTSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping&mdash;this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form "
+ "<code>host1:port1,host2:port2,...</code>. Since these servers are just used for the initial connection to "
+ "discover the full cluster membership (which may change dynamically), this list need not contain the full set of "
+ "servers (you may want more than one, though, in case a server is down).";
public static final String CLIENT_DNS_LOOKUP_CONFIG = "client.dns.lookup";
public static final String CLIENT_DNS_LOOKUP_DOC = "Controls how the client uses DNS lookups. If set to <code>use_all_dns_ips</code> then, when the lookup returns multiple IP addresses for a hostname,"
+ " they will all be attempted to connect to before failing the connection. Applies to both bootstrap and advertised servers."
+ " If the value is <code>resolve_canonical_bootstrap_servers_only</code> each entry will be resolved and expanded into a list of canonical names.";
public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms";
public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions.";
public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes";
public static final String SEND_BUFFER_DOC = "The size of the TCP send buffer (SO_SNDBUF) to use when sending data. If the value is -1, the OS default will be used.";
public static final int SEND_BUFFER_LOWER_BOUND = -1;
public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes";
public static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer (SO_RCVBUF) to use when reading data. If the value is -1, the OS default will be used.";
public static final int RECEIVE_BUFFER_LOWER_BOUND = -1;
public static final String CLIENT_ID_CONFIG = "client.id";
public static final String CLIENT_ID_DOC = "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging.";
public static final String CLIENT_RACK_CONFIG = "client.rack";
public static final String CLIENT_RACK_DOC = "A rack identifier for this client. This can be any string value which indicates where this client is physically located. It corresponds with the broker config 'broker.rack'";
public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms";
public static final String RECONNECT_BACKOFF_MS_DOC = "The base amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all connection attempts by the client to a broker.";
public static final String RECONNECT_BACKOFF_MAX_MS_CONFIG = "reconnect.backoff.max.ms";
public static final String RECONNECT_BACKOFF_MAX_MS_DOC = "The maximum amount of time in milliseconds to wait when reconnecting to a broker that has repeatedly failed to connect. If provided, the backoff per host will increase exponentially for each consecutive connection failure, up to this maximum. After calculating the backoff increase, 20% random jitter is added to avoid connection storms.";
public static final String RETRIES_CONFIG = "retries";
public static final String RETRIES_DOC = "Setting a value greater than zero will cause the client to resend any request that fails with a potentially transient error.";
public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms";
public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed request to a given topic partition. This avoids repeatedly sending requests in a tight loop under some failure scenarios.";
public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms";
public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The window of time a metrics sample is computed over.";
public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples";
public static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics.";
public static final String METRICS_RECORDING_LEVEL_CONFIG = "metrics.recording.level";
public static final String METRICS_RECORDING_LEVEL_DOC = "The highest recording level for metrics.";
public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters";
public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>org.apache.kafka.common.metrics.MetricsReporter</code> interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics.";
public static final String SECURITY_PROTOCOL_CONFIG = "security.protocol";
public static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Valid values are: " +
Utils.join(SecurityProtocol.names(), ", ") + ".";
public static final String DEFAULT_SECURITY_PROTOCOL = "PLAINTEXT";
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = "connections.max.idle.ms";
public static final String CONNECTIONS_MAX_IDLE_MS_DOC = "Close idle connections after the number of milliseconds specified by this config.";
public static final String REQUEST_TIMEOUT_MS_CONFIG = "request.timeout.ms";
public static final String REQUEST_TIMEOUT_MS_DOC = "The configuration controls the maximum amount of time the client will wait "
+ "for the response of a request. If the response is not received before the timeout "
+ "elapses the client will resend the request if necessary or fail the request if "
+ "retries are exhausted.";
public static final String GROUP_ID_CONFIG = "group.id";
public static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using <code>subscribe(topic)</code> or the Kafka-based offset management strategy.";
public static final String GROUP_INSTANCE_ID_CONFIG = "group.instance.id";
public static final String GROUP_INSTANCE_ID_DOC = "A unique identifier of the consumer instance provided by the end user. "
+ "Only non-empty strings are permitted. If set, the consumer is treated as a static member, "
+ "which means that only one instance with this ID is allowed in the consumer group at any time. "
+ "This can be used in combination with a larger session timeout to avoid group rebalances caused by transient unavailability "
+ "(e.g. process restarts). If not set, the consumer will join the group as a dynamic member, which is the traditional behavior.";
public static final String MAX_POLL_INTERVAL_MS_CONFIG = "max.poll.interval.ms";
public static final String MAX_POLL_INTERVAL_MS_DOC = "The maximum delay between invocations of poll() when using "
+ "consumer group management. This places an upper bound on the amount of time that the consumer can be idle "
+ "before fetching more records. If poll() is not called before expiration of this timeout, then the consumer "
+ "is considered failed and the group will rebalance in order to reassign the partitions to another member. "
+ "For consumers using a non-null <code>group.instance.id</code> which reach this timeout, partitions will not be immediately reassigned. "
+ "Instead, the consumer will stop sending heartbeats and partitions will be reassigned "
+ "after expiration of <code>session.timeout.ms</code>. This mirrors the behavior of a static consumer which has shutdown.";
public static final String REBALANCE_TIMEOUT_MS_CONFIG = "rebalance.timeout.ms";
public static final String REBALANCE_TIMEOUT_MS_DOC = "The maximum allowed time for each worker to join the group "
+ "once a rebalance has begun. This is basically a limit on the amount of time needed for all tasks to "
+ "flush any pending data and commit offsets. If the timeout is exceeded, then the worker will be removed "
+ "from the group, which will cause offset commit failures.";
public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms";
public static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect client failures when using "
+ "Kafka's group management facility. The client sends periodic heartbeats to indicate its liveness "
+ "to the broker. If no heartbeats are received by the broker before the expiration of this session timeout, "
+ "then the broker will remove this client from the group and initiate a rebalance. Note that the value "
+ "must be in the allowable range as configured in the broker configuration by <code>group.min.session.timeout.ms</code> "
+ "and <code>group.max.session.timeout.ms</code>.";
public static final String HEARTBEAT_INTERVAL_MS_CONFIG = "heartbeat.interval.ms";
public static final String HEARTBEAT_INTERVAL_MS_DOC = "The expected time between heartbeats to the consumer "
+ "coordinator when using Kafka's group management facilities. Heartbeats are used to ensure that the "
+ "consumer's session stays active and to facilitate rebalancing when new consumers join or leave the group. "
+ "The value must be set lower than <code>session.timeout.ms</code>, but typically should be set no higher "
+ "than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances.";
public static final String DEFAULT_API_TIMEOUT_MS_CONFIG = "default.api.timeout.ms";
public static final String DEFAULT_API_TIMEOUT_MS_DOC = "Specifies the timeout (in milliseconds) for client APIs. " +
"This configuration is used as the default timeout for all client operations that do not specify a <code>timeout</code> parameter.";
/**
* Postprocess the configuration so that exponential backoff is disabled when reconnect backoff
* is explicitly configured but the maximum reconnect backoff is not explicitly configured.
*
* @param config The config object.
* @param parsedValues The parsedValues as provided to postProcessParsedConfig.
*
* @return The new values which have been set as described in postProcessParsedConfig.
*/
public static Map<String, Object> postProcessReconnectBackoffConfigs(AbstractConfig config,
Map<String, Object> parsedValues) {
HashMap<String, Object> rval = new HashMap<>();
if ((!config.originals().containsKey(RECONNECT_BACKOFF_MAX_MS_CONFIG)) &&
config.originals().containsKey(RECONNECT_BACKOFF_MS_CONFIG)) {
log.debug("Disabling exponential reconnect backoff because {} is set, but {} is not.",
RECONNECT_BACKOFF_MS_CONFIG, RECONNECT_BACKOFF_MAX_MS_CONFIG);
rval.put(RECONNECT_BACKOFF_MAX_MS_CONFIG, parsedValues.get(RECONNECT_BACKOFF_MS_CONFIG));
}
return rval;
}
}

View File

@@ -0,0 +1,38 @@
/*
* 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;
/**
* The states of a node connection
*
* DISCONNECTED: connection has not been successfully established yet
* CONNECTING: connection is under progress
* CHECKING_API_VERSIONS: connection has been established and api versions check is in progress. Failure of this check will cause connection to close
* READY: connection is ready to send requests
* AUTHENTICATION_FAILED: connection failed due to an authentication error
*/
public enum ConnectionState {
DISCONNECTED, CONNECTING, CHECKING_API_VERSIONS, READY, AUTHENTICATION_FAILED;
public boolean isDisconnected() {
return this == AUTHENTICATION_FAILED || this == DISCONNECTED;
}
public boolean isConnected() {
return this == CHECKING_API_VERSIONS || this == READY;
}
}

View File

@@ -0,0 +1,484 @@
/*
* 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;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.FetchMetadata;
import org.apache.kafka.common.requests.FetchRequest.PartitionData;
import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Utils;
import org.slf4j.Logger;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
/**
* FetchSessionHandler maintains the fetch session state for connecting to a broker.
*
* Using the protocol outlined by KIP-227, clients can create incremental fetch sessions.
* These sessions allow the client to fetch information about a set of partition over
* and over, without explicitly enumerating all the partitions in the request and the
* response.
*
* FetchSessionHandler tracks the partitions which are in the session. It also
* determines which partitions need to be included in each fetch request, and what
* the attached fetch session metadata should be for each request. The corresponding
* class on the receiving broker side is FetchManager.
*/
public class FetchSessionHandler {
private final Logger log;
private final int node;
/**
* The metadata for the next fetch request.
*/
private FetchMetadata nextMetadata = FetchMetadata.INITIAL;
public FetchSessionHandler(LogContext logContext, int node) {
this.log = logContext.logger(FetchSessionHandler.class);
this.node = node;
}
/**
* All of the partitions which exist in the fetch request session.
*/
private LinkedHashMap<TopicPartition, PartitionData> sessionPartitions =
new LinkedHashMap<>(0);
public static class FetchRequestData {
/**
* The partitions to send in the fetch request.
*/
private final Map<TopicPartition, PartitionData> toSend;
/**
* The partitions to send in the request's "forget" list.
*/
private final List<TopicPartition> toForget;
/**
* All of the partitions which exist in the fetch request session.
*/
private final Map<TopicPartition, PartitionData> sessionPartitions;
/**
* The metadata to use in this fetch request.
*/
private final FetchMetadata metadata;
FetchRequestData(Map<TopicPartition, PartitionData> toSend,
List<TopicPartition> toForget,
Map<TopicPartition, PartitionData> sessionPartitions,
FetchMetadata metadata) {
this.toSend = toSend;
this.toForget = toForget;
this.sessionPartitions = sessionPartitions;
this.metadata = metadata;
}
/**
* Get the set of partitions to send in this fetch request.
*/
public Map<TopicPartition, PartitionData> toSend() {
return toSend;
}
/**
* Get a list of partitions to forget in this fetch request.
*/
public List<TopicPartition> toForget() {
return toForget;
}
/**
* Get the full set of partitions involved in this fetch request.
*/
public Map<TopicPartition, PartitionData> sessionPartitions() {
return sessionPartitions;
}
public FetchMetadata metadata() {
return metadata;
}
@Override
public String toString() {
if (metadata.isFull()) {
StringBuilder bld = new StringBuilder("FullFetchRequest(");
String prefix = "";
for (TopicPartition partition : toSend.keySet()) {
bld.append(prefix);
bld.append(partition);
prefix = ", ";
}
bld.append(")");
return bld.toString();
} else {
StringBuilder bld = new StringBuilder("IncrementalFetchRequest(toSend=(");
String prefix = "";
for (TopicPartition partition : toSend.keySet()) {
bld.append(prefix);
bld.append(partition);
prefix = ", ";
}
bld.append("), toForget=(");
prefix = "";
for (TopicPartition partition : toForget) {
bld.append(prefix);
bld.append(partition);
prefix = ", ";
}
bld.append("), implied=(");
prefix = "";
for (TopicPartition partition : sessionPartitions.keySet()) {
if (!toSend.containsKey(partition)) {
bld.append(prefix);
bld.append(partition);
prefix = ", ";
}
}
bld.append("))");
return bld.toString();
}
}
}
public class Builder {
/**
* The next partitions which we want to fetch.
*
* It is important to maintain the insertion order of this list by using a LinkedHashMap rather
* than a regular Map.
*
* One reason is that when dealing with FULL fetch requests, if there is not enough response
* space to return data from all partitions, the server will only return data from partitions
* early in this list.
*
* Another reason is because we make use of the list ordering to optimize the preparation of
* incremental fetch requests (see below).
*/
private LinkedHashMap<TopicPartition, PartitionData> next;
private final boolean copySessionPartitions;
Builder() {
this.next = new LinkedHashMap<>();
this.copySessionPartitions = true;
}
Builder(int initialSize, boolean copySessionPartitions) {
this.next = new LinkedHashMap<>(initialSize);
this.copySessionPartitions = copySessionPartitions;
}
/**
* Mark that we want data from this partition in the upcoming fetch.
*/
public void add(TopicPartition topicPartition, PartitionData data) {
next.put(topicPartition, data);
}
public FetchRequestData build() {
if (nextMetadata.isFull()) {
if (log.isDebugEnabled()) {
log.debug("Built full fetch {} for node {} with {}.",
nextMetadata, node, partitionsToLogString(next.keySet()));
}
sessionPartitions = next;
next = null;
Map<TopicPartition, PartitionData> toSend =
Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions));
return new FetchRequestData(toSend, Collections.emptyList(), toSend, nextMetadata);
}
List<TopicPartition> added = new ArrayList<>();
List<TopicPartition> removed = new ArrayList<>();
List<TopicPartition> altered = new ArrayList<>();
for (Iterator<Entry<TopicPartition, PartitionData>> iter =
sessionPartitions.entrySet().iterator(); iter.hasNext(); ) {
Entry<TopicPartition, PartitionData> entry = iter.next();
TopicPartition topicPartition = entry.getKey();
PartitionData prevData = entry.getValue();
PartitionData nextData = next.remove(topicPartition);
if (nextData != null) {
if (!prevData.equals(nextData)) {
// Re-add the altered partition to the end of 'next'
next.put(topicPartition, nextData);
entry.setValue(nextData);
altered.add(topicPartition);
}
} else {
// Remove this partition from the session.
iter.remove();
// Indicate that we no longer want to listen to this partition.
removed.add(topicPartition);
}
}
// Add any new partitions to the session.
for (Entry<TopicPartition, PartitionData> entry : next.entrySet()) {
TopicPartition topicPartition = entry.getKey();
PartitionData nextData = entry.getValue();
if (sessionPartitions.containsKey(topicPartition)) {
// In the previous loop, all the partitions which existed in both sessionPartitions
// and next were moved to the end of next, or removed from next. Therefore,
// once we hit one of them, we know there are no more unseen entries to look
// at in next.
break;
}
sessionPartitions.put(topicPartition, nextData);
added.add(topicPartition);
}
if (log.isDebugEnabled()) {
log.debug("Built incremental fetch {} for node {}. Added {}, altered {}, removed {} " +
"out of {}", nextMetadata, node, partitionsToLogString(added),
partitionsToLogString(altered), partitionsToLogString(removed),
partitionsToLogString(sessionPartitions.keySet()));
}
Map<TopicPartition, PartitionData> toSend = Collections.unmodifiableMap(next);
Map<TopicPartition, PartitionData> curSessionPartitions = copySessionPartitions
? Collections.unmodifiableMap(new LinkedHashMap<>(sessionPartitions))
: Collections.unmodifiableMap(sessionPartitions);
next = null;
return new FetchRequestData(toSend, Collections.unmodifiableList(removed),
curSessionPartitions, nextMetadata);
}
}
public Builder newBuilder() {
return new Builder();
}
/** A builder that allows for presizing the PartitionData hashmap, and avoiding making a
* secondary copy of the sessionPartitions, in cases where this is not necessarily.
* This builder is primarily for use by the Replica Fetcher
* @param size the initial size of the PartitionData hashmap
* @param copySessionPartitions boolean denoting whether the builder should make a deep copy of
* session partitions
*/
public Builder newBuilder(int size, boolean copySessionPartitions) {
return new Builder(size, copySessionPartitions);
}
private String partitionsToLogString(Collection<TopicPartition> partitions) {
if (!log.isTraceEnabled()) {
return String.format("%d partition(s)", partitions.size());
}
return "(" + Utils.join(partitions, ", ") + ")";
}
/**
* Return some partitions which are expected to be in a particular set, but which are not.
*
* @param toFind The partitions to look for.
* @param toSearch The set of partitions to search.
* @return null if all partitions were found; some of the missing ones
* in string form, if not.
*/
static Set<TopicPartition> findMissing(Set<TopicPartition> toFind, Set<TopicPartition> toSearch) {
Set<TopicPartition> ret = new LinkedHashSet<>();
for (TopicPartition partition : toFind) {
if (!toSearch.contains(partition)) {
ret.add(partition);
}
}
return ret;
}
/**
* Verify that a full fetch response contains all the partitions in the fetch session.
*
* @param response The response.
* @return True if the full fetch response partitions are valid.
*/
String verifyFullFetchResponsePartitions(FetchResponse<?> response) {
StringBuilder bld = new StringBuilder();
Set<TopicPartition> extra =
findMissing(response.responseData().keySet(), sessionPartitions.keySet());
Set<TopicPartition> omitted =
findMissing(sessionPartitions.keySet(), response.responseData().keySet());
if (!omitted.isEmpty()) {
bld.append("omitted=(").append(Utils.join(omitted, ", ")).append(", ");
}
if (!extra.isEmpty()) {
bld.append("extra=(").append(Utils.join(extra, ", ")).append(", ");
}
if ((!omitted.isEmpty()) || (!extra.isEmpty())) {
bld.append("response=(").append(Utils.join(response.responseData().keySet(), ", ")).append(")");
return bld.toString();
}
return null;
}
/**
* Verify that the partitions in an incremental fetch response are contained in the session.
*
* @param response The response.
* @return True if the incremental fetch response partitions are valid.
*/
String verifyIncrementalFetchResponsePartitions(FetchResponse<?> response) {
Set<TopicPartition> extra =
findMissing(response.responseData().keySet(), sessionPartitions.keySet());
if (!extra.isEmpty()) {
StringBuilder bld = new StringBuilder();
bld.append("extra=(").append(Utils.join(extra, ", ")).append("), ");
bld.append("response=(").append(
Utils.join(response.responseData().keySet(), ", ")).append("), ");
return bld.toString();
}
return null;
}
/**
* Create a string describing the partitions in a FetchResponse.
*
* @param response The FetchResponse.
* @return The string to log.
*/
private String responseDataToLogString(FetchResponse<?> response) {
if (!log.isTraceEnabled()) {
int implied = sessionPartitions.size() - response.responseData().size();
if (implied > 0) {
return String.format(" with %d response partition(s), %d implied partition(s)",
response.responseData().size(), implied);
} else {
return String.format(" with %d response partition(s)",
response.responseData().size());
}
}
StringBuilder bld = new StringBuilder();
bld.append(" with response=(").
append(Utils.join(response.responseData().keySet(), ", ")).
append(")");
String prefix = ", implied=(";
String suffix = "";
for (TopicPartition partition : sessionPartitions.keySet()) {
if (!response.responseData().containsKey(partition)) {
bld.append(prefix);
bld.append(partition);
prefix = ", ";
suffix = ")";
}
}
bld.append(suffix);
return bld.toString();
}
/**
* Handle the fetch response.
*
* @param response The response.
* @return True if the response is well-formed; false if it can't be processed
* because of missing or unexpected partitions.
*/
public boolean handleResponse(FetchResponse<?> response) {
if (response.error() != Errors.NONE) {
log.info("Node {} was unable to process the fetch request with {}: {}.",
node, nextMetadata, response.error());
if (response.error() == Errors.FETCH_SESSION_ID_NOT_FOUND) {
nextMetadata = FetchMetadata.INITIAL;
} else {
nextMetadata = nextMetadata.nextCloseExisting();
}
return false;
}
if (nextMetadata.isFull()) {
if (response.responseData().isEmpty() && response.throttleTimeMs() > 0) {
// Normally, an empty full fetch response would be invalid. However, KIP-219
// specifies that if the broker wants to throttle the client, it will respond
// to a full fetch request with an empty response and a throttleTimeMs
// value set. We don't want to log this with a warning, since it's not an error.
// However, the empty full fetch response can't be processed, so it's still appropriate
// to return false here.
if (log.isDebugEnabled()) {
log.debug("Node {} sent a empty full fetch response to indicate that this " +
"client should be throttled for {} ms.", node, response.throttleTimeMs());
}
nextMetadata = FetchMetadata.INITIAL;
return false;
}
String problem = verifyFullFetchResponsePartitions(response);
if (problem != null) {
log.info("Node {} sent an invalid full fetch response with {}", node, problem);
nextMetadata = FetchMetadata.INITIAL;
return false;
} else if (response.sessionId() == INVALID_SESSION_ID) {
if (log.isDebugEnabled())
log.debug("Node {} sent a full fetch response{}", node, responseDataToLogString(response));
nextMetadata = FetchMetadata.INITIAL;
return true;
} else {
// The server created a new incremental fetch session.
if (log.isDebugEnabled())
log.debug("Node {} sent a full fetch response that created a new incremental " +
"fetch session {}{}", node, response.sessionId(), responseDataToLogString(response));
nextMetadata = FetchMetadata.newIncremental(response.sessionId());
return true;
}
} else {
String problem = verifyIncrementalFetchResponsePartitions(response);
if (problem != null) {
log.info("Node {} sent an invalid incremental fetch response with {}", node, problem);
nextMetadata = nextMetadata.nextCloseExisting();
return false;
} else if (response.sessionId() == INVALID_SESSION_ID) {
// The incremental fetch session was closed by the server.
if (log.isDebugEnabled())
log.debug("Node {} sent an incremental fetch response closing session {}{}",
node, nextMetadata.sessionId(), responseDataToLogString(response));
nextMetadata = FetchMetadata.INITIAL;
return true;
} else {
// The incremental fetch session was continued by the server.
// We don't have to do anything special here to support KIP-219, since an empty incremental
// fetch request is perfectly valid.
if (log.isDebugEnabled())
log.debug("Node {} sent an incremental fetch response with throttleTimeMs = {} " +
"for session {}{}", response.throttleTimeMs(), node, response.sessionId(),
responseDataToLogString(response));
nextMetadata = nextMetadata.nextIncremental();
return true;
}
}
}
/**
* Handle an error sending the prepared request.
*
* When a network error occurs, we close any existing fetch session on our next request,
* and try to create a new session.
*
* @param t The exception.
*/
public void handleError(Throwable t) {
log.info("Error sending fetch request {} to node {}: {}.", nextMetadata, node, t);
nextMetadata = nextMetadata.nextCloseExisting();
}
}

View File

@@ -0,0 +1,100 @@
/*
* 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;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.requests.JoinGroupRequest;
import java.util.Locale;
import java.util.Optional;
/**
* Class to extract group rebalance related configs.
*/
public class GroupRebalanceConfig {
public enum ProtocolType {
CONSUMER,
CONNECT;
@Override
public String toString() {
return super.toString().toLowerCase(Locale.ROOT);
}
}
public final int sessionTimeoutMs;
public final int rebalanceTimeoutMs;
public final int heartbeatIntervalMs;
public final String groupId;
public final Optional<String> groupInstanceId;
public final long retryBackoffMs;
public final boolean leaveGroupOnClose;
public GroupRebalanceConfig(AbstractConfig config, ProtocolType protocolType) {
this.sessionTimeoutMs = config.getInt(CommonClientConfigs.SESSION_TIMEOUT_MS_CONFIG);
// Consumer and Connect use different config names for defining rebalance timeout
if (protocolType == ProtocolType.CONSUMER) {
this.rebalanceTimeoutMs = config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
} else {
this.rebalanceTimeoutMs = config.getInt(CommonClientConfigs.REBALANCE_TIMEOUT_MS_CONFIG);
}
this.heartbeatIntervalMs = config.getInt(CommonClientConfigs.HEARTBEAT_INTERVAL_MS_CONFIG);
this.groupId = config.getString(CommonClientConfigs.GROUP_ID_CONFIG);
// Static membership is only introduced in consumer API.
if (protocolType == ProtocolType.CONSUMER) {
String groupInstanceId = config.getString(CommonClientConfigs.GROUP_INSTANCE_ID_CONFIG);
if (groupInstanceId != null) {
JoinGroupRequest.validateGroupInstanceId(groupInstanceId);
this.groupInstanceId = Optional.of(groupInstanceId);
} else {
this.groupInstanceId = Optional.empty();
}
} else {
this.groupInstanceId = Optional.empty();
}
this.retryBackoffMs = config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG);
// Internal leave group config is only defined in Consumer.
if (protocolType == ProtocolType.CONSUMER) {
this.leaveGroupOnClose = config.getBoolean("internal.leave.group.on.close");
} else {
this.leaveGroupOnClose = true;
}
}
// For testing purpose.
public GroupRebalanceConfig(final int sessionTimeoutMs,
final int rebalanceTimeoutMs,
final int heartbeatIntervalMs,
String groupId,
Optional<String> groupInstanceId,
long retryBackoffMs,
boolean leaveGroupOnClose) {
this.sessionTimeoutMs = sessionTimeoutMs;
this.rebalanceTimeoutMs = rebalanceTimeoutMs;
this.heartbeatIntervalMs = heartbeatIntervalMs;
this.groupId = groupId;
this.groupInstanceId = groupInstanceId;
this.retryBackoffMs = retryBackoffMs;
this.leaveGroupOnClose = leaveGroupOnClose;
}
}

View File

@@ -0,0 +1,185 @@
/*
* 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;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Deque;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
/**
* The set of requests which have been sent or are being sent but haven't yet received a response
*/
final class InFlightRequests {
private final int maxInFlightRequestsPerConnection;
private final Map<String, Deque<NetworkClient.InFlightRequest>> requests = new HashMap<>();
/** Thread safe total number of in flight requests. */
private final AtomicInteger inFlightRequestCount = new AtomicInteger(0);
public InFlightRequests(int maxInFlightRequestsPerConnection) {
this.maxInFlightRequestsPerConnection = maxInFlightRequestsPerConnection;
}
/**
* Add the given request to the queue for the connection it was directed to
*/
public void add(NetworkClient.InFlightRequest request) {
String destination = request.destination;
Deque<NetworkClient.InFlightRequest> reqs = this.requests.get(destination);
if (reqs == null) {
reqs = new ArrayDeque<>();
this.requests.put(destination, reqs);
}
reqs.addFirst(request);
inFlightRequestCount.incrementAndGet();
}
/**
* Get the request queue for the given node
*/
private Deque<NetworkClient.InFlightRequest> requestQueue(String node) {
Deque<NetworkClient.InFlightRequest> reqs = requests.get(node);
if (reqs == null || reqs.isEmpty())
throw new IllegalStateException("There are no in-flight requests for node " + node);
return reqs;
}
/**
* Get the oldest request (the one that will be completed next) for the given node
*/
public NetworkClient.InFlightRequest completeNext(String node) {
NetworkClient.InFlightRequest inFlightRequest = requestQueue(node).pollLast();
inFlightRequestCount.decrementAndGet();
return inFlightRequest;
}
/**
* Get the last request we sent to the given node (but don't remove it from the queue)
* @param node The node id
*/
public NetworkClient.InFlightRequest lastSent(String node) {
return requestQueue(node).peekFirst();
}
/**
* Complete the last request that was sent to a particular node.
* @param node The node the request was sent to
* @return The request
*/
public NetworkClient.InFlightRequest completeLastSent(String node) {
NetworkClient.InFlightRequest inFlightRequest = requestQueue(node).pollFirst();
inFlightRequestCount.decrementAndGet();
return inFlightRequest;
}
/**
* Can we send more requests to this node?
*
* @param node Node in question
* @return true iff we have no requests still being sent to the given node
*/
public boolean canSendMore(String node) {
Deque<NetworkClient.InFlightRequest> queue = requests.get(node);
return queue == null || queue.isEmpty() ||
(queue.peekFirst().send.completed() && queue.size() < this.maxInFlightRequestsPerConnection);
}
/**
* Return the number of in-flight requests directed at the given node
* @param node The node
* @return The request count.
*/
public int count(String node) {
Deque<NetworkClient.InFlightRequest> queue = requests.get(node);
return queue == null ? 0 : queue.size();
}
/**
* Return true if there is no in-flight request directed at the given node and false otherwise
*/
public boolean isEmpty(String node) {
Deque<NetworkClient.InFlightRequest> queue = requests.get(node);
return queue == null || queue.isEmpty();
}
/**
* Count all in-flight requests for all nodes. This method is thread safe, but may lag the actual count.
*/
public int count() {
return inFlightRequestCount.get();
}
/**
* Return true if there is no in-flight request and false otherwise
*/
public boolean isEmpty() {
for (Deque<NetworkClient.InFlightRequest> deque : this.requests.values()) {
if (!deque.isEmpty())
return false;
}
return true;
}
/**
* Clear out all the in-flight requests for the given node and return them
*
* @param node The node
* @return All the in-flight requests for that node that have been removed
*/
public Iterable<NetworkClient.InFlightRequest> clearAll(String node) {
Deque<NetworkClient.InFlightRequest> reqs = requests.get(node);
if (reqs == null) {
return Collections.emptyList();
} else {
final Deque<NetworkClient.InFlightRequest> clearedRequests = requests.remove(node);
inFlightRequestCount.getAndAdd(-clearedRequests.size());
return () -> clearedRequests.descendingIterator();
}
}
private Boolean hasExpiredRequest(long now, Deque<NetworkClient.InFlightRequest> deque) {
for (NetworkClient.InFlightRequest request : deque) {
long timeSinceSend = Math.max(0, now - request.sendTimeMs);
if (timeSinceSend > request.requestTimeoutMs)
return true;
}
return false;
}
/**
* Returns a list of nodes with pending in-flight request, that need to be timed out
*
* @param now current time in milliseconds
* @return list of nodes
*/
public List<String> nodesWithTimedOutRequests(long now) {
List<String> nodeIds = new ArrayList<>();
for (Map.Entry<String, Deque<NetworkClient.InFlightRequest>> requestEntry : requests.entrySet()) {
String nodeId = requestEntry.getKey();
Deque<NetworkClient.InFlightRequest> deque = requestEntry.getValue();
if (hasExpiredRequest(now, deque))
nodeIds.add(nodeId);
}
return nodeIds;
}
}

View File

@@ -0,0 +1,216 @@
/*
* 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;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.requests.AbstractRequest;
import java.io.Closeable;
import java.util.List;
/**
* The interface for {@link NetworkClient}
*/
public interface KafkaClient extends Closeable {
/**
* Check if we are currently ready to send another request to the given node but don't attempt to connect if we
* aren't.
*
* @param node The node to check
* @param now The current timestamp
*/
boolean isReady(Node node, long now);
/**
* Initiate a connection to the given node (if necessary), and return true if already connected. The readiness of a
* node will change only when poll is invoked.
*
* @param node The node to connect to.
* @param now The current time
* @return true iff we are ready to immediately initiate the sending of another request to the given node.
*/
boolean ready(Node node, long now);
/**
* Return the number of milliseconds to wait, based on the connection state, before attempting to send data. When
* disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
* connections.
*
* @param node The node to check
* @param now The current timestamp
* @return The number of milliseconds to wait.
*/
long connectionDelay(Node node, long now);
/**
* Return the number of milliseconds to wait, based on the connection state and the throttle time, before
* attempting to send data. If the connection has been established but being throttled, return throttle delay.
* Otherwise, return connection delay.
*
* @param node the connection to check
* @param now the current time in ms
*/
long pollDelayMs(Node node, long now);
/**
* Check if the connection of the node has failed, based on the connection state. Such connection failure are
* usually transient and can be resumed in the next {@link #ready(org.apache.kafka.common.Node, long)} }
* call, but there are cases where transient failures needs to be caught and re-acted upon.
*
* @param node the node to check
* @return true iff the connection has failed and the node is disconnected
*/
boolean connectionFailed(Node node);
/**
* Check if authentication to this node has failed, based on the connection state. Authentication failures are
* propagated without any retries.
*
* @param node the node to check
* @return an AuthenticationException iff authentication has failed, null otherwise
*/
AuthenticationException authenticationException(Node node);
/**
* Queue up the given request for sending. Requests can only be sent on ready connections.
* @param request The request
* @param now The current timestamp
*/
void send(ClientRequest request, long now);
/**
* Do actual reads and writes from sockets.
*
* @param timeout The maximum amount of time to wait for responses in ms, must be non-negative. The implementation
* is free to use a lower value if appropriate (common reasons for this are a lower request or
* metadata update timeout)
* @param now The current time in ms
* @throws IllegalStateException If a request is sent to an unready node
*/
List<ClientResponse> poll(long timeout, long now);
/**
* Disconnects the connection to a particular node, if there is one.
* Any pending ClientRequests for this connection will receive disconnections.
*
* @param nodeId The id of the node
*/
void disconnect(String nodeId);
/**
* Closes the connection to a particular node (if there is one).
* All requests on the connection will be cleared. ClientRequest callbacks will not be invoked
* for the cleared requests, nor will they be returned from poll().
*
* @param nodeId The id of the node
*/
void close(String nodeId);
/**
* Choose the node with the fewest outstanding requests. This method will prefer a node with an existing connection,
* but will potentially choose a node for which we don't yet have a connection if all existing connections are in
* use.
*
* @param now The current time in ms
* @return The node with the fewest in-flight requests.
*/
Node leastLoadedNode(long now);
/**
* The number of currently in-flight requests for which we have not yet returned a response
*/
int inFlightRequestCount();
/**
* Return true if there is at least one in-flight request and false otherwise.
*/
boolean hasInFlightRequests();
/**
* Get the total in-flight requests for a particular node
*
* @param nodeId The id of the node
*/
int inFlightRequestCount(String nodeId);
/**
* Return true if there is at least one in-flight request for a particular node and false otherwise.
*/
boolean hasInFlightRequests(String nodeId);
/**
* Return true if there is at least one node with connection in the READY state and not throttled. Returns false
* otherwise.
*
* @param now the current time
*/
boolean hasReadyNodes(long now);
/**
* Wake up the client if it is currently blocked waiting for I/O
*/
void wakeup();
/**
* Create a new ClientRequest.
*
* @param nodeId the node to send to
* @param requestBuilder the request builder to use
* @param createdTimeMs the time in milliseconds to use as the creation time of the request
* @param expectResponse true iff we expect a response
*/
ClientRequest newClientRequest(String nodeId, AbstractRequest.Builder<?> requestBuilder,
long createdTimeMs, boolean expectResponse);
/**
* Create a new ClientRequest.
*
* @param nodeId the node to send to
* @param requestBuilder the request builder to use
* @param createdTimeMs the time in milliseconds to use as the creation time of the request
* @param expectResponse true iff we expect a response
* @param requestTimeoutMs Upper bound time in milliseconds to await a response before disconnecting the socket and
* cancelling the request. The request may get cancelled sooner if the socket disconnects
* for any reason including if another pending request to the same node timed out first.
* @param callback the callback to invoke when we get a response
*/
ClientRequest newClientRequest(String nodeId,
AbstractRequest.Builder<?> requestBuilder,
long createdTimeMs,
boolean expectResponse,
int requestTimeoutMs,
RequestCompletionHandler callback);
/**
* Initiates shutdown of this client. This method may be invoked from another thread while this
* client is being polled. No further requests may be sent using the client. The current poll()
* will be terminated using wakeup(). The client should be explicitly shutdown using {@link #close()}
* after poll returns. Note that {@link #close()} should not be invoked concurrently while polling.
*/
void initiateClose();
/**
* Returns true if the client is still active. Returns false if {@link #initiateClose()} or {@link #close()}
* was invoked for this client.
*/
boolean active();
}

View File

@@ -0,0 +1,87 @@
/*
* 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;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.RequestHeader;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
/**
* A simple implementation of `MetadataUpdater` that returns the cluster nodes set via the constructor or via
* `setNodes`.
*
* This is useful in cases where automatic metadata updates are not required. An example is controller/broker
* communication.
*
* This class is not thread-safe!
*/
public class ManualMetadataUpdater implements MetadataUpdater {
private List<Node> nodes;
public ManualMetadataUpdater() {
this(new ArrayList<Node>(0));
}
public ManualMetadataUpdater(List<Node> nodes) {
this.nodes = nodes;
}
public void setNodes(List<Node> nodes) {
this.nodes = nodes;
}
@Override
public List<Node> fetchNodes() {
return new ArrayList<>(nodes);
}
@Override
public boolean isUpdateDue(long now) {
return false;
}
@Override
public long maybeUpdate(long now) {
return Long.MAX_VALUE;
}
@Override
public void handleServerDisconnect(long now, String nodeId, Optional<AuthenticationException> maybeAuthException) {
// We don't fail the broker on failures. There should be sufficient information from
// the NetworkClient logs to indicate the reason for the failure.
}
@Override
public void handleFailedRequest(long now, Optional<KafkaException> maybeFatalException) {
// Do nothing
}
@Override
public void handleSuccessfulResponse(RequestHeader requestHeader, long now, MetadataResponse response) {
// Do nothing
}
@Override
public void close() {
}
}

View File

@@ -0,0 +1,603 @@
/*
* 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;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.InvalidMetadataException;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.MetadataRequest;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.utils.LogContext;
import org.slf4j.Logger;
import java.io.Closeable;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.function.Supplier;
import static org.apache.kafka.common.record.RecordBatch.NO_PARTITION_LEADER_EPOCH;
/**
* A class encapsulating some of the logic around metadata.
* <p>
* This class is shared by the client thread (for partitioning) and the background sender thread.
*
* Metadata is maintained for only a subset of topics, which can be added to over time. When we request metadata for a
* topic we don't have any metadata for it will trigger a metadata update.
* <p>
* If topic expiry is enabled for the metadata, any topic that has not been used within the expiry interval
* is removed from the metadata refresh set after an update. Consumers disable topic expiry since they explicitly
* manage topics while producers rely on topic expiry to limit the refresh set.
*/
public class Metadata implements Closeable {
private final Logger log;
private final long refreshBackoffMs;
private final long metadataExpireMs;
private int updateVersion; // bumped on every metadata response
private int requestVersion; // bumped on every new topic addition
private long lastRefreshMs;
private long lastSuccessfulRefreshMs;
private KafkaException fatalException;
private Set<String> invalidTopics;
private Set<String> unauthorizedTopics;
private MetadataCache cache = MetadataCache.empty();
private boolean needFullUpdate;
private boolean needPartialUpdate;
private final ClusterResourceListeners clusterResourceListeners;
private boolean isClosed;
private final Map<TopicPartition, Integer> lastSeenLeaderEpochs;
/**
* Create a new Metadata instance
*
* @param refreshBackoffMs The minimum amount of time that must expire between metadata refreshes to avoid busy
* polling
* @param metadataExpireMs The maximum amount of time that metadata can be retained without refresh
* @param logContext Log context corresponding to the containing client
* @param clusterResourceListeners List of ClusterResourceListeners which will receive metadata updates.
*/
public Metadata(long refreshBackoffMs,
long metadataExpireMs,
LogContext logContext,
ClusterResourceListeners clusterResourceListeners) {
this.log = logContext.logger(Metadata.class);
this.refreshBackoffMs = refreshBackoffMs;
this.metadataExpireMs = metadataExpireMs;
this.lastRefreshMs = 0L;
this.lastSuccessfulRefreshMs = 0L;
this.requestVersion = 0;
this.updateVersion = 0;
this.needFullUpdate = false;
this.needPartialUpdate = false;
this.clusterResourceListeners = clusterResourceListeners;
this.isClosed = false;
this.lastSeenLeaderEpochs = new HashMap<>();
this.invalidTopics = Collections.emptySet();
this.unauthorizedTopics = Collections.emptySet();
}
/**
* Get the current cluster info without blocking
*/
public synchronized Cluster fetch() {
return cache.cluster();
}
/**
* Return the next time when the current cluster info can be updated (i.e., backoff time has elapsed).
*
* @param nowMs current time in ms
* @return remaining time in ms till the cluster info can be updated again
*/
public synchronized long timeToAllowUpdate(long nowMs) {
return Math.max(this.lastRefreshMs + this.refreshBackoffMs - nowMs, 0);
}
/**
* The next time to update the cluster info is the maximum of the time the current info will expire and the time the
* current info can be updated (i.e. backoff time has elapsed); If an update has been request then the expiry time
* is now
*
* @param nowMs current time in ms
* @return remaining time in ms till updating the cluster info
*/
public synchronized long timeToNextUpdate(long nowMs) {
long timeToExpire = updateRequested() ? 0 : Math.max(this.lastSuccessfulRefreshMs + this.metadataExpireMs - nowMs, 0);
return Math.max(timeToExpire, timeToAllowUpdate(nowMs));
}
public long metadataExpireMs() {
return this.metadataExpireMs;
}
/**
* Request an update of the current cluster metadata info, return the current updateVersion before the update
*/
public synchronized int requestUpdate() {
this.needFullUpdate = true;
return this.updateVersion;
}
public synchronized int requestUpdateForNewTopics() {
// Override the timestamp of last refresh to let immediate update.
this.lastRefreshMs = 0;
this.needPartialUpdate = true;
this.requestVersion++;
return this.updateVersion;
}
/**
* Request an update for the partition metadata iff we have seen a newer leader epoch. This is called by the client
* any time it handles a response from the broker that includes leader epoch, except for UpdateMetadata which
* follows a different code path ({@link #update}).
*
* @param topicPartition
* @param leaderEpoch
* @return true if we updated the last seen epoch, false otherwise
*/
public synchronized boolean updateLastSeenEpochIfNewer(TopicPartition topicPartition, int leaderEpoch) {
Objects.requireNonNull(topicPartition, "TopicPartition cannot be null");
if (leaderEpoch < 0)
throw new IllegalArgumentException("Invalid leader epoch " + leaderEpoch + " (must be non-negative)");
Integer oldEpoch = lastSeenLeaderEpochs.get(topicPartition);
log.trace("Determining if we should replace existing epoch {} with new epoch {} for partition {}", oldEpoch, leaderEpoch, topicPartition);
final boolean updated;
if (oldEpoch == null) {
log.debug("Not replacing null epoch with new epoch {} for partition {}", leaderEpoch, topicPartition);
updated = false;
} else if (leaderEpoch > oldEpoch) {
log.debug("Updating last seen epoch from {} to {} for partition {}", oldEpoch, leaderEpoch, topicPartition);
lastSeenLeaderEpochs.put(topicPartition, leaderEpoch);
updated = true;
} else {
log.debug("Not replacing existing epoch {} with new epoch {} for partition {}", oldEpoch, leaderEpoch, topicPartition);
updated = false;
}
this.needFullUpdate = this.needFullUpdate || updated;
return updated;
}
public Optional<Integer> lastSeenLeaderEpoch(TopicPartition topicPartition) {
return Optional.ofNullable(lastSeenLeaderEpochs.get(topicPartition));
}
/**
* Check whether an update has been explicitly requested.
*
* @return true if an update was requested, false otherwise
*/
public synchronized boolean updateRequested() {
return this.needFullUpdate || this.needPartialUpdate;
}
/**
* Return the cached partition info if it exists and a newer leader epoch isn't known about.
*/
synchronized Optional<MetadataResponse.PartitionMetadata> partitionMetadataIfCurrent(TopicPartition topicPartition) {
Integer epoch = lastSeenLeaderEpochs.get(topicPartition);
Optional<MetadataResponse.PartitionMetadata> partitionMetadata = cache.partitionMetadata(topicPartition);
if (epoch == null) {
// old cluster format (no epochs)
return partitionMetadata;
} else {
return partitionMetadata.filter(metadata ->
metadata.leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH).equals(epoch));
}
}
public synchronized LeaderAndEpoch currentLeader(TopicPartition topicPartition) {
Optional<MetadataResponse.PartitionMetadata> maybeMetadata = partitionMetadataIfCurrent(topicPartition);
if (!maybeMetadata.isPresent())
return new LeaderAndEpoch(Optional.empty(), Optional.ofNullable(lastSeenLeaderEpochs.get(topicPartition)));
MetadataResponse.PartitionMetadata partitionMetadata = maybeMetadata.get();
Optional<Integer> leaderEpochOpt = partitionMetadata.leaderEpoch;
Optional<Node> leaderNodeOpt = partitionMetadata.leaderId.flatMap(cache::nodeById);
return new LeaderAndEpoch(leaderNodeOpt, leaderEpochOpt);
}
public synchronized void bootstrap(List<InetSocketAddress> addresses) {
this.needFullUpdate = true;
this.updateVersion += 1;
this.cache = MetadataCache.bootstrap(addresses);
}
/**
* Update metadata assuming the current request version. This is mainly for convenience in testing.
*/
public synchronized void updateWithCurrentRequestVersion(MetadataResponse response, boolean isPartialUpdate, long nowMs) {
this.update(this.requestVersion, response, isPartialUpdate, nowMs);
}
/**
* Updates the cluster metadata. If topic expiry is enabled, expiry time
* is set for topics if required and expired topics are removed from the metadata.
*
* @param requestVersion The request version corresponding to the update response, as provided by
* {@link #newMetadataRequestAndVersion()}.
* @param response metadata response received from the broker
* @param isPartialUpdate whether the metadata request was for a subset of the active topics
* @param nowMs current time in milliseconds
*/
public synchronized void update(int requestVersion, MetadataResponse response, boolean isPartialUpdate, long nowMs) {
Objects.requireNonNull(response, "Metadata response cannot be null");
if (isClosed())
throw new IllegalStateException("Update requested after metadata close");
this.needPartialUpdate = requestVersion < this.requestVersion;
this.lastRefreshMs = nowMs;
this.updateVersion += 1;
if (!isPartialUpdate) {
this.needFullUpdate = false;
this.lastSuccessfulRefreshMs = nowMs;
}
String previousClusterId = cache.clusterResource().clusterId();
this.cache = handleMetadataResponse(response, isPartialUpdate, nowMs);
Cluster cluster = cache.cluster();
maybeSetMetadataError(cluster);
this.lastSeenLeaderEpochs.keySet().removeIf(tp -> !retainTopic(tp.topic(), false, nowMs));
String newClusterId = cache.clusterResource().clusterId();
if (!Objects.equals(previousClusterId, newClusterId)) {
log.info("Cluster ID: {}", newClusterId);
}
clusterResourceListeners.onUpdate(cache.clusterResource());
log.debug("Updated cluster metadata updateVersion {} to {}", this.updateVersion, this.cache);
}
private void maybeSetMetadataError(Cluster cluster) {
clearRecoverableErrors();
checkInvalidTopics(cluster);
checkUnauthorizedTopics(cluster);
}
private void checkInvalidTopics(Cluster cluster) {
if (!cluster.invalidTopics().isEmpty()) {
log.error("Metadata response reported invalid topics {}", cluster.invalidTopics());
invalidTopics = new HashSet<>(cluster.invalidTopics());
}
}
private void checkUnauthorizedTopics(Cluster cluster) {
if (!cluster.unauthorizedTopics().isEmpty()) {
log.error("Topic authorization failed for topics {}", cluster.unauthorizedTopics());
unauthorizedTopics = new HashSet<>(cluster.unauthorizedTopics());
}
}
/**
* Transform a MetadataResponse into a new MetadataCache instance.
*/
private MetadataCache handleMetadataResponse(MetadataResponse metadataResponse, boolean isPartialUpdate, long nowMs) {
// All encountered topics.
Set<String> topics = new HashSet<>();
// Retained topics to be passed to the metadata cache.
Set<String> internalTopics = new HashSet<>();
Set<String> unauthorizedTopics = new HashSet<>();
Set<String> invalidTopics = new HashSet<>();
List<MetadataResponse.PartitionMetadata> partitions = new ArrayList<>();
for (MetadataResponse.TopicMetadata metadata : metadataResponse.topicMetadata()) {
topics.add(metadata.topic());
if (!retainTopic(metadata.topic(), metadata.isInternal(), nowMs))
continue;
if (metadata.isInternal())
internalTopics.add(metadata.topic());
if (metadata.error() == Errors.NONE) {
for (MetadataResponse.PartitionMetadata partitionMetadata : metadata.partitionMetadata()) {
// Even if the partition's metadata includes an error, we need to handle
// the update to catch new epochs
updateLatestMetadata(partitionMetadata, metadataResponse.hasReliableLeaderEpochs())
.ifPresent(partitions::add);
if (partitionMetadata.error.exception() instanceof InvalidMetadataException) {
log.debug("Requesting metadata update for partition {} due to error {}",
partitionMetadata.topicPartition, partitionMetadata.error);
requestUpdate();
}
}
} else {
if (metadata.error().exception() instanceof InvalidMetadataException) {
log.debug("Requesting metadata update for topic {} due to error {}", metadata.topic(), metadata.error());
requestUpdate();
}
if (metadata.error() == Errors.INVALID_TOPIC_EXCEPTION)
invalidTopics.add(metadata.topic());
else if (metadata.error() == Errors.TOPIC_AUTHORIZATION_FAILED)
unauthorizedTopics.add(metadata.topic());
}
}
Map<Integer, Node> nodes = metadataResponse.brokersById();
if (isPartialUpdate)
return this.cache.mergeWith(metadataResponse.clusterId(), nodes, partitions,
unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller(),
(topic, isInternal) -> !topics.contains(topic) && retainTopic(topic, isInternal, nowMs));
else
return new MetadataCache(metadataResponse.clusterId(), nodes, partitions,
unauthorizedTopics, invalidTopics, internalTopics, metadataResponse.controller());
}
/**
* Compute the latest partition metadata to cache given ordering by leader epochs (if both
* available and reliable).
*/
private Optional<MetadataResponse.PartitionMetadata> updateLatestMetadata(
MetadataResponse.PartitionMetadata partitionMetadata,
boolean hasReliableLeaderEpoch) {
TopicPartition tp = partitionMetadata.topicPartition;
if (hasReliableLeaderEpoch && partitionMetadata.leaderEpoch.isPresent()) {
int newEpoch = partitionMetadata.leaderEpoch.get();
// If the received leader epoch is at least the same as the previous one, update the metadata
Integer currentEpoch = lastSeenLeaderEpochs.get(tp);
if (currentEpoch == null || newEpoch >= currentEpoch) {
log.debug("Updating last seen epoch for partition {} from {} to epoch {} from new metadata", tp, currentEpoch, newEpoch);
lastSeenLeaderEpochs.put(tp, newEpoch);
return Optional.of(partitionMetadata);
} else {
// Otherwise ignore the new metadata and use the previously cached info
log.debug("Got metadata for an older epoch {} (current is {}) for partition {}, not updating", newEpoch, currentEpoch, tp);
return cache.partitionMetadata(tp);
}
} else {
// Handle old cluster formats as well as error responses where leader and epoch are missing
lastSeenLeaderEpochs.remove(tp);
return Optional.of(partitionMetadata.withoutLeaderEpoch());
}
}
/**
* If any non-retriable exceptions were encountered during metadata update, clear and throw the exception.
* This is used by the consumer to propagate any fatal exceptions or topic exceptions for any of the topics
* in the consumer's Metadata.
*/
public synchronized void maybeThrowAnyException() {
clearErrorsAndMaybeThrowException(this::recoverableException);
}
/**
* If any fatal exceptions were encountered during metadata update, throw the exception. This is used by
* the producer to abort waiting for metadata if there were fatal exceptions (e.g. authentication failures)
* in the last metadata update.
*/
public synchronized void maybeThrowFatalException() {
KafkaException metadataException = this.fatalException;
if (metadataException != null) {
fatalException = null;
throw metadataException;
}
}
/**
* If any non-retriable exceptions were encountered during metadata update, throw exception if the exception
* is fatal or related to the specified topic. All exceptions from the last metadata update are cleared.
* This is used by the producer to propagate topic metadata errors for send requests.
*/
public synchronized void maybeThrowExceptionForTopic(String topic) {
clearErrorsAndMaybeThrowException(() -> recoverableExceptionForTopic(topic));
}
private void clearErrorsAndMaybeThrowException(Supplier<KafkaException> recoverableExceptionSupplier) {
KafkaException metadataException = Optional.ofNullable(fatalException).orElseGet(recoverableExceptionSupplier);
fatalException = null;
clearRecoverableErrors();
if (metadataException != null)
throw metadataException;
}
// We may be able to recover from this exception if metadata for this topic is no longer needed
private KafkaException recoverableException() {
if (!unauthorizedTopics.isEmpty())
return new TopicAuthorizationException(unauthorizedTopics);
else if (!invalidTopics.isEmpty())
return new InvalidTopicException(invalidTopics);
else
return null;
}
private KafkaException recoverableExceptionForTopic(String topic) {
if (unauthorizedTopics.contains(topic))
return new TopicAuthorizationException(Collections.singleton(topic));
else if (invalidTopics.contains(topic))
return new InvalidTopicException(Collections.singleton(topic));
else
return null;
}
private void clearRecoverableErrors() {
invalidTopics = Collections.emptySet();
unauthorizedTopics = Collections.emptySet();
}
/**
* Record an attempt to update the metadata that failed. We need to keep track of this
* to avoid retrying immediately.
*/
public synchronized void failedUpdate(long now) {
this.lastRefreshMs = now;
}
/**
* Propagate a fatal error which affects the ability to fetch metadata for the cluster.
* Two examples are authentication and unsupported version exceptions.
*
* @param exception The fatal exception
*/
public synchronized void fatalError(KafkaException exception) {
this.fatalException = exception;
}
/**
* @return The current metadata updateVersion
*/
public synchronized int updateVersion() {
return this.updateVersion;
}
/**
* The last time metadata was successfully updated.
*/
public synchronized long lastSuccessfulUpdate() {
return this.lastSuccessfulRefreshMs;
}
/**
* Close this metadata instance to indicate that metadata updates are no longer possible.
*/
@Override
public synchronized void close() {
this.isClosed = true;
}
/**
* Check if this metadata instance has been closed. See {@link #close()} for more information.
*
* @return True if this instance has been closed; false otherwise
*/
public synchronized boolean isClosed() {
return this.isClosed;
}
public synchronized MetadataRequestAndVersion newMetadataRequestAndVersion(long nowMs) {
MetadataRequest.Builder request = null;
boolean isPartialUpdate = false;
// Perform a partial update only if a full update hasn't been requested, and the last successful
// hasn't exceeded the metadata refresh time.
if (!this.needFullUpdate && this.lastSuccessfulRefreshMs + this.metadataExpireMs > nowMs) {
request = newMetadataRequestBuilderForNewTopics();
isPartialUpdate = true;
}
if (request == null) {
request = newMetadataRequestBuilder();
isPartialUpdate = false;
}
return new MetadataRequestAndVersion(request, requestVersion, isPartialUpdate);
}
/**
* Constructs and returns a metadata request builder for fetching cluster data and all active topics.
*
* @return the constructed non-null metadata builder
*/
protected MetadataRequest.Builder newMetadataRequestBuilder() {
return MetadataRequest.Builder.allTopics();
}
/**
* Constructs and returns a metadata request builder for fetching cluster data and any uncached topics,
* otherwise null if the functionality is not supported.
*
* @return the constructed metadata builder, or null if not supported
*/
protected MetadataRequest.Builder newMetadataRequestBuilderForNewTopics() {
return null;
}
protected boolean retainTopic(String topic, boolean isInternal, long nowMs) {
return true;
}
public static class MetadataRequestAndVersion {
public final MetadataRequest.Builder requestBuilder;
public final int requestVersion;
public final boolean isPartialUpdate;
private MetadataRequestAndVersion(MetadataRequest.Builder requestBuilder,
int requestVersion,
boolean isPartialUpdate) {
this.requestBuilder = requestBuilder;
this.requestVersion = requestVersion;
this.isPartialUpdate = isPartialUpdate;
}
}
/**
* Represents current leader state known in metadata. It is possible that we know the leader, but not the
* epoch if the metadata is received from a broker which does not support a sufficient Metadata API version.
* It is also possible that we know of the leader epoch, but not the leader when it is derived
* from an external source (e.g. a committed offset).
*/
public static class LeaderAndEpoch {
private static final LeaderAndEpoch NO_LEADER_OR_EPOCH = new LeaderAndEpoch(Optional.empty(), Optional.empty());
public final Optional<Node> leader;
public final Optional<Integer> epoch;
public LeaderAndEpoch(Optional<Node> leader, Optional<Integer> epoch) {
this.leader = Objects.requireNonNull(leader);
this.epoch = Objects.requireNonNull(epoch);
}
public static LeaderAndEpoch noLeaderOrEpoch() {
return NO_LEADER_OR_EPOCH;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
LeaderAndEpoch that = (LeaderAndEpoch) o;
if (!leader.equals(that.leader)) return false;
return epoch.equals(that.epoch);
}
@Override
public int hashCode() {
int result = leader.hashCode();
result = 31 * result + epoch.hashCode();
return result;
}
@Override
public String toString() {
return "LeaderAndEpoch{" +
"leader=" + leader +
", epoch=" + epoch.map(Number::toString).orElse("absent") +
'}';
}
}
}

View File

@@ -0,0 +1,210 @@
/*
* 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;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.ClusterResource;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.MetadataResponse.PartitionMetadata;
import java.net.InetSocketAddress;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.function.BiPredicate;
import java.util.function.Predicate;
import java.util.stream.Collectors;
/**
* An internal mutable cache of nodes, topics, and partitions in the Kafka cluster. This keeps an up-to-date Cluster
* instance which is optimized for read access.
*/
public class MetadataCache {
private final String clusterId;
private final Map<Integer, Node> nodes;
private final Set<String> unauthorizedTopics;
private final Set<String> invalidTopics;
private final Set<String> internalTopics;
private final Node controller;
private final Map<TopicPartition, PartitionMetadata> metadataByPartition;
private Cluster clusterInstance;
MetadataCache(String clusterId,
Map<Integer, Node> nodes,
Collection<PartitionMetadata> partitions,
Set<String> unauthorizedTopics,
Set<String> invalidTopics,
Set<String> internalTopics,
Node controller) {
this(clusterId, nodes, partitions, unauthorizedTopics, invalidTopics, internalTopics, controller, null);
}
private MetadataCache(String clusterId,
Map<Integer, Node> nodes,
Collection<PartitionMetadata> partitions,
Set<String> unauthorizedTopics,
Set<String> invalidTopics,
Set<String> internalTopics,
Node controller,
Cluster clusterInstance) {
this.clusterId = clusterId;
this.nodes = nodes;
this.unauthorizedTopics = unauthorizedTopics;
this.invalidTopics = invalidTopics;
this.internalTopics = internalTopics;
this.controller = controller;
this.metadataByPartition = new HashMap<>(partitions.size());
for (PartitionMetadata p : partitions) {
this.metadataByPartition.put(p.topicPartition, p);
}
if (clusterInstance == null) {
computeClusterView();
} else {
this.clusterInstance = clusterInstance;
}
}
Optional<PartitionMetadata> partitionMetadata(TopicPartition topicPartition) {
return Optional.ofNullable(metadataByPartition.get(topicPartition));
}
Optional<Node> nodeById(int id) {
return Optional.ofNullable(nodes.get(id));
}
Cluster cluster() {
if (clusterInstance == null) {
throw new IllegalStateException("Cached Cluster instance should not be null, but was.");
} else {
return clusterInstance;
}
}
ClusterResource clusterResource() {
return new ClusterResource(clusterId);
}
/**
* Merges the metadata cache's contents with the provided metadata, returning a new metadata cache. The provided
* metadata is presumed to be more recent than the cache's metadata, and therefore all overlapping metadata will
* be overridden.
*
* @param newClusterId the new cluster Id
* @param newNodes the new set of nodes
* @param addPartitions partitions to add
* @param addUnauthorizedTopics unauthorized topics to add
* @param addInternalTopics internal topics to add
* @param newController the new controller node
* @param retainTopic returns whether a topic's metadata should be retained
* @return the merged metadata cache
*/
MetadataCache mergeWith(String newClusterId,
Map<Integer, Node> newNodes,
Collection<PartitionMetadata> addPartitions,
Set<String> addUnauthorizedTopics,
Set<String> addInvalidTopics,
Set<String> addInternalTopics,
Node newController,
BiPredicate<String, Boolean> retainTopic) {
Predicate<String> shouldRetainTopic = topic -> retainTopic.test(topic, internalTopics.contains(topic));
Map<TopicPartition, PartitionMetadata> newMetadataByPartition = new HashMap<>(addPartitions.size());
for (PartitionMetadata partition : addPartitions) {
newMetadataByPartition.put(partition.topicPartition, partition);
}
for (Map.Entry<TopicPartition, PartitionMetadata> entry : metadataByPartition.entrySet()) {
if (shouldRetainTopic.test(entry.getKey().topic())) {
newMetadataByPartition.putIfAbsent(entry.getKey(), entry.getValue());
}
}
Set<String> newUnauthorizedTopics = fillSet(addUnauthorizedTopics, unauthorizedTopics, shouldRetainTopic);
Set<String> newInvalidTopics = fillSet(addInvalidTopics, invalidTopics, shouldRetainTopic);
Set<String> newInternalTopics = fillSet(addInternalTopics, internalTopics, shouldRetainTopic);
return new MetadataCache(newClusterId, newNodes, newMetadataByPartition.values(), newUnauthorizedTopics,
newInvalidTopics, newInternalTopics, newController);
}
/**
* Copies {@code baseSet} and adds all non-existent elements in {@code fillSet} such that {@code predicate} is true.
* In other words, all elements of {@code baseSet} will be contained in the result, with additional non-overlapping
* elements in {@code fillSet} where the predicate is true.
*
* @param baseSet the base elements for the resulting set
* @param fillSet elements to be filled into the resulting set
* @param predicate tested against the fill set to determine whether elements should be added to the base set
*/
private static <T> Set<T> fillSet(Set<T> baseSet, Set<T> fillSet, Predicate<T> predicate) {
Set<T> result = new HashSet<>(baseSet);
for (T element : fillSet) {
if (predicate.test(element)) {
result.add(element);
}
}
return result;
}
private void computeClusterView() {
List<PartitionInfo> partitionInfos = metadataByPartition.values()
.stream()
.map(metadata -> MetadataResponse.toPartitionInfo(metadata, nodes))
.collect(Collectors.toList());
this.clusterInstance = new Cluster(clusterId, nodes.values(), partitionInfos, unauthorizedTopics,
invalidTopics, internalTopics, controller);
}
static MetadataCache bootstrap(List<InetSocketAddress> addresses) {
Map<Integer, Node> nodes = new HashMap<>();
int nodeId = -1;
for (InetSocketAddress address : addresses) {
nodes.put(nodeId, new Node(nodeId, address.getHostString(), address.getPort()));
nodeId--;
}
return new MetadataCache(null, nodes, Collections.emptyList(),
Collections.emptySet(), Collections.emptySet(), Collections.emptySet(),
null, Cluster.bootstrap(addresses));
}
static MetadataCache empty() {
return new MetadataCache(null, Collections.emptyMap(), Collections.emptyList(),
Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), null, Cluster.empty());
}
@Override
public String toString() {
return "MetadataCache{" +
"clusterId='" + clusterId + '\'' +
", nodes=" + nodes +
", partitions=" + metadataByPartition.values() +
", controller=" + controller +
'}';
}
}

View File

@@ -0,0 +1,93 @@
/*
* 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;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.RequestHeader;
import java.io.Closeable;
import java.util.List;
import java.util.Optional;
/**
* The interface used by `NetworkClient` to request cluster metadata info to be updated and to retrieve the cluster nodes
* from such metadata. This is an internal class.
* <p>
* This class is not thread-safe!
*/
public interface MetadataUpdater extends Closeable {
/**
* Gets the current cluster info without blocking.
*/
List<Node> fetchNodes();
/**
* Returns true if an update to the cluster metadata info is due.
*/
boolean isUpdateDue(long now);
/**
* Starts a cluster metadata update if needed and possible. Returns the time until the metadata update (which would
* be 0 if an update has been started as a result of this call).
*
* If the implementation relies on `NetworkClient` to send requests, `handleSuccessfulResponse` will be
* invoked after the metadata response is received.
*
* The semantics of `needed` and `possible` are implementation-dependent and may take into account a number of
* factors like node availability, how long since the last metadata update, etc.
*/
long maybeUpdate(long now);
/**
* Handle a server disconnect.
*
* This provides a mechanism for the `MetadataUpdater` implementation to use the NetworkClient instance for its own
* requests with special handling for disconnections of such requests.
*
* @param now Current time in milliseconds
* @param nodeId The id of the node that disconnected
* @param maybeAuthException Optional authentication error
*/
void handleServerDisconnect(long now, String nodeId, Optional<AuthenticationException> maybeAuthException);
/**
* Handle a metadata request failure.
*
* @param now Current time in milliseconds
* @param maybeFatalException Optional fatal error (e.g. {@link UnsupportedVersionException})
*/
void handleFailedRequest(long now, Optional<KafkaException> maybeFatalException);
/**
* Handle responses for metadata requests.
*
* This provides a mechanism for the `MetadataUpdater` implementation to use the NetworkClient instance for its own
* requests with special handling for completed receives of such requests.
*/
void handleSuccessfulResponse(RequestHeader requestHeader, long now, MetadataResponse metadataResponse);
/**
* Close this updater.
*/
@Override
void close();
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,118 @@
/*
* 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;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.errors.DisconnectException;
import org.apache.kafka.common.utils.Time;
import java.io.IOException;
import java.util.List;
/**
* Provides additional utilities for {@link NetworkClient} (e.g. to implement blocking behaviour).
*/
public final class NetworkClientUtils {
private NetworkClientUtils() {}
/**
* Checks whether the node is currently connected, first calling `client.poll` to ensure that any pending
* disconnects have been processed.
*
* This method can be used to check the status of a connection prior to calling the blocking version to be able
* to tell whether the latter completed a new connection.
*/
public static boolean isReady(KafkaClient client, Node node, long currentTime) {
client.poll(0, currentTime);
return client.isReady(node, currentTime);
}
/**
* Invokes `client.poll` to discard pending disconnects, followed by `client.ready` and 0 or more `client.poll`
* invocations until the connection to `node` is ready, the timeoutMs expires or the connection fails.
*
* It returns `true` if the call completes normally or `false` if the timeoutMs expires. If the connection fails,
* an `IOException` is thrown instead. Note that if the `NetworkClient` has been configured with a positive
* connection timeoutMs, it is possible for this method to raise an `IOException` for a previous connection which
* has recently disconnected. If authentication to the node fails, an `AuthenticationException` is thrown.
*
* This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with
* care.
*/
public static boolean awaitReady(KafkaClient client, Node node, Time time, long timeoutMs) throws IOException {
if (timeoutMs < 0) {
throw new IllegalArgumentException("Timeout needs to be greater than 0");
}
long startTime = time.milliseconds();
long expiryTime = startTime + timeoutMs;
if (isReady(client, node, startTime) || client.ready(node, startTime))
return true;
long attemptStartTime = time.milliseconds();
while (!client.isReady(node, attemptStartTime) && attemptStartTime < expiryTime) {
if (client.connectionFailed(node)) {
throw new IOException("Connection to " + node + " failed.");
}
long pollTimeout = expiryTime - attemptStartTime;
client.poll(pollTimeout, attemptStartTime);
if (client.authenticationException(node) != null)
throw client.authenticationException(node);
attemptStartTime = time.milliseconds();
}
return client.isReady(node, attemptStartTime);
}
/**
* Invokes `client.send` followed by 1 or more `client.poll` invocations until a response is received or a
* disconnection happens (which can happen for a number of reasons including a request timeout).
*
* In case of a disconnection, an `IOException` is thrown.
* If shutdown is initiated on the client during this method, an IOException is thrown.
*
* This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with
* care.
*/
public static ClientResponse sendAndReceive(KafkaClient client, ClientRequest request, Time time) throws IOException {
try {
client.send(request, time.milliseconds());
while (client.active()) {
List<ClientResponse> responses = client.poll(Long.MAX_VALUE, time.milliseconds());
for (ClientResponse response : responses) {
if (response.requestHeader().correlationId() == request.correlationId()) {
if (response.wasDisconnected()) {
throw new IOException("Connection to " + response.destination() + " was disconnected before the response was read");
}
if (response.versionMismatch() != null) {
throw response.versionMismatch();
}
return response;
}
}
}
throw new IOException("Client was shutdown before response was read");
} catch (DisconnectException e) {
if (client.active())
throw e;
else
throw new IOException("Client was shutdown before response was read");
}
}
}

View File

@@ -0,0 +1,233 @@
/*
* 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;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedList;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKey;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKeyCollection;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.utils.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.EnumMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
/**
* An internal class which represents the API versions supported by a particular node.
*/
public class NodeApiVersions {
private static final Logger log = LoggerFactory.getLogger(NodeApiVersions.class);
// A map of the usable versions of each API, keyed by the ApiKeys instance
private final Map<ApiKeys, ApiVersion> supportedVersions = new EnumMap<>(ApiKeys.class);
// List of APIs which the broker supports, but which are unknown to the client
private final List<ApiVersion> unknownApis = new ArrayList<>();
/**
* Create a NodeApiVersions object with the current ApiVersions.
*
* @return A new NodeApiVersions object.
*/
public static NodeApiVersions create() {
return create(Collections.<ApiVersion>emptyList());
}
/**
* Create a NodeApiVersions object.
*
* @param overrides API versions to override. Any ApiVersion not specified here will be set to the current client
* value.
* @return A new NodeApiVersions object.
*/
public static NodeApiVersions create(Collection<ApiVersion> overrides) {
List<ApiVersion> apiVersions = new LinkedList<>(overrides);
for (ApiKeys apiKey : ApiKeys.values()) {
boolean exists = false;
for (ApiVersion apiVersion : apiVersions) {
if (apiVersion.apiKey == apiKey.id) {
exists = true;
break;
}
}
if (!exists) {
apiVersions.add(new ApiVersion(apiKey));
}
}
return new NodeApiVersions(apiVersions);
}
/**
* Create a NodeApiVersions object with a single ApiKey. It is mainly used in tests.
*
* @param apiKey ApiKey's id.
* @param minVersion ApiKey's minimum version.
* @param maxVersion ApiKey's maximum version.
* @return A new NodeApiVersions object.
*/
public static NodeApiVersions create(short apiKey, short minVersion, short maxVersion) {
return create(Collections.singleton(new ApiVersion(apiKey, minVersion, maxVersion)));
}
public NodeApiVersions(ApiVersionsResponseKeyCollection nodeApiVersions) {
for (ApiVersionsResponseKey nodeApiVersion : nodeApiVersions) {
if (ApiKeys.hasId(nodeApiVersion.apiKey()) && ApiKeys.forId(nodeApiVersion.apiKey()) != null) {
ApiKeys nodeApiKey = ApiKeys.forId(nodeApiVersion.apiKey());
supportedVersions.put(nodeApiKey, new ApiVersion(nodeApiVersion));
} else {
// Newer brokers may support ApiKeys we don't know about
unknownApis.add(new ApiVersion(nodeApiVersion));
}
}
}
public NodeApiVersions(Collection<ApiVersion> nodeApiVersions) {
for (ApiVersion nodeApiVersion : nodeApiVersions) {
if (ApiKeys.hasId(nodeApiVersion.apiKey)) {
ApiKeys nodeApiKey = ApiKeys.forId(nodeApiVersion.apiKey);
supportedVersions.put(nodeApiKey, nodeApiVersion);
} else {
// Newer brokers may support ApiKeys we don't know about
unknownApis.add(nodeApiVersion);
}
}
}
/**
* Return the most recent version supported by both the node and the local software.
*/
public short latestUsableVersion(ApiKeys apiKey) {
return latestUsableVersion(apiKey, apiKey.oldestVersion(), apiKey.latestVersion());
}
/**
* Get the latest version supported by the broker within an allowed range of versions
*/
public short latestUsableVersion(ApiKeys apiKey, short oldestAllowedVersion, short latestAllowedVersion) {
ApiVersion usableVersion = supportedVersions.get(apiKey);
if (usableVersion == null)
throw new UnsupportedVersionException("The broker does not support " + apiKey);
return latestUsableVersion(apiKey, usableVersion, oldestAllowedVersion, latestAllowedVersion);
}
private short latestUsableVersion(ApiKeys apiKey, ApiVersion supportedVersions,
short minAllowedVersion, short maxAllowedVersion) {
short minVersion = (short) Math.max(minAllowedVersion, supportedVersions.minVersion);
short maxVersion = (short) Math.min(maxAllowedVersion, supportedVersions.maxVersion);
if (minVersion > maxVersion)
throw new UnsupportedVersionException("The broker does not support " + apiKey +
" with version in range [" + minAllowedVersion + "," + maxAllowedVersion + "]. The supported" +
" range is [" + supportedVersions.minVersion + "," + supportedVersions.maxVersion + "].");
return maxVersion;
}
/**
* Convert the object to a string with no linebreaks.<p/>
* <p>
* This toString method is relatively expensive, so avoid calling it unless debug logging is turned on.
*/
@Override
public String toString() {
return toString(false);
}
/**
* Convert the object to a string.
*
* @param lineBreaks True if we should add a linebreak after each api.
*/
public String toString(boolean lineBreaks) {
// The apiVersion collection may not be in sorted order. We put it into
// a TreeMap before printing it out to ensure that we always print in
// ascending order.
TreeMap<Short, String> apiKeysText = new TreeMap<>();
for (ApiVersion supportedVersion : this.supportedVersions.values())
apiKeysText.put(supportedVersion.apiKey, apiVersionToText(supportedVersion));
for (ApiVersion apiVersion : unknownApis)
apiKeysText.put(apiVersion.apiKey, apiVersionToText(apiVersion));
// Also handle the case where some apiKey types are not specified at all in the given ApiVersions,
// which may happen when the remote is too old.
for (ApiKeys apiKey : ApiKeys.values()) {
if (!apiKeysText.containsKey(apiKey.id)) {
StringBuilder bld = new StringBuilder();
bld.append(apiKey.name).append("(").
append(apiKey.id).append("): ").append("UNSUPPORTED");
apiKeysText.put(apiKey.id, bld.toString());
}
}
String separator = lineBreaks ? ",\n\t" : ", ";
StringBuilder bld = new StringBuilder();
bld.append("(");
if (lineBreaks)
bld.append("\n\t");
bld.append(Utils.join(apiKeysText.values(), separator));
if (lineBreaks)
bld.append("\n");
bld.append(")");
return bld.toString();
}
private String apiVersionToText(ApiVersion apiVersion) {
StringBuilder bld = new StringBuilder();
ApiKeys apiKey = null;
if (ApiKeys.hasId(apiVersion.apiKey)) {
apiKey = ApiKeys.forId(apiVersion.apiKey);
bld.append(apiKey.name).append("(").append(apiKey.id).append("): ");
} else {
bld.append("UNKNOWN(").append(apiVersion.apiKey).append("): ");
}
if (apiVersion.minVersion == apiVersion.maxVersion) {
bld.append(apiVersion.minVersion);
} else {
bld.append(apiVersion.minVersion).append(" to ").append(apiVersion.maxVersion);
}
if (apiKey != null) {
ApiVersion supportedVersion = supportedVersions.get(apiKey);
if (apiKey.latestVersion() < supportedVersion.minVersion) {
bld.append(" [unusable: node too new]");
} else if (supportedVersion.maxVersion < apiKey.oldestVersion()) {
bld.append(" [unusable: node too old]");
} else {
short latestUsableVersion = Utils.min(apiKey.latestVersion(), supportedVersion.maxVersion);
bld.append(" [usable: ").append(latestUsableVersion).append("]");
}
}
return bld.toString();
}
/**
* Get the version information for a given API.
*
* @param apiKey The api key to lookup
* @return The api version information from the broker or null if it is unsupported
*/
public ApiVersion apiVersion(ApiKeys apiKey) {
return supportedVersions.get(apiKey);
}
}

View File

@@ -0,0 +1,27 @@
/*
* 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;
/**
* A callback interface for attaching an action to be executed when a request is complete and the corresponding response
* has been received. This handler will also be invoked if there is a disconnection while handling the request.
*/
public interface RequestCompletionHandler {
public void onComplete(ClientResponse response);
}

View File

@@ -0,0 +1,35 @@
/*
* 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;
import org.apache.kafka.common.errors.InvalidMetadataException;
/**
* Thrown when current metadata cannot be used. This is often used as a way to trigger a metadata
* update before retrying another operation.
*
* Note: this is not a public API.
*/
public class StaleMetadataException extends InvalidMetadataException {
private static final long serialVersionUID = 1L;
public StaleMetadataException() {}
public StaleMetadataException(String message) {
super(message);
}
}

View File

@@ -0,0 +1,47 @@
/*
* 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.admin;
/*
* This class implements the common APIs that are shared by Options classes for various AdminClient commands
*/
public abstract class AbstractOptions<T extends AbstractOptions> {
protected Integer timeoutMs = null;
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*/
@SuppressWarnings("unchecked")
public T timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return (T) this;
}
/**
* The timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*/
public Integer timeoutMs() {
return timeoutMs;
}
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,52 @@
/*
* 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.admin;
import java.util.Map;
import java.util.Properties;
/**
* The base class for in-built admin clients.
*
* Client code should use the newer {@link Admin} interface in preference to this class.
*
* This class may be removed in a later release, but has not be marked as deprecated to avoid unnecessary noise.
*/
public abstract class AdminClient implements Admin {
/**
* Create a new Admin with the given configuration.
*
* @param props The configuration.
* @return The new KafkaAdminClient.
*/
public static AdminClient create(Properties props) {
return (AdminClient) Admin.create(props);
}
/**
* Create a new Admin with the given configuration.
*
* @param conf The configuration.
* @return The new KafkaAdminClient.
*/
public static AdminClient create(Map<String, Object> conf) {
return (AdminClient) Admin.create(conf);
}
}

View File

@@ -0,0 +1,231 @@
/*
* 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.admin;
import org.apache.kafka.clients.ClientDnsLookup;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.SecurityConfig;
import org.apache.kafka.common.metrics.Sensor;
import java.util.Map;
import java.util.Set;
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
import static org.apache.kafka.common.config.ConfigDef.Range.between;
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
/**
* The AdminClient configuration class, which also contains constants for configuration entry names.
*/
public class AdminClientConfig extends AbstractConfig {
private static final ConfigDef CONFIG;
/**
* <code>bootstrap.servers</code>
*/
public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
private static final String BOOTSTRAP_SERVERS_DOC = CommonClientConfigs.BOOTSTRAP_SERVERS_DOC;
/**
* <code>client.dns.lookup</code>
*/
public static final String CLIENT_DNS_LOOKUP_CONFIG = CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG;
private static final String CLIENT_DNS_LOOKUP_DOC = CommonClientConfigs.CLIENT_DNS_LOOKUP_DOC;
/**
* <code>reconnect.backoff.ms</code>
*/
public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG;
private static final String RECONNECT_BACKOFF_MS_DOC = CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC;
/**
* <code>reconnect.backoff.max.ms</code>
*/
public static final String RECONNECT_BACKOFF_MAX_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG;
private static final String RECONNECT_BACKOFF_MAX_MS_DOC = CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_DOC;
/**
* <code>retry.backoff.ms</code>
*/
public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG;
private static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to " +
"retry a failed request. This avoids repeatedly sending requests in a tight loop under " +
"some failure scenarios.";
/** <code>connections.max.idle.ms</code> */
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;
private static final String CONNECTIONS_MAX_IDLE_MS_DOC = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC;
/** <code>request.timeout.ms</code> */
public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG;
private static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC;
public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
private static final String CLIENT_ID_DOC = CommonClientConfigs.CLIENT_ID_DOC;
public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG;
private static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC;
public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG;
private static final String SEND_BUFFER_DOC = CommonClientConfigs.SEND_BUFFER_DOC;
public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG;
private static final String RECEIVE_BUFFER_DOC = CommonClientConfigs.RECEIVE_BUFFER_DOC;
public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG;
private static final String METRIC_REPORTER_CLASSES_DOC = CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC;
public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG;
private static final String METRICS_NUM_SAMPLES_DOC = CommonClientConfigs.METRICS_NUM_SAMPLES_DOC;
public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
private static final String METRICS_SAMPLE_WINDOW_MS_DOC = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC;
public static final String METRICS_RECORDING_LEVEL_CONFIG = CommonClientConfigs.METRICS_RECORDING_LEVEL_CONFIG;
public static final String SECURITY_PROTOCOL_CONFIG = CommonClientConfigs.SECURITY_PROTOCOL_CONFIG;
public static final String DEFAULT_SECURITY_PROTOCOL = CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL;
private static final String SECURITY_PROTOCOL_DOC = CommonClientConfigs.SECURITY_PROTOCOL_DOC;
private static final String METRICS_RECORDING_LEVEL_DOC = CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC;
public static final String RETRIES_CONFIG = CommonClientConfigs.RETRIES_CONFIG;
public static final String DEFAULT_API_TIMEOUT_MS_CONFIG = CommonClientConfigs.DEFAULT_API_TIMEOUT_MS_CONFIG;
/**
* <code>security.providers</code>
*/
public static final String SECURITY_PROVIDERS_CONFIG = SecurityConfig.SECURITY_PROVIDERS_CONFIG;
private static final String SECURITY_PROVIDERS_DOC = SecurityConfig.SECURITY_PROVIDERS_DOC;
static {
CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
Type.LIST,
Importance.HIGH,
BOOTSTRAP_SERVERS_DOC)
.define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CLIENT_ID_DOC)
.define(METADATA_MAX_AGE_CONFIG, Type.LONG, 5 * 60 * 1000, atLeast(0), Importance.LOW, METADATA_MAX_AGE_DOC)
.define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(-1), Importance.MEDIUM, SEND_BUFFER_DOC)
.define(RECEIVE_BUFFER_CONFIG, Type.INT, 64 * 1024, atLeast(-1), Importance.MEDIUM, RECEIVE_BUFFER_DOC)
.define(RECONNECT_BACKOFF_MS_CONFIG,
Type.LONG,
50L,
atLeast(0L),
Importance.LOW,
RECONNECT_BACKOFF_MS_DOC)
.define(RECONNECT_BACKOFF_MAX_MS_CONFIG,
Type.LONG,
1000L,
atLeast(0L),
Importance.LOW,
RECONNECT_BACKOFF_MAX_MS_DOC)
.define(RETRY_BACKOFF_MS_CONFIG,
Type.LONG,
100L,
atLeast(0L),
Importance.LOW,
RETRY_BACKOFF_MS_DOC)
.define(REQUEST_TIMEOUT_MS_CONFIG,
Type.INT,
30000,
atLeast(0),
Importance.MEDIUM,
REQUEST_TIMEOUT_MS_DOC)
.define(CONNECTIONS_MAX_IDLE_MS_CONFIG,
Type.LONG,
5 * 60 * 1000,
Importance.MEDIUM,
CONNECTIONS_MAX_IDLE_MS_DOC)
.define(RETRIES_CONFIG,
Type.INT,
Integer.MAX_VALUE,
between(0, Integer.MAX_VALUE),
Importance.LOW,
CommonClientConfigs.RETRIES_DOC)
.define(DEFAULT_API_TIMEOUT_MS_CONFIG,
Type.INT,
60000,
atLeast(0),
Importance.MEDIUM,
CommonClientConfigs.DEFAULT_API_TIMEOUT_MS_DOC)
.define(METRICS_SAMPLE_WINDOW_MS_CONFIG,
Type.LONG,
30000,
atLeast(0),
Importance.LOW,
METRICS_SAMPLE_WINDOW_MS_DOC)
.define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC)
.define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC)
.define(METRICS_RECORDING_LEVEL_CONFIG,
Type.STRING,
Sensor.RecordingLevel.INFO.toString(),
in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString()),
Importance.LOW,
METRICS_RECORDING_LEVEL_DOC)
.define(CLIENT_DNS_LOOKUP_CONFIG,
Type.STRING,
ClientDnsLookup.DEFAULT.toString(),
in(ClientDnsLookup.DEFAULT.toString(),
ClientDnsLookup.USE_ALL_DNS_IPS.toString(),
ClientDnsLookup.RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY.toString()),
Importance.MEDIUM,
CLIENT_DNS_LOOKUP_DOC)
// security support
.define(SECURITY_PROVIDERS_CONFIG,
Type.STRING,
null,
Importance.LOW,
SECURITY_PROVIDERS_DOC)
.define(SECURITY_PROTOCOL_CONFIG,
Type.STRING,
DEFAULT_SECURITY_PROTOCOL,
Importance.MEDIUM,
SECURITY_PROTOCOL_DOC)
.withClientSslSupport()
.withClientSaslSupport();
}
@Override
protected Map<String, Object> postProcessParsedConfig(final Map<String, Object> parsedValues) {
return CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues);
}
public AdminClientConfig(Map<?, ?> props) {
this(props, false);
}
protected AdminClientConfig(Map<?, ?> props, boolean doLog) {
super(CONFIG, props, doLog);
}
public static Set<String> configNames() {
return CONFIG.names();
}
public static ConfigDef configDef() {
return new ConfigDef(CONFIG);
}
public static void main(String[] args) {
System.out.println(CONFIG.toHtml());
}
}

View File

@@ -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.kafka.clients.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
import java.util.function.Function;
import java.util.stream.Collectors;
/**
* A class representing a alter configuration entry containing name, value and operation type.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class AlterConfigOp {
public enum OpType {
SET((byte) 0), DELETE((byte) 1), APPEND((byte) 2), SUBTRACT((byte) 3);
private static final Map<Byte, OpType> OP_TYPES = Collections.unmodifiableMap(
Arrays.stream(values()).collect(Collectors.toMap(OpType::id, Function.identity()))
);
private final byte id;
OpType(final byte id) {
this.id = id;
}
public byte id() {
return id;
}
public static OpType forId(final byte id) {
return OP_TYPES.get(id);
}
}
private final ConfigEntry configEntry;
private final OpType opType;
public AlterConfigOp(ConfigEntry configEntry, OpType operationType) {
this.configEntry = configEntry;
this.opType = operationType;
}
public ConfigEntry configEntry() {
return configEntry;
};
public OpType opType() {
return opType;
};
@Override
public boolean equals(final Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
final AlterConfigOp that = (AlterConfigOp) o;
return opType == that.opType &&
Objects.equals(configEntry, that.configEntry);
}
@Override
public int hashCode() {
return Objects.hash(opType, configEntry);
}
@Override
public String toString() {
return "AlterConfigOp{" +
"opType=" + opType +
", configEntry=" + configEntry +
'}';
}
}

View File

@@ -0,0 +1,59 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
/**
* Options for {@link Admin#alterConfigs(Map)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class AlterConfigsOptions extends AbstractOptions<AlterConfigsOptions> {
private boolean validateOnly = false;
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public AlterConfigsOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
/**
* Return true if the request should be validated without altering the configs.
*/
public boolean shouldValidateOnly() {
return validateOnly;
}
/**
* Set to true if the request should be validated without altering the configs.
*/
public AlterConfigsOptions validateOnly(boolean validateOnly) {
this.validateOnly = validateOnly;
return this;
}
}

View File

@@ -0,0 +1,54 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.config.ConfigResource;
import java.util.Map;
/**
* The result of the {@link Admin#alterConfigs(Map)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class AlterConfigsResult {
private final Map<ConfigResource, KafkaFuture<Void>> futures;
AlterConfigsResult(Map<ConfigResource, KafkaFuture<Void>> futures) {
this.futures = futures;
}
/**
* Return a map from resources to futures which can be used to check the status of the operation on each resource.
*/
public Map<ConfigResource, KafkaFuture<Void>> values() {
return futures;
}
/**
* Return a future which succeeds only if all the alter configs operations succeed.
*/
public KafkaFuture<Void> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]));
}
}

View File

@@ -0,0 +1,28 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for the {@link AdminClient#alterConsumerGroupOffsets(String, Map)} call.
*
* The API of this class is evolving, see {@link AdminClient} for details.
*/
@InterfaceStability.Evolving
public class AlterConsumerGroupOffsetsOptions extends AbstractOptions<AlterConsumerGroupOffsetsOptions> {
}

View File

@@ -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.kafka.clients.admin;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.KafkaFuture.BaseFunction;
import org.apache.kafka.common.KafkaFuture.BiConsumer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.internals.KafkaFutureImpl;
import org.apache.kafka.common.protocol.Errors;
/**
* The result of the {@link AdminClient#alterConsumerGroupOffsets(String, Map)} call.
*
* The API of this class is evolving, see {@link AdminClient} for details.
*/
@InterfaceStability.Evolving
public class AlterConsumerGroupOffsetsResult {
private final KafkaFuture<Map<TopicPartition, Errors>> future;
AlterConsumerGroupOffsetsResult(KafkaFuture<Map<TopicPartition, Errors>> future) {
this.future = future;
}
/**
* Return a future which can be used to check the result for a given partition.
*/
public KafkaFuture<Void> partitionResult(final TopicPartition partition) {
final KafkaFutureImpl<Void> result = new KafkaFutureImpl<>();
this.future.whenComplete(new BiConsumer<Map<TopicPartition, Errors>, Throwable>() {
@Override
public void accept(final Map<TopicPartition, Errors> topicPartitions, final Throwable throwable) {
if (throwable != null) {
result.completeExceptionally(throwable);
} else if (!topicPartitions.containsKey(partition)) {
result.completeExceptionally(new IllegalArgumentException(
"Alter offset for partition \"" + partition + "\" was not attempted"));
} else {
final Errors error = topicPartitions.get(partition);
if (error == Errors.NONE) {
result.complete(null);
} else {
result.completeExceptionally(error.exception());
}
}
}
});
return result;
}
/**
* Return a future which succeeds if all the alter offsets succeed.
*/
public KafkaFuture<Void> all() {
return this.future.thenApply(new BaseFunction<Map<TopicPartition, Errors>, Void>() {
@Override
public Void apply(final Map<TopicPartition, Errors> topicPartitionErrorsMap) {
List<TopicPartition> partitionsFailed = topicPartitionErrorsMap.entrySet()
.stream()
.filter(e -> e.getValue() != Errors.NONE)
.map(Map.Entry::getKey)
.collect(Collectors.toList());
for (Errors error : topicPartitionErrorsMap.values()) {
if (error != Errors.NONE) {
throw error.exception(
"Failed altering consumer group offsets for the following partitions: " + partitionsFailed);
}
}
return null;
}
});
}
}

View File

@@ -0,0 +1,31 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
/**
* Options for {@link AdminClient#alterPartitionReassignments(Map, AlterPartitionReassignmentsOptions)}
*
* The API of this class is evolving. See {@link AdminClient} for details.
*/
@InterfaceStability.Evolving
public class AlterPartitionReassignmentsOptions extends AbstractOptions<AlterPartitionReassignmentsOptions> {
}

View File

@@ -0,0 +1,59 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
/**
* The result of {@link AdminClient#alterPartitionReassignments(Map, AlterPartitionReassignmentsOptions)}.
*
* The API of this class is evolving. See {@link AdminClient} for details.
*/
@InterfaceStability.Evolving
public class AlterPartitionReassignmentsResult {
private final Map<TopicPartition, KafkaFuture<Void>> futures;
AlterPartitionReassignmentsResult(Map<TopicPartition, KafkaFuture<Void>> futures) {
this.futures = futures;
}
/**
* Return a map from partitions to futures which can be used to check the status of the reassignment.
*
* Possible error codes:
*
* INVALID_REPLICA_ASSIGNMENT (39) - if the specified replica assignment was not valid -- for example, if it included negative numbers, repeated numbers, or specified a broker ID that the controller was not aware of.
* NO_REASSIGNMENT_IN_PROGRESS (85) - if the request wants to cancel reassignments but none exist
* UNKNOWN (-1)
*
*/
public Map<TopicPartition, KafkaFuture<Void>> values() {
return futures;
}
/**
* Return a future which succeeds only if all the reassignments were successfully initiated.
*/
public KafkaFuture<Void> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]));
}
}

View File

@@ -0,0 +1,29 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
/**
* Options for {@link Admin#alterReplicaLogDirs(Map, AlterReplicaLogDirsOptions)}.
*/
@InterfaceStability.Evolving
public class AlterReplicaLogDirsOptions extends AbstractOptions<AlterReplicaLogDirsOptions> {
}

View File

@@ -0,0 +1,79 @@
/*
* 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.admin;
import java.util.Map;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartitionReplica;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.errors.ClusterAuthorizationException;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.KafkaStorageException;
import org.apache.kafka.common.errors.LogDirNotFoundException;
import org.apache.kafka.common.errors.ReplicaNotAvailableException;
import org.apache.kafka.common.errors.UnknownServerException;
/**
* The result of {@link Admin#alterReplicaLogDirs(Map, AlterReplicaLogDirsOptions)}.
*
* To retrieve the detailed result per specified {@link TopicPartitionReplica}, use {@link #values()}. To retrieve the
* overall result only, use {@link #all()}.
*/
@InterfaceStability.Evolving
public class AlterReplicaLogDirsResult {
private final Map<TopicPartitionReplica, KafkaFuture<Void>> futures;
AlterReplicaLogDirsResult(Map<TopicPartitionReplica, KafkaFuture<Void>> futures) {
this.futures = futures;
}
/**
* Return a map from {@link TopicPartitionReplica} to {@link KafkaFuture} which holds the status of individual
* replica movement.
*
* To check the result of individual replica movement, call {@link KafkaFuture#get()} from the value contained
* in the returned map. If there is no error, it will return silently; if not, an {@link Exception} will be thrown
* like the following:
*
* <ul>
* <li>{@link CancellationException}: The task was canceled.</li>
* <li>{@link InterruptedException}: Interrupted while joining I/O thread.</li>
* <li>{@link ExecutionException}: Execution failed with the following causes:</li>
* <ul>
* <li>{@link ClusterAuthorizationException}: Authorization failed. (CLUSTER_AUTHORIZATION_FAILED, 31)</li>
* <li>{@link InvalidTopicException}: The specified topic name is too long. (INVALID_TOPIC_EXCEPTION, 17)</li>
* <li>{@link LogDirNotFoundException}: The specified log directory is not found in the broker. (LOG_DIR_NOT_FOUND, 57)</li>
* <li>{@link ReplicaNotAvailableException}: The replica does not exist on the broker. (REPLICA_NOT_AVAILABLE, 9)</li>
* <li>{@link KafkaStorageException}: Disk error occurred. (KAFKA_STORAGE_ERROR, 56)</li>
* <li>{@link UnknownServerException}: Unknown. (UNKNOWN_SERVER_ERROR, -1)</li>
* </ul>
* </ul>
*/
public Map<TopicPartitionReplica, KafkaFuture<Void>> values() {
return futures;
}
/**
* Return a {@link KafkaFuture} which succeeds on {@link KafkaFuture#get()} if all the replica movement have succeeded.
* if not, it throws an {@link Exception} described in {@link #values()} method.
*/
public KafkaFuture<Void> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]));
}
}

View File

@@ -0,0 +1,81 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
/**
* A configuration object containing the configuration entries for a resource.
* <p>
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class Config {
private final Map<String, ConfigEntry> entries = new HashMap<>();
/**
* Create a configuration instance with the provided entries.
*/
public Config(Collection<ConfigEntry> entries) {
for (ConfigEntry entry : entries) {
this.entries.put(entry.name(), entry);
}
}
/**
* Configuration entries for a resource.
*/
public Collection<ConfigEntry> entries() {
return Collections.unmodifiableCollection(entries.values());
}
/**
* Get the configuration entry with the provided name or null if there isn't one.
*/
public ConfigEntry get(String name) {
return entries.get(name);
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
Config config = (Config) o;
return entries.equals(config.entries);
}
@Override
public int hashCode() {
return entries.hashCode();
}
@Override
public String toString() {
return "Config(entries=" + entries.values() + ")";
}
}

View File

@@ -0,0 +1,266 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
/**
* A class representing a configuration entry containing name, value and additional metadata.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ConfigEntry {
private final String name;
private final String value;
private final ConfigSource source;
private final boolean isSensitive;
private final boolean isReadOnly;
private final List<ConfigSynonym> synonyms;
/**
* Create a configuration entry with the provided values.
*
* @param name the non-null config name
* @param value the config value or null
*/
public ConfigEntry(String name, String value) {
this(name, value, false, false, false);
}
/**
* Create a configuration with the provided values.
*
* @param name the non-null config name
* @param value the config value or null
* @param isDefault whether the config value is the default or if it's been explicitly set
* @param isSensitive whether the config value is sensitive, the broker never returns the value if it is sensitive
* @param isReadOnly whether the config is read-only and cannot be updated
* @deprecated since 1.1.0. This constructor will be removed in a future release.
*/
@Deprecated
public ConfigEntry(String name, String value, boolean isDefault, boolean isSensitive, boolean isReadOnly) {
this(name,
value,
isDefault ? ConfigSource.DEFAULT_CONFIG : ConfigSource.UNKNOWN,
isSensitive,
isReadOnly,
Collections.<ConfigSynonym>emptyList());
}
/**
* Create a configuration with the provided values.
*
* @param name the non-null config name
* @param value the config value or null
* @param source the source of this config entry
* @param isSensitive whether the config value is sensitive, the broker never returns the value if it is sensitive
* @param isReadOnly whether the config is read-only and cannot be updated
* @param synonyms Synonym configs in order of precedence
*/
ConfigEntry(String name, String value, ConfigSource source, boolean isSensitive, boolean isReadOnly,
List<ConfigSynonym> synonyms) {
Objects.requireNonNull(name, "name should not be null");
this.name = name;
this.value = value;
this.source = source;
this.isSensitive = isSensitive;
this.isReadOnly = isReadOnly;
this.synonyms = synonyms;
}
/**
* Return the config name.
*/
public String name() {
return name;
}
/**
* Return the value or null. Null is returned if the config is unset or if isSensitive is true.
*/
public String value() {
return value;
}
/**
* Return the source of this configuration entry.
*/
public ConfigSource source() {
return source;
}
/**
* Return whether the config value is the default or if it's been explicitly set.
*/
public boolean isDefault() {
return source == ConfigSource.DEFAULT_CONFIG;
}
/**
* Return whether the config value is sensitive. The value is always set to null by the broker if the config value
* is sensitive.
*/
public boolean isSensitive() {
return isSensitive;
}
/**
* Return whether the config is read-only and cannot be updated.
*/
public boolean isReadOnly() {
return isReadOnly;
}
/**
* Returns all config values that may be used as the value of this config along with their source,
* in the order of precedence. The list starts with the value returned in this ConfigEntry.
* The list is empty if synonyms were not requested using {@link DescribeConfigsOptions#includeSynonyms(boolean)}
*/
public List<ConfigSynonym> synonyms() {
return synonyms;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
ConfigEntry that = (ConfigEntry) o;
return this.name.equals(that.name) &&
this.value != null ? this.value.equals(that.value) : that.value == null &&
this.isSensitive == that.isSensitive &&
this.isReadOnly == that.isReadOnly &&
this.source == that.source &&
Objects.equals(this.synonyms, that.synonyms);
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + name.hashCode();
result = prime * result + ((value == null) ? 0 : value.hashCode());
result = prime * result + (isSensitive ? 1 : 0);
result = prime * result + (isReadOnly ? 1 : 0);
result = prime * result + source.hashCode();
result = prime * result + synonyms.hashCode();
return result;
}
@Override
public String toString() {
return "ConfigEntry(" +
"name=" + name +
", value=" + value +
", source=" + source +
", isSensitive=" + isSensitive +
", isReadOnly=" + isReadOnly +
", synonyms=" + synonyms +
")";
}
/**
* Source of configuration entries.
*/
public enum ConfigSource {
DYNAMIC_TOPIC_CONFIG, // dynamic topic config that is configured for a specific topic
DYNAMIC_BROKER_LOGGER_CONFIG, // dynamic broker logger config that is configured for a specific broker
DYNAMIC_BROKER_CONFIG, // dynamic broker config that is configured for a specific broker
DYNAMIC_DEFAULT_BROKER_CONFIG, // dynamic broker config that is configured as default for all brokers in the cluster
STATIC_BROKER_CONFIG, // static broker config provided as broker properties at start up (e.g. server.properties file)
DEFAULT_CONFIG, // built-in default configuration for configs that have a default value
UNKNOWN // source unknown e.g. in the ConfigEntry used for alter requests where source is not set
}
/**
* Class representing a configuration synonym of a {@link ConfigEntry}.
*/
public static class ConfigSynonym {
private final String name;
private final String value;
private final ConfigSource source;
/**
* Create a configuration synonym with the provided values.
*
* @param name Configuration name (this may be different from the name of the associated {@link ConfigEntry}
* @param value Configuration value
* @param source {@link ConfigSource} of this configuraton
*/
ConfigSynonym(String name, String value, ConfigSource source) {
this.name = name;
this.value = value;
this.source = source;
}
/**
* Returns the name of this configuration.
*/
public String name() {
return name;
}
/**
* Returns the value of this configuration, which may be null if the configuration is sensitive.
*/
public String value() {
return value;
}
/**
* Returns the source of this configuration.
*/
public ConfigSource source() {
return source;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
ConfigSynonym that = (ConfigSynonym) o;
return Objects.equals(name, that.name) && Objects.equals(value, that.value) && source == that.source;
}
@Override
public int hashCode() {
return Objects.hash(name, value, source);
}
@Override
public String toString() {
return "ConfigSynonym(" +
"name=" + name +
", value=" + value +
", source=" + source +
")";
}
}
}

View File

@@ -0,0 +1,148 @@
/*
* 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.admin;
import org.apache.kafka.common.ConsumerGroupState;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.utils.Utils;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Objects;
import java.util.Set;
/**
* A detailed description of a single consumer group in the cluster.
*/
public class ConsumerGroupDescription {
private final String groupId;
private final boolean isSimpleConsumerGroup;
private final Collection<MemberDescription> members;
private final String partitionAssignor;
private final ConsumerGroupState state;
private final Node coordinator;
private final Set<AclOperation> authorizedOperations;
public ConsumerGroupDescription(String groupId,
boolean isSimpleConsumerGroup,
Collection<MemberDescription> members,
String partitionAssignor,
ConsumerGroupState state,
Node coordinator) {
this(groupId, isSimpleConsumerGroup, members, partitionAssignor, state, coordinator, Collections.emptySet());
}
ConsumerGroupDescription(String groupId,
boolean isSimpleConsumerGroup,
Collection<MemberDescription> members,
String partitionAssignor,
ConsumerGroupState state,
Node coordinator,
Set<AclOperation> authorizedOperations) {
this.groupId = groupId == null ? "" : groupId;
this.isSimpleConsumerGroup = isSimpleConsumerGroup;
this.members = members == null ? Collections.emptyList() :
Collections.unmodifiableList(new ArrayList<>(members));
this.partitionAssignor = partitionAssignor == null ? "" : partitionAssignor;
this.state = state;
this.coordinator = coordinator;
this.authorizedOperations = authorizedOperations;
}
@Override
public boolean equals(final Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
final ConsumerGroupDescription that = (ConsumerGroupDescription) o;
return isSimpleConsumerGroup == that.isSimpleConsumerGroup &&
Objects.equals(groupId, that.groupId) &&
Objects.equals(members, that.members) &&
Objects.equals(partitionAssignor, that.partitionAssignor) &&
state == that.state &&
Objects.equals(coordinator, that.coordinator) &&
Objects.equals(authorizedOperations, that.authorizedOperations);
}
@Override
public int hashCode() {
return Objects.hash(groupId, isSimpleConsumerGroup, members, partitionAssignor, state, coordinator, authorizedOperations);
}
/**
* The id of the consumer group.
*/
public String groupId() {
return groupId;
}
/**
* If consumer group is simple or not.
*/
public boolean isSimpleConsumerGroup() {
return isSimpleConsumerGroup;
}
/**
* A list of the members of the consumer group.
*/
public Collection<MemberDescription> members() {
return members;
}
/**
* The consumer group partition assignor.
*/
public String partitionAssignor() {
return partitionAssignor;
}
/**
* The consumer group state, or UNKNOWN if the state is too new for us to parse.
*/
public ConsumerGroupState state() {
return state;
}
/**
* The consumer group coordinator, or null if the coordinator is not known.
*/
public Node coordinator() {
return coordinator;
}
/**
* authorizedOperations for this group, or null if that information is not known.
*/
public Set<AclOperation> authorizedOperations() {
return authorizedOperations;
}
@Override
public String toString() {
return "(groupId=" + groupId +
", isSimpleConsumerGroup=" + isSimpleConsumerGroup +
", members=" + Utils.join(members, ",") +
", partitionAssignor=" + partitionAssignor +
", state=" + state +
", coordinator=" + coordinator +
", authorizedOperations=" + authorizedOperations +
")";
}
}

View File

@@ -0,0 +1,59 @@
/*
* 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.admin;
/**
* A listing of a consumer group in the cluster.
*/
public class ConsumerGroupListing {
private final String groupId;
private final boolean isSimpleConsumerGroup;
/**
* Create an instance with the specified parameters.
*
* @param groupId Group Id
* @param isSimpleConsumerGroup If consumer group is simple or not.
*/
public ConsumerGroupListing(String groupId, boolean isSimpleConsumerGroup) {
this.groupId = groupId;
this.isSimpleConsumerGroup = isSimpleConsumerGroup;
}
/**
* Consumer Group Id
*/
public String groupId() {
return groupId;
}
/**
* If Consumer Group is simple or not.
*/
public boolean isSimpleConsumerGroup() {
return isSimpleConsumerGroup;
}
@Override
public String toString() {
return "(" +
"groupId='" + groupId + '\'' +
", isSimpleConsumerGroup=" + isSimpleConsumerGroup +
')';
}
}

View File

@@ -0,0 +1,43 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for {@link Admin#createAcls(Collection)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class CreateAclsOptions extends AbstractOptions<CreateAclsOptions> {
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public CreateAclsOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
}

View File

@@ -0,0 +1,54 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.Map;
/**
* The result of the {@link Admin#createAcls(Collection)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class CreateAclsResult {
private final Map<AclBinding, KafkaFuture<Void>> futures;
CreateAclsResult(Map<AclBinding, KafkaFuture<Void>> futures) {
this.futures = futures;
}
/**
* Return a map from ACL bindings to futures which can be used to check the status of the creation of each ACL
* binding.
*/
public Map<AclBinding, KafkaFuture<Void>> values() {
return futures;
}
/**
* Return a future which succeeds only if all the ACL creations succeed.
*/
public KafkaFuture<Void> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]));
}
}

View File

@@ -0,0 +1,53 @@
/*
* 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.admin;
import java.util.LinkedList;
import java.util.List;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.security.auth.KafkaPrincipal;
/**
* Options for {@link Admin#createDelegationToken(CreateDelegationTokenOptions)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class CreateDelegationTokenOptions extends AbstractOptions<CreateDelegationTokenOptions> {
private long maxLifeTimeMs = -1;
private List<KafkaPrincipal> renewers = new LinkedList<>();
public CreateDelegationTokenOptions renewers(List<KafkaPrincipal> renewers) {
this.renewers = renewers;
return this;
}
public List<KafkaPrincipal> renewers() {
return renewers;
}
public CreateDelegationTokenOptions maxlifeTimeMs(long maxLifeTimeMs) {
this.maxLifeTimeMs = maxLifeTimeMs;
return this;
}
public long maxlifeTimeMs() {
return maxLifeTimeMs;
}
}

View File

@@ -0,0 +1,43 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.security.token.delegation.DelegationToken;
/**
* The result of the {@link KafkaAdminClient#createDelegationToken(CreateDelegationTokenOptions)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class CreateDelegationTokenResult {
private final KafkaFuture<DelegationToken> delegationToken;
CreateDelegationTokenResult(KafkaFuture<DelegationToken> delegationToken) {
this.delegationToken = delegationToken;
}
/**
* Returns a future which yields a delegation token
*/
public KafkaFuture<DelegationToken> delegationToken() {
return delegationToken;
}
}

View File

@@ -0,0 +1,51 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
/**
* Options for {@link Admin#createPartitions(Map)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class CreatePartitionsOptions extends AbstractOptions<CreatePartitionsOptions> {
private boolean validateOnly = false;
public CreatePartitionsOptions() {
}
/**
* Return true if the request should be validated without creating new partitions.
*/
public boolean validateOnly() {
return validateOnly;
}
/**
* Set to true if the request should be validated without creating new partitions.
*/
public CreatePartitionsOptions validateOnly(boolean validateOnly) {
this.validateOnly = validateOnly;
return this;
}
}

View File

@@ -0,0 +1,53 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
/**
* The result of the {@link Admin#createPartitions(Map)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class CreatePartitionsResult {
private final Map<String, KafkaFuture<Void>> values;
CreatePartitionsResult(Map<String, KafkaFuture<Void>> values) {
this.values = values;
}
/**
* Return a map from topic names to futures, which can be used to check the status of individual
* partition creations.
*/
public Map<String, KafkaFuture<Void>> values() {
return values;
}
/**
* Return a future which succeeds if all the partition creations succeed.
*/
public KafkaFuture<Void> all() {
return KafkaFuture.allOf(values.values().toArray(new KafkaFuture[0]));
}
}

View File

@@ -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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for {@link Admin#createTopics(Collection)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class CreateTopicsOptions extends AbstractOptions<CreateTopicsOptions> {
private boolean validateOnly = false;
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public CreateTopicsOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
/**
* Set to true if the request should be validated without creating the topic.
*/
public CreateTopicsOptions validateOnly(boolean validateOnly) {
this.validateOnly = validateOnly;
return this;
}
/**
* Return true if the request should be validated without creating the topic.
*/
public boolean shouldValidateOnly() {
return validateOnly;
}
}

View File

@@ -0,0 +1,137 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.errors.ApiException;
import java.util.Collection;
import java.util.Map;
import java.util.stream.Collectors;
/**
* The result of {@link Admin#createTopics(Collection)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class CreateTopicsResult {
final static int UNKNOWN = -1;
private final Map<String, KafkaFuture<TopicMetadataAndConfig>> futures;
protected CreateTopicsResult(Map<String, KafkaFuture<TopicMetadataAndConfig>> futures) {
this.futures = futures;
}
/**
* Return a map from topic names to futures, which can be used to check the status of individual
* topic creations.
*/
public Map<String, KafkaFuture<Void>> values() {
return futures.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().thenApply(v -> (Void) null)));
}
/**
* Return a future which succeeds if all the topic creations succeed.
*/
public KafkaFuture<Void> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]));
}
/**
* Returns a future that provides topic configs for the topic when the request completes.
* <p>
* If broker version doesn't support replication factor in the response, throw
* {@link org.apache.kafka.common.errors.UnsupportedVersionException}.
* If broker returned an error for topic configs, throw appropriate exception. For example,
* {@link org.apache.kafka.common.errors.TopicAuthorizationException} is thrown if user does not
* have permission to describe topic configs.
*/
public KafkaFuture<Config> config(String topic) {
return futures.get(topic).thenApply(TopicMetadataAndConfig::config);
}
/**
* Returns a future that provides number of partitions in the topic when the request completes.
* <p>
* If broker version doesn't support replication factor in the response, throw
* {@link org.apache.kafka.common.errors.UnsupportedVersionException}.
* If broker returned an error for topic configs, throw appropriate exception. For example,
* {@link org.apache.kafka.common.errors.TopicAuthorizationException} is thrown if user does not
* have permission to describe topic configs.
*/
public KafkaFuture<Integer> numPartitions(String topic) {
return futures.get(topic).thenApply(TopicMetadataAndConfig::numPartitions);
}
/**
* Returns a future that provides replication factor for the topic when the request completes.
* <p>
* If broker version doesn't support replication factor in the response, throw
* {@link org.apache.kafka.common.errors.UnsupportedVersionException}.
* If broker returned an error for topic configs, throw appropriate exception. For example,
* {@link org.apache.kafka.common.errors.TopicAuthorizationException} is thrown if user does not
* have permission to describe topic configs.
*/
public KafkaFuture<Integer> replicationFactor(String topic) {
return futures.get(topic).thenApply(TopicMetadataAndConfig::replicationFactor);
}
public static class TopicMetadataAndConfig {
private final ApiException exception;
private final int numPartitions;
private final int replicationFactor;
private final Config config;
TopicMetadataAndConfig(int numPartitions, int replicationFactor, Config config) {
this.exception = null;
this.numPartitions = numPartitions;
this.replicationFactor = replicationFactor;
this.config = config;
}
TopicMetadataAndConfig(ApiException exception) {
this.exception = exception;
this.numPartitions = UNKNOWN;
this.replicationFactor = UNKNOWN;
this.config = null;
}
public int numPartitions() {
ensureSuccess();
return numPartitions;
}
public int replicationFactor() {
ensureSuccess();
return replicationFactor;
}
public Config config() {
ensureSuccess();
return config;
}
private void ensureSuccess() {
if (exception != null)
throw exception;
}
}
}

View File

@@ -0,0 +1,43 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for the {@link Admin#deleteAcls(Collection)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteAclsOptions extends AbstractOptions<DeleteAclsOptions> {
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public DeleteAclsOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
}

View File

@@ -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.admin;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.errors.ApiException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
/**
* The result of the {@link Admin#deleteAcls(Collection)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteAclsResult {
/**
* A class containing either the deleted ACL binding or an exception if the delete failed.
*/
public static class FilterResult {
private final AclBinding binding;
private final ApiException exception;
FilterResult(AclBinding binding, ApiException exception) {
this.binding = binding;
this.exception = exception;
}
/**
* Return the deleted ACL binding or null if there was an error.
*/
public AclBinding binding() {
return binding;
}
/**
* Return an exception if the ACL delete was not successful or null if it was.
*/
public ApiException exception() {
return exception;
}
}
/**
* A class containing the results of the delete ACLs operation.
*/
public static class FilterResults {
private final List<FilterResult> values;
FilterResults(List<FilterResult> values) {
this.values = values;
}
/**
* Return a list of delete ACLs results for a given filter.
*/
public List<FilterResult> values() {
return values;
}
}
private final Map<AclBindingFilter, KafkaFuture<FilterResults>> futures;
DeleteAclsResult(Map<AclBindingFilter, KafkaFuture<FilterResults>> futures) {
this.futures = futures;
}
/**
* Return a map from acl filters to futures which can be used to check the status of the deletions by each
* filter.
*/
public Map<AclBindingFilter, KafkaFuture<FilterResults>> values() {
return futures;
}
/**
* Return a future which succeeds only if all the ACLs deletions succeed, and which contains all the deleted ACLs.
* Note that it if the filters don't match any ACLs, this is not considered an error.
*/
public KafkaFuture<Collection<AclBinding>> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).thenApply(v -> getAclBindings(futures));
}
private List<AclBinding> getAclBindings(Map<AclBindingFilter, KafkaFuture<FilterResults>> futures) {
List<AclBinding> acls = new ArrayList<>();
for (KafkaFuture<FilterResults> value: futures.values()) {
FilterResults results;
try {
results = value.get();
} catch (Throwable e) {
// This should be unreachable, since the future returned by KafkaFuture#allOf should
// have failed if any Future failed.
throw new KafkaException("DeleteAclsResult#all: internal error", e);
}
for (FilterResult result : results.values()) {
if (result.exception() != null)
throw result.exception();
acls.add(result.binding());
}
}
return acls;
}
}

View File

@@ -0,0 +1,30 @@
/*
* 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.admin;
import java.util.Set;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for the {@link Admin#deleteConsumerGroupOffsets(String, Set)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteConsumerGroupOffsetsOptions extends AbstractOptions<DeleteConsumerGroupOffsetsOptions> {
}

View File

@@ -0,0 +1,97 @@
/*
* 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.admin;
import java.util.Set;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
import org.apache.kafka.common.internals.KafkaFutureImpl;
import org.apache.kafka.common.protocol.Errors;
/**
* The result of the {@link Admin#deleteConsumerGroupOffsets(String, Set)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteConsumerGroupOffsetsResult {
private final KafkaFuture<Map<TopicPartition, Errors>> future;
private final Set<TopicPartition> partitions;
DeleteConsumerGroupOffsetsResult(KafkaFuture<Map<TopicPartition, Errors>> future, Set<TopicPartition> partitions) {
this.future = future;
this.partitions = partitions;
}
/**
* Return a future which can be used to check the result for a given partition.
*/
public KafkaFuture<Void> partitionResult(final TopicPartition partition) {
if (!partitions.contains(partition)) {
throw new IllegalArgumentException("Partition " + partition + " was not included in the original request");
}
final KafkaFutureImpl<Void> result = new KafkaFutureImpl<>();
this.future.whenComplete((topicPartitions, throwable) -> {
if (throwable != null) {
result.completeExceptionally(throwable);
} else if (!maybeCompleteExceptionally(topicPartitions, partition, result)) {
result.complete(null);
}
});
return result;
}
/**
* Return a future which succeeds only if all the deletions succeed.
* If not, the first partition error shall be returned.
*/
public KafkaFuture<Void> all() {
final KafkaFutureImpl<Void> result = new KafkaFutureImpl<>();
this.future.whenComplete((topicPartitions, throwable) -> {
if (throwable != null) {
result.completeExceptionally(throwable);
} else {
for (TopicPartition partition : partitions) {
if (maybeCompleteExceptionally(topicPartitions, partition, result)) {
return;
}
}
result.complete(null);
}
});
return result;
}
private boolean maybeCompleteExceptionally(Map<TopicPartition, Errors> partitionLevelErrors,
TopicPartition partition,
KafkaFutureImpl<Void> result) {
Throwable exception = KafkaAdminClient.getSubLevelError(partitionLevelErrors, partition,
"Offset deletion result for partition \"" + partition + "\" was not included in the response");
if (exception != null) {
result.completeExceptionally(exception);
return true;
} else {
return false;
}
}
}

View File

@@ -0,0 +1,31 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for the {@link Admin#deleteConsumerGroups(Collection)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteConsumerGroupsOptions extends AbstractOptions<DeleteConsumerGroupsOptions> {
}

View File

@@ -0,0 +1,52 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.Map;
/**
* The result of the {@link Admin#deleteConsumerGroups(Collection)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteConsumerGroupsResult {
private final Map<String, KafkaFuture<Void>> futures;
DeleteConsumerGroupsResult(final Map<String, KafkaFuture<Void>> futures) {
this.futures = futures;
}
/**
* Return a map from group id to futures which can be used to check the status of
* individual deletions.
*/
public Map<String, KafkaFuture<Void>> deletedGroups() {
return futures;
}
/**
* Return a future which succeeds only if all the consumer group deletions succeed.
*/
public KafkaFuture<Void> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]));
}
}

View File

@@ -0,0 +1,32 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
/**
* Options for {@link Admin#deleteRecords(Map, DeleteRecordsOptions)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteRecordsOptions extends AbstractOptions<DeleteRecordsOptions> {
}

View File

@@ -0,0 +1,54 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
/**
* The result of the {@link Admin#deleteRecords(Map)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteRecordsResult {
private final Map<TopicPartition, KafkaFuture<DeletedRecords>> futures;
public DeleteRecordsResult(Map<TopicPartition, KafkaFuture<DeletedRecords>> futures) {
this.futures = futures;
}
/**
* Return a map from topic partition to futures which can be used to check the status of
* individual deletions.
*/
public Map<TopicPartition, KafkaFuture<DeletedRecords>> lowWatermarks() {
return futures;
}
/**
* Return a future which succeeds only if all the records deletions succeed.
*/
public KafkaFuture<Void> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]));
}
}

View File

@@ -0,0 +1,43 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for {@link Admin#deleteTopics(Collection)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteTopicsOptions extends AbstractOptions<DeleteTopicsOptions> {
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public DeleteTopicsOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
}

View File

@@ -0,0 +1,53 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.Map;
/**
* The result of the {@link Admin#deleteTopics(Collection)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DeleteTopicsResult {
final Map<String, KafkaFuture<Void>> futures;
DeleteTopicsResult(Map<String, KafkaFuture<Void>> futures) {
this.futures = futures;
}
/**
* Return a map from topic names to futures which can be used to check the status of
* individual deletions.
*/
public Map<String, KafkaFuture<Void>> values() {
return futures;
}
/**
* Return a future which succeeds only if all the topic deletions succeed.
*/
public KafkaFuture<Void> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]));
}
}

View File

@@ -0,0 +1,47 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Represents information about deleted records
*
* The API for this class is still evolving and we may break compatibility in minor releases, if necessary.
*/
@InterfaceStability.Evolving
public class DeletedRecords {
private final long lowWatermark;
/**
* Create an instance of this class with the provided parameters.
*
* @param lowWatermark "low watermark" for the topic partition on which the deletion was executed
*/
public DeletedRecords(long lowWatermark) {
this.lowWatermark = lowWatermark;
}
/**
* Return the "low watermark" for the topic partition on which the deletion was executed
*/
public long lowWatermark() {
return lowWatermark;
}
}

View File

@@ -0,0 +1,42 @@
/*
* 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.admin;
import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for {@link Admin#describeAcls(AclBindingFilter)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeAclsOptions extends AbstractOptions<DescribeAclsOptions> {
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public DescribeAclsOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
}

View File

@@ -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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* The result of the {@link KafkaAdminClient#describeAcls(AclBindingFilter)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeAclsResult {
private final KafkaFuture<Collection<AclBinding>> future;
DescribeAclsResult(KafkaFuture<Collection<AclBinding>> future) {
this.future = future;
}
/**
* Return a future containing the ACLs requested.
*/
public KafkaFuture<Collection<AclBinding>> values() {
return future;
}
}

View File

@@ -0,0 +1,55 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for {@link Admin#describeCluster()}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeClusterOptions extends AbstractOptions<DescribeClusterOptions> {
private boolean includeAuthorizedOperations;
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public DescribeClusterOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
public DescribeClusterOptions includeAuthorizedOperations(boolean includeAuthorizedOperations) {
this.includeAuthorizedOperations = includeAuthorizedOperations;
return this;
}
/**
* Specify if authorized operations should be included in the response. Note that some
* older brokers cannot not supply this information even if it is requested.
*/
public boolean includeAuthorizedOperations() {
return includeAuthorizedOperations;
}
}

View File

@@ -0,0 +1,80 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.Set;
/**
* The result of the {@link KafkaAdminClient#describeCluster()} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeClusterResult {
private final KafkaFuture<Collection<Node>> nodes;
private final KafkaFuture<Node> controller;
private final KafkaFuture<String> clusterId;
private final KafkaFuture<Set<AclOperation>> authorizedOperations;
DescribeClusterResult(KafkaFuture<Collection<Node>> nodes,
KafkaFuture<Node> controller,
KafkaFuture<String> clusterId,
KafkaFuture<Set<AclOperation>> authorizedOperations) {
this.nodes = nodes;
this.controller = controller;
this.clusterId = clusterId;
this.authorizedOperations = authorizedOperations;
}
/**
* Returns a future which yields a collection of nodes.
*/
public KafkaFuture<Collection<Node>> nodes() {
return nodes;
}
/**
* Returns a future which yields the current controller id.
* Note that this may yield null, if the controller ID is not yet known.
*/
public KafkaFuture<Node> controller() {
return controller;
}
/**
* Returns a future which yields the current cluster id. The future value will be non-null if the
* broker version is 0.10.1.0 or higher and null otherwise.
*/
public KafkaFuture<String> clusterId() {
return clusterId;
}
/**
* Returns a future which yields authorized operations. The future value will be non-null if the
* broker supplied this information, and null otherwise.
*/
public KafkaFuture<Set<AclOperation>> authorizedOperations() {
return authorizedOperations;
}
}

View File

@@ -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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for {@link Admin#describeConfigs(Collection)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeConfigsOptions extends AbstractOptions<DescribeConfigsOptions> {
private boolean includeSynonyms = false;
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public DescribeConfigsOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
/**
* Return true if synonym configs should be returned in the response.
*/
public boolean includeSynonyms() {
return includeSynonyms;
}
/**
* Set to true if synonym configs should be returned in the response.
*/
public DescribeConfigsOptions includeSynonyms(boolean includeSynonyms) {
this.includeSynonyms = includeSynonyms;
return this;
}
}

View File

@@ -0,0 +1,73 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.config.ConfigResource;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ExecutionException;
/**
* The result of the {@link KafkaAdminClient#describeConfigs(Collection)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeConfigsResult {
private final Map<ConfigResource, KafkaFuture<Config>> futures;
DescribeConfigsResult(Map<ConfigResource, KafkaFuture<Config>> futures) {
this.futures = futures;
}
/**
* Return a map from resources to futures which can be used to check the status of the configuration for each
* resource.
*/
public Map<ConfigResource, KafkaFuture<Config>> values() {
return futures;
}
/**
* Return a future which succeeds only if all the config descriptions succeed.
*/
public KafkaFuture<Map<ConfigResource, Config>> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).
thenApply(new KafkaFuture.BaseFunction<Void, Map<ConfigResource, Config>>() {
@Override
public Map<ConfigResource, Config> apply(Void v) {
Map<ConfigResource, Config> configs = new HashMap<>(futures.size());
for (Map.Entry<ConfigResource, KafkaFuture<Config>> entry : futures.entrySet()) {
try {
configs.put(entry.getKey(), entry.getValue().get());
} catch (InterruptedException | ExecutionException e) {
// This should be unreachable, because allOf ensured that all the futures
// completed successfully.
throw new RuntimeException(e);
}
}
return configs;
}
});
}
}

View File

@@ -0,0 +1,41 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for {@link Admin#describeConsumerGroups(Collection, DescribeConsumerGroupsOptions)}.
* <p>
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeConsumerGroupsOptions extends AbstractOptions<DescribeConsumerGroupsOptions> {
private boolean includeAuthorizedOperations;
public DescribeConsumerGroupsOptions includeAuthorizedOperations(boolean includeAuthorizedOperations) {
this.includeAuthorizedOperations = includeAuthorizedOperations;
return this;
}
public boolean includeAuthorizedOperations() {
return includeAuthorizedOperations;
}
}

View File

@@ -0,0 +1,72 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ExecutionException;
/**
* The result of the {@link KafkaAdminClient#describeConsumerGroups(Collection, DescribeConsumerGroupsOptions)}} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeConsumerGroupsResult {
private final Map<String, KafkaFuture<ConsumerGroupDescription>> futures;
public DescribeConsumerGroupsResult(final Map<String, KafkaFuture<ConsumerGroupDescription>> futures) {
this.futures = futures;
}
/**
* Return a map from group id to futures which yield group descriptions.
*/
public Map<String, KafkaFuture<ConsumerGroupDescription>> describedGroups() {
return futures;
}
/**
* Return a future which yields all ConsumerGroupDescription objects, if all the describes succeed.
*/
public KafkaFuture<Map<String, ConsumerGroupDescription>> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).thenApply(
new KafkaFuture.BaseFunction<Void, Map<String, ConsumerGroupDescription>>() {
@Override
public Map<String, ConsumerGroupDescription> apply(Void v) {
try {
Map<String, ConsumerGroupDescription> descriptions = new HashMap<>(futures.size());
for (Map.Entry<String, KafkaFuture<ConsumerGroupDescription>> entry : futures.entrySet()) {
descriptions.put(entry.getKey(), entry.getValue().get());
}
return descriptions;
} catch (InterruptedException | ExecutionException e) {
// This should be unreachable, since the KafkaFuture#allOf already ensured
// that all of the futures completed successfully.
throw new RuntimeException(e);
}
}
});
}
}

View File

@@ -0,0 +1,48 @@
/*
* 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.admin;
import java.util.List;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.security.auth.KafkaPrincipal;
/**
* Options for {@link Admin#describeDelegationToken(DescribeDelegationTokenOptions)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeDelegationTokenOptions extends AbstractOptions<DescribeDelegationTokenOptions> {
private List<KafkaPrincipal> owners;
/**
* if owners is null, all the user owned tokens and tokens where user have Describe permission
* will be returned.
* @param owners
* @return this instance
*/
public DescribeDelegationTokenOptions owners(List<KafkaPrincipal> owners) {
this.owners = owners;
return this;
}
public List<KafkaPrincipal> owners() {
return owners;
}
}

View File

@@ -0,0 +1,45 @@
/*
* 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.admin;
import java.util.List;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.security.token.delegation.DelegationToken;
/**
* The result of the {@link KafkaAdminClient#describeDelegationToken(DescribeDelegationTokenOptions)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeDelegationTokenResult {
private final KafkaFuture<List<DelegationToken>> delegationTokens;
DescribeDelegationTokenResult(KafkaFuture<List<DelegationToken>> delegationTokens) {
this.delegationTokens = delegationTokens;
}
/**
* Returns a future which yields list of delegation tokens
*/
public KafkaFuture<List<DelegationToken>> delegationTokens() {
return delegationTokens;
}
}

View File

@@ -0,0 +1,33 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for {@link Admin#describeLogDirs(Collection)}
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeLogDirsOptions extends AbstractOptions<DescribeLogDirsOptions> {
}

View File

@@ -0,0 +1,70 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.HashMap;
import java.util.Collection;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import org.apache.kafka.common.requests.DescribeLogDirsResponse.LogDirInfo;
/**
* The result of the {@link Admin#describeLogDirs(Collection)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeLogDirsResult {
private final Map<Integer, KafkaFuture<Map<String, LogDirInfo>>> futures;
DescribeLogDirsResult(Map<Integer, KafkaFuture<Map<String, LogDirInfo>>> futures) {
this.futures = futures;
}
/**
* Return a map from brokerId to future which can be used to check the information of partitions on each individual broker
*/
public Map<Integer, KafkaFuture<Map<String, LogDirInfo>>> values() {
return futures;
}
/**
* Return a future which succeeds only if all the brokers have responded without error
*/
public KafkaFuture<Map<Integer, Map<String, LogDirInfo>>> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).
thenApply(new KafkaFuture.BaseFunction<Void, Map<Integer, Map<String, LogDirInfo>>>() {
@Override
public Map<Integer, Map<String, LogDirInfo>> apply(Void v) {
Map<Integer, Map<String, LogDirInfo>> descriptions = new HashMap<>(futures.size());
for (Map.Entry<Integer, KafkaFuture<Map<String, LogDirInfo>>> entry : futures.entrySet()) {
try {
descriptions.put(entry.getKey(), entry.getValue().get());
} catch (InterruptedException | ExecutionException e) {
// This should be unreachable, because allOf ensured that all the futures completed successfully.
throw new RuntimeException(e);
}
}
return descriptions;
}
});
}
}

View File

@@ -0,0 +1,31 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for {@link Admin#describeReplicaLogDirs(Collection)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeReplicaLogDirsOptions extends AbstractOptions<DescribeReplicaLogDirsOptions> {
}

View File

@@ -0,0 +1,132 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartitionReplica;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.requests.DescribeLogDirsResponse;
import java.util.HashMap;
import java.util.Map;
import java.util.Collection;
import java.util.concurrent.ExecutionException;
/**
* The result of {@link Admin#describeReplicaLogDirs(Collection)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeReplicaLogDirsResult {
private final Map<TopicPartitionReplica, KafkaFuture<ReplicaLogDirInfo>> futures;
DescribeReplicaLogDirsResult(Map<TopicPartitionReplica, KafkaFuture<ReplicaLogDirInfo>> futures) {
this.futures = futures;
}
/**
* Return a map from replica to future which can be used to check the log directory information of individual replicas
*/
public Map<TopicPartitionReplica, KafkaFuture<ReplicaLogDirInfo>> values() {
return futures;
}
/**
* Return a future which succeeds if log directory information of all replicas are available
*/
public KafkaFuture<Map<TopicPartitionReplica, ReplicaLogDirInfo>> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).
thenApply(new KafkaFuture.BaseFunction<Void, Map<TopicPartitionReplica, ReplicaLogDirInfo>>() {
@Override
public Map<TopicPartitionReplica, ReplicaLogDirInfo> apply(Void v) {
Map<TopicPartitionReplica, ReplicaLogDirInfo> replicaLogDirInfos = new HashMap<>();
for (Map.Entry<TopicPartitionReplica, KafkaFuture<ReplicaLogDirInfo>> entry : futures.entrySet()) {
try {
replicaLogDirInfos.put(entry.getKey(), entry.getValue().get());
} catch (InterruptedException | ExecutionException e) {
// This should be unreachable, because allOf ensured that all the futures completed successfully.
throw new RuntimeException(e);
}
}
return replicaLogDirInfos;
}
});
}
static public class ReplicaLogDirInfo {
// The current log directory of the replica of this partition on the given broker.
// Null if no replica is not found for this partition on the given broker.
private final String currentReplicaLogDir;
// Defined as max(HW of partition - LEO of the replica, 0).
private final long currentReplicaOffsetLag;
// The future log directory of the replica of this partition on the given broker.
// Null if the replica of this partition is not being moved to another log directory on the given broker.
private final String futureReplicaLogDir;
// The LEO of the replica - LEO of the future log of this replica in the destination log directory.
// -1 if either there is not replica for this partition or the replica of this partition is not being moved to another log directory on the given broker.
private final long futureReplicaOffsetLag;
ReplicaLogDirInfo() {
this(null, DescribeLogDirsResponse.INVALID_OFFSET_LAG, null, DescribeLogDirsResponse.INVALID_OFFSET_LAG);
}
ReplicaLogDirInfo(String currentReplicaLogDir,
long currentReplicaOffsetLag,
String futureReplicaLogDir,
long futureReplicaOffsetLag) {
this.currentReplicaLogDir = currentReplicaLogDir;
this.currentReplicaOffsetLag = currentReplicaOffsetLag;
this.futureReplicaLogDir = futureReplicaLogDir;
this.futureReplicaOffsetLag = futureReplicaOffsetLag;
}
public String getCurrentReplicaLogDir() {
return currentReplicaLogDir;
}
public long getCurrentReplicaOffsetLag() {
return currentReplicaOffsetLag;
}
public String getFutureReplicaLogDir() {
return futureReplicaLogDir;
}
public long getFutureReplicaOffsetLag() {
return futureReplicaOffsetLag;
}
@Override
public String toString() {
StringBuilder builder = new StringBuilder();
if (futureReplicaLogDir != null) {
builder.append("(currentReplicaLogDir=")
.append(currentReplicaLogDir)
.append(", futureReplicaLogDir=")
.append(futureReplicaLogDir)
.append(", futureReplicaOffsetLag=")
.append(futureReplicaOffsetLag)
.append(")");
} else {
builder.append("ReplicaLogDirInfo(currentReplicaLogDir=").append(currentReplicaLogDir).append(")");
}
return builder.toString();
}
}
}

View File

@@ -0,0 +1,54 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for {@link Admin#describeTopics(Collection)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeTopicsOptions extends AbstractOptions<DescribeTopicsOptions> {
private boolean includeAuthorizedOperations;
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public DescribeTopicsOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
public DescribeTopicsOptions includeAuthorizedOperations(boolean includeAuthorizedOperations) {
this.includeAuthorizedOperations = includeAuthorizedOperations;
return this;
}
public boolean includeAuthorizedOperations() {
return includeAuthorizedOperations;
}
}

View File

@@ -0,0 +1,68 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ExecutionException;
/**
* The result of the {@link KafkaAdminClient#describeTopics(Collection)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class DescribeTopicsResult {
private final Map<String, KafkaFuture<TopicDescription>> futures;
protected DescribeTopicsResult(Map<String, KafkaFuture<TopicDescription>> futures) {
this.futures = futures;
}
/**
* Return a map from topic names to futures which can be used to check the status of
* individual topics.
*/
public Map<String, KafkaFuture<TopicDescription>> values() {
return futures;
}
/**
* Return a future which succeeds only if all the topic descriptions succeed.
*/
public KafkaFuture<Map<String, TopicDescription>> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).
thenApply(v -> {
Map<String, TopicDescription> descriptions = new HashMap<>(futures.size());
for (Map.Entry<String, KafkaFuture<TopicDescription>> entry : futures.entrySet()) {
try {
descriptions.put(entry.getKey(), entry.getValue().get());
} catch (InterruptedException | ExecutionException e) {
// This should be unreachable, because allOf ensured that all the futures
// completed successfully.
throw new RuntimeException(e);
}
}
return descriptions;
});
}
}

View File

@@ -0,0 +1,29 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for {@link Admin#electLeaders(ElectionType, Set, ElectLeadersOptions)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
final public class ElectLeadersOptions extends AbstractOptions<ElectLeadersOptions> {
}

View File

@@ -0,0 +1,76 @@
/*
* 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.admin;
import java.util.Map;
import java.util.Optional;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.internals.KafkaFutureImpl;
/**
* The result of {@link Admin#electLeaders(ElectionType, Set, ElectLeadersOptions)}
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
final public class ElectLeadersResult {
private final KafkaFutureImpl<Map<TopicPartition, Optional<Throwable>>> electionFuture;
ElectLeadersResult(KafkaFutureImpl<Map<TopicPartition, Optional<Throwable>>> electionFuture) {
this.electionFuture = electionFuture;
}
/**
* <p>Get a future for the topic partitions for which a leader election was attempted.
* If the election succeeded then the value for a topic partition will be the empty Optional.
* Otherwise the election failed and the Optional will be set with the error.</p>
*/
public KafkaFuture<Map<TopicPartition, Optional<Throwable>>> partitions() {
return electionFuture;
}
/**
* Return a future which succeeds if all the topic elections succeed.
*/
public KafkaFuture<Void> all() {
final KafkaFutureImpl<Void> result = new KafkaFutureImpl<>();
partitions().whenComplete(
new KafkaFuture.BiConsumer<Map<TopicPartition, Optional<Throwable>>, Throwable>() {
@Override
public void accept(Map<TopicPartition, Optional<Throwable>> topicPartitions, Throwable throwable) {
if (throwable != null) {
result.completeExceptionally(throwable);
} else {
for (Optional<Throwable> exception : topicPartitions.values()) {
if (exception.isPresent()) {
result.completeExceptionally(exception.get());
return;
}
}
result.complete(null);
}
}
});
return result;
}
}

View File

@@ -0,0 +1,33 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/**
* Options for {@link Admin#electPreferredLeaders(Collection, ElectPreferredLeadersOptions)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*
* @deprecated Since 2.4.0. Use {@link Admin#electLeaders(ElectionType, Set, ElectLeadersOptions)}.
*/
@InterfaceStability.Evolving
@Deprecated
public class ElectPreferredLeadersOptions extends AbstractOptions<ElectPreferredLeadersOptions> {
}

View File

@@ -0,0 +1,112 @@
/*
* 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.admin;
import java.util.Collection;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.internals.KafkaFutureImpl;
/**
* The result of {@link Admin#electPreferredLeaders(Collection, ElectPreferredLeadersOptions)}
*
* The API of this class is evolving, see {@link Admin} for details.
*
* @deprecated Since 2.4.0. Use {@link Admin#electLeaders(ElectionType, Set, ElectLeadersOptions)}.
*/
@InterfaceStability.Evolving
@Deprecated
public class ElectPreferredLeadersResult {
private final ElectLeadersResult electionResult;
ElectPreferredLeadersResult(ElectLeadersResult electionResult) {
this.electionResult = electionResult;
}
/**
* Get the result of the election for the given {@code partition}.
* If there was not an election triggered for the given {@code partition}, the
* returned future will complete with an error.
*/
public KafkaFuture<Void> partitionResult(final TopicPartition partition) {
final KafkaFutureImpl<Void> result = new KafkaFutureImpl<>();
electionResult.partitions().whenComplete(
new KafkaFuture.BiConsumer<Map<TopicPartition, Optional<Throwable>>, Throwable>() {
@Override
public void accept(Map<TopicPartition, Optional<Throwable>> topicPartitions, Throwable throwable) {
if (throwable != null) {
result.completeExceptionally(throwable);
} else if (!topicPartitions.containsKey(partition)) {
result.completeExceptionally(new UnknownTopicOrPartitionException(
"Preferred leader election for partition \"" + partition +
"\" was not attempted"));
} else {
Optional<Throwable> exception = topicPartitions.get(partition);
if (exception.isPresent()) {
result.completeExceptionally(exception.get());
} else {
result.complete(null);
}
}
}
});
return result;
}
/**
* <p>Get a future for the topic partitions for which a leader election
* was attempted. A partition will be present in this result if
* an election was attempted even if the election was not successful.</p>
*
* <p>This method is provided to discover the partitions attempted when
* {@link Admin#electPreferredLeaders(Collection)} is called
* with a null {@code partitions} argument.</p>
*/
public KafkaFuture<Set<TopicPartition>> partitions() {
final KafkaFutureImpl<Set<TopicPartition>> result = new KafkaFutureImpl<>();
electionResult.partitions().whenComplete(
new KafkaFuture.BiConsumer<Map<TopicPartition, Optional<Throwable>>, Throwable>() {
@Override
public void accept(Map<TopicPartition, Optional<Throwable>> topicPartitions, Throwable throwable) {
if (throwable != null) {
result.completeExceptionally(throwable);
} else {
result.complete(topicPartitions.keySet());
}
}
});
return result;
}
/**
* Return a future which succeeds if all the topic elections succeed.
*/
public KafkaFuture<Void> all() {
return electionResult.all();
}
}

View File

@@ -0,0 +1,39 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for {@link Admin#expireDelegationToken(byte[], ExpireDelegationTokenOptions)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ExpireDelegationTokenOptions extends AbstractOptions<ExpireDelegationTokenOptions> {
private long expiryTimePeriodMs = -1L;
public ExpireDelegationTokenOptions expiryTimePeriodMs(long expiryTimePeriodMs) {
this.expiryTimePeriodMs = expiryTimePeriodMs;
return this;
}
public long expiryTimePeriodMs() {
return expiryTimePeriodMs;
}
}

View File

@@ -0,0 +1,42 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* The result of the {@link KafkaAdminClient#expireDelegationToken(byte[], ExpireDelegationTokenOptions)} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ExpireDelegationTokenResult {
private final KafkaFuture<Long> expiryTimestamp;
ExpireDelegationTokenResult(KafkaFuture<Long> expiryTimestamp) {
this.expiryTimestamp = expiryTimestamp;
}
/**
* Returns a future which yields expiry timestamp
*/
public KafkaFuture<Long> expiryTimestamp() {
return expiryTimestamp;
}
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,53 @@
/*
* 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.admin;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.List;
/**
* Options for {@link Admin#listConsumerGroupOffsets(String)}.
* <p>
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ListConsumerGroupOffsetsOptions extends AbstractOptions<ListConsumerGroupOffsetsOptions> {
private List<TopicPartition> topicPartitions = null;
/**
* Set the topic partitions to list as part of the result.
* {@code null} includes all topic partitions.
*
* @param topicPartitions List of topic partitions to include
* @return This ListGroupOffsetsOptions
*/
public ListConsumerGroupOffsetsOptions topicPartitions(List<TopicPartition> topicPartitions) {
this.topicPartitions = topicPartitions;
return this;
}
/**
* Returns a list of topic partitions to add as part of the result.
*/
public List<TopicPartition> topicPartitions() {
return topicPartitions;
}
}

View File

@@ -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.admin;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Map;
/**
* The result of the {@link Admin#listConsumerGroupOffsets(String)} call.
* <p>
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ListConsumerGroupOffsetsResult {
final KafkaFuture<Map<TopicPartition, OffsetAndMetadata>> future;
ListConsumerGroupOffsetsResult(KafkaFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
this.future = future;
}
/**
* Return a future which yields a map of topic partitions to OffsetAndMetadata objects.
* If the group does not have a committed offset for this partition, the corresponding value in the returned map will be null.
*/
public KafkaFuture<Map<TopicPartition, OffsetAndMetadata>> partitionsToOffsetAndMetadata() {
return future;
}
}

View File

@@ -0,0 +1,29 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for {@link Admin#listConsumerGroups()}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ListConsumerGroupsOptions extends AbstractOptions<ListConsumerGroupsOptions> {
}

View File

@@ -0,0 +1,102 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.internals.KafkaFutureImpl;
import java.util.ArrayList;
import java.util.Collection;
/**
* The result of the {@link Admin#listConsumerGroups()} call.
* <p>
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ListConsumerGroupsResult {
private final KafkaFutureImpl<Collection<ConsumerGroupListing>> all;
private final KafkaFutureImpl<Collection<ConsumerGroupListing>> valid;
private final KafkaFutureImpl<Collection<Throwable>> errors;
ListConsumerGroupsResult(KafkaFutureImpl<Collection<Object>> future) {
this.all = new KafkaFutureImpl<>();
this.valid = new KafkaFutureImpl<>();
this.errors = new KafkaFutureImpl<>();
future.thenApply(new KafkaFuture.BaseFunction<Collection<Object>, Void>() {
@Override
public Void apply(Collection<Object> results) {
ArrayList<Throwable> curErrors = new ArrayList<>();
ArrayList<ConsumerGroupListing> curValid = new ArrayList<>();
for (Object resultObject : results) {
if (resultObject instanceof Throwable) {
curErrors.add((Throwable) resultObject);
} else {
curValid.add((ConsumerGroupListing) resultObject);
}
}
if (!curErrors.isEmpty()) {
all.completeExceptionally(curErrors.get(0));
} else {
all.complete(curValid);
}
valid.complete(curValid);
errors.complete(curErrors);
return null;
}
});
}
/**
* Returns a future that yields either an exception, or the full set of consumer group
* listings.
*
* In the event of a failure, the future yields nothing but the first exception which
* occurred.
*/
public KafkaFuture<Collection<ConsumerGroupListing>> all() {
return all;
}
/**
* Returns a future which yields just the valid listings.
*
* This future never fails with an error, no matter what happens. Errors are completely
* ignored. If nothing can be fetched, an empty collection is yielded.
* If there is an error, but some results can be returned, this future will yield
* those partial results. When using this future, it is a good idea to also check
* the errors future so that errors can be displayed and handled.
*/
public KafkaFuture<Collection<ConsumerGroupListing>> valid() {
return valid;
}
/**
* Returns a future which yields just the errors which occurred.
*
* If this future yields a non-empty collection, it is very likely that elements are
* missing from the valid() set.
*
* This future itself never fails with an error. In the event of an error, this future
* will successfully yield a collection containing at least one exception.
*/
public KafkaFuture<Collection<Throwable>> errors() {
return errors;
}
}

View File

@@ -0,0 +1,43 @@
/*
* 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.admin;
import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for {@link AdminClient#listOffsets(Map)}.
*
* The API of this class is evolving, see {@link AdminClient} for details.
*/
@InterfaceStability.Evolving
public class ListOffsetsOptions extends AbstractOptions<ListOffsetsOptions> {
private final IsolationLevel isolationLevel;
public ListOffsetsOptions() {
this(IsolationLevel.READ_UNCOMMITTED);
}
public ListOffsetsOptions(IsolationLevel isolationLevel) {
this.isolationLevel = isolationLevel;
}
public IsolationLevel isolationLevel() {
return isolationLevel;
}
}

View File

@@ -0,0 +1,107 @@
/*
* 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.admin;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* The result of the {@link AdminClient#listOffsets(Map)} call.
*
* The API of this class is evolving, see {@link AdminClient} for details.
*/
@InterfaceStability.Evolving
public class ListOffsetsResult {
private final Map<TopicPartition, KafkaFuture<ListOffsetsResultInfo>> futures;
ListOffsetsResult(Map<TopicPartition, KafkaFuture<ListOffsetsResultInfo>> futures) {
this.futures = futures;
}
/**
* Return a future which can be used to check the result for a given partition.
*/
public KafkaFuture<ListOffsetsResultInfo> partitionResult(final TopicPartition partition) {
KafkaFuture<ListOffsetsResultInfo> future = futures.get(partition);
if (future == null) {
throw new IllegalArgumentException(
"List Offsets for partition \"" + partition + "\" was not attempted");
}
return future;
}
/**
* Return a future which succeeds only if offsets for all specified partitions have been successfully
* retrieved.
*/
public KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>> all() {
return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]))
.thenApply(new KafkaFuture.BaseFunction<Void, Map<TopicPartition, ListOffsetsResultInfo>>() {
@Override
public Map<TopicPartition, ListOffsetsResultInfo> apply(Void v) {
Map<TopicPartition, ListOffsetsResultInfo> offsets = new HashMap<>(futures.size());
for (Map.Entry<TopicPartition, KafkaFuture<ListOffsetsResultInfo>> entry : futures.entrySet()) {
try {
offsets.put(entry.getKey(), entry.getValue().get());
} catch (InterruptedException | ExecutionException e) {
// This should be unreachable, because allOf ensured that all the futures completed successfully.
throw new RuntimeException(e);
}
}
return offsets;
}
});
}
public static class ListOffsetsResultInfo {
private final long offset;
private final long timestamp;
private final Optional<Integer> leaderEpoch;
ListOffsetsResultInfo(long offset, long timestamp, Optional<Integer> leaderEpoch) {
this.offset = offset;
this.timestamp = timestamp;
this.leaderEpoch = leaderEpoch;
}
public long offset() {
return offset;
}
public long timestamp() {
return timestamp;
}
public Optional<Integer> leaderEpoch() {
return leaderEpoch;
}
@Override
public String toString() {
return "ListOffsetsResultInfo(offset=" + offset + ", timestamp=" + timestamp + ", leaderEpoch="
+ leaderEpoch + ")";
}
}
}

View File

@@ -0,0 +1,29 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for {@link AdminClient#listPartitionReassignments(ListPartitionReassignmentsOptions)}
*
* The API of this class is evolving. See {@link AdminClient} for details.
*/
@InterfaceStability.Evolving
public class ListPartitionReassignmentsOptions extends AbstractOptions<ListPartitionReassignmentsOptions> {
}

View File

@@ -0,0 +1,43 @@
/*
* 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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.TopicPartition;
import java.util.Map;
/**
* The result of {@link AdminClient#listPartitionReassignments(ListPartitionReassignmentsOptions)}.
*
* The API of this class is evolving. See {@link AdminClient} for details.
*/
public class ListPartitionReassignmentsResult {
private final KafkaFuture<Map<TopicPartition, PartitionReassignment>> future;
ListPartitionReassignmentsResult(KafkaFuture<Map<TopicPartition, PartitionReassignment>> reassignments) {
this.future = reassignments;
}
/**
* Return a future which yields a map containing each partition's reassignments
*/
public KafkaFuture<Map<TopicPartition, PartitionReassignment>> reassignments() {
return future;
}
}

View File

@@ -0,0 +1,61 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* Options for {@link Admin#listTopics()}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ListTopicsOptions extends AbstractOptions<ListTopicsOptions> {
private boolean listInternal = false;
/**
* Set the timeout in milliseconds for this operation or {@code null} if the default api timeout for the
* AdminClient should be used.
*
*/
// This method is retained to keep binary compatibility with 0.11
public ListTopicsOptions timeoutMs(Integer timeoutMs) {
this.timeoutMs = timeoutMs;
return this;
}
/**
* Set whether we should list internal topics.
*
* @param listInternal Whether we should list internal topics. null means to use
* the default.
* @return This ListTopicsOptions object.
*/
public ListTopicsOptions listInternal(boolean listInternal) {
this.listInternal = listInternal;
return this;
}
/**
* Return true if we should list internal topics.
*/
public boolean shouldListInternal() {
return listInternal;
}
}

View File

@@ -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.admin;
import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.Map;
import java.util.Set;
/**
* The result of the {@link Admin#listTopics()} call.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class ListTopicsResult {
final KafkaFuture<Map<String, TopicListing>> future;
ListTopicsResult(KafkaFuture<Map<String, TopicListing>> future) {
this.future = future;
}
/**
* Return a future which yields a map of topic names to TopicListing objects.
*/
public KafkaFuture<Map<String, TopicListing>> namesToListings() {
return future;
}
/**
* Return a future which yields a collection of TopicListing objects.
*/
public KafkaFuture<Collection<TopicListing>> listings() {
return future.thenApply(namesToDescriptions -> namesToDescriptions.values());
}
/**
* Return a future which yields a collection of topic names.
*/
public KafkaFuture<Set<String>> names() {
return future.thenApply(namesToListings -> namesToListings.keySet());
}
}

View File

@@ -0,0 +1,69 @@
/*
* 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.admin;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.Utils;
import java.util.Collections;
import java.util.HashSet;
import java.util.Objects;
import java.util.Set;
/**
* A description of the assignments of a specific group member.
*/
public class MemberAssignment {
private final Set<TopicPartition> topicPartitions;
/**
* Creates an instance with the specified parameters.
*
* @param topicPartitions List of topic partitions
*/
public MemberAssignment(Set<TopicPartition> topicPartitions) {
this.topicPartitions = topicPartitions == null ? Collections.<TopicPartition>emptySet() :
Collections.unmodifiableSet(new HashSet<>(topicPartitions));
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
MemberAssignment that = (MemberAssignment) o;
return Objects.equals(topicPartitions, that.topicPartitions);
}
@Override
public int hashCode() {
return topicPartitions != null ? topicPartitions.hashCode() : 0;
}
/**
* The topic partitions assigned to a group member.
*/
public Set<TopicPartition> topicPartitions() {
return topicPartitions;
}
@Override
public String toString() {
return "(topicPartitions=" + Utils.join(topicPartitions, ",") + ")";
}
}

View File

@@ -0,0 +1,113 @@
/*
* 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.admin;
import java.util.Collections;
import java.util.Objects;
import java.util.Optional;
/**
* A detailed description of a single group instance in the cluster.
*/
public class MemberDescription {
private final String memberId;
private final Optional<String> groupInstanceId;
private final String clientId;
private final String host;
private final MemberAssignment assignment;
public MemberDescription(String memberId,
Optional<String> groupInstanceId,
String clientId,
String host,
MemberAssignment assignment) {
this.memberId = memberId == null ? "" : memberId;
this.groupInstanceId = groupInstanceId;
this.clientId = clientId == null ? "" : clientId;
this.host = host == null ? "" : host;
this.assignment = assignment == null ?
new MemberAssignment(Collections.emptySet()) : assignment;
}
public MemberDescription(String memberId,
String clientId,
String host,
MemberAssignment assignment) {
this(memberId, Optional.empty(), clientId, host, assignment);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
MemberDescription that = (MemberDescription) o;
return memberId.equals(that.memberId) &&
groupInstanceId.equals(that.groupInstanceId) &&
clientId.equals(that.clientId) &&
host.equals(that.host) &&
assignment.equals(that.assignment);
}
@Override
public int hashCode() {
return Objects.hash(memberId, groupInstanceId, clientId, host, assignment);
}
/**
* The consumer id of the group member.
*/
public String consumerId() {
return memberId;
}
/**
* The instance id of the group member.
*/
public Optional<String> groupInstanceId() {
return groupInstanceId;
}
/**
* The client id of the group member.
*/
public String clientId() {
return clientId;
}
/**
* The host where the group member is running.
*/
public String host() {
return host;
}
/**
* The assignment of the group member.
*/
public MemberAssignment assignment() {
return assignment;
}
@Override
public String toString() {
return "(memberId=" + memberId +
", groupInstanceId=" + groupInstanceId.orElse("null") +
", clientId=" + clientId +
", host=" + host +
", assignment=" + assignment + ")";
}
}

View File

@@ -0,0 +1,58 @@
/*
* 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.admin;
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity;
import org.apache.kafka.common.requests.JoinGroupRequest;
import java.util.Objects;
/**
* A struct containing information about the member to be removed.
*/
public class MemberToRemove {
private final String groupInstanceId;
public MemberToRemove(String groupInstanceId) {
this.groupInstanceId = groupInstanceId;
}
@Override
public boolean equals(Object o) {
if (o instanceof MemberToRemove) {
MemberToRemove otherMember = (MemberToRemove) o;
return this.groupInstanceId.equals(otherMember.groupInstanceId);
} else {
return false;
}
}
@Override
public int hashCode() {
return Objects.hash(groupInstanceId);
}
MemberIdentity toMemberIdentity() {
return new MemberIdentity()
.setGroupInstanceId(groupInstanceId)
.setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID);
}
public String groupInstanceId() {
return groupInstanceId;
}
}

View File

@@ -0,0 +1,43 @@
/*
* 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.admin;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
/**
* A new partition reassignment, which can be applied via {@link AdminClient#alterPartitionReassignments(Map, AlterPartitionReassignmentsOptions)}.
*/
public class NewPartitionReassignment {
private final List<Integer> targetReplicas;
/**
* @throws IllegalArgumentException if no replicas are supplied
*/
public NewPartitionReassignment(List<Integer> targetReplicas) {
if (targetReplicas == null || targetReplicas.size() == 0)
throw new IllegalArgumentException("Cannot create a new partition reassignment without any replicas");
this.targetReplicas = Collections.unmodifiableList(new ArrayList<>(targetReplicas));
}
public List<Integer> targetReplicas() {
return targetReplicas;
}
}

View File

@@ -0,0 +1,99 @@
/*
* 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.admin;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.List;
import java.util.Map;
/**
* Describes new partitions for a particular topic in a call to {@link Admin#createPartitions(Map)}.
*
* The API of this class is evolving, see {@link Admin} for details.
*/
@InterfaceStability.Evolving
public class NewPartitions {
private int totalCount;
private List<List<Integer>> newAssignments;
private NewPartitions(int totalCount, List<List<Integer>> newAssignments) {
this.totalCount = totalCount;
this.newAssignments = newAssignments;
}
/**
* Increase the partition count for a topic to the given {@code totalCount}.
* The assignment of new replicas to brokers will be decided by the broker.
*
* @param totalCount The total number of partitions after the operation succeeds.
*/
public static NewPartitions increaseTo(int totalCount) {
return new NewPartitions(totalCount, null);
}
/**
* <p>Increase the partition count for a topic to the given {@code totalCount}
* assigning the new partitions according to the given {@code newAssignments}.
* The length of the given {@code newAssignments} should equal {@code totalCount - oldCount}, since
* the assignment of existing partitions are not changed.
* Each inner list of {@code newAssignments} should have a length equal to
* the topic's replication factor.
* The first broker id in each inner list is the "preferred replica".</p>
*
* <p>For example, suppose a topic currently has a replication factor of 2, and
* has 3 partitions. The number of partitions can be increased to 6 using a
* {@code NewPartition} constructed like this:</p>
*
* <pre><code>
* NewPartitions.increaseTo(6, asList(asList(1, 2),
* asList(2, 3),
* asList(3, 1)))
* </code></pre>
* <p>In this example partition 3's preferred leader will be broker 1, partition 4's preferred leader will be
* broker 2 and partition 5's preferred leader will be broker 3.</p>
*
* @param totalCount The total number of partitions after the operation succeeds.
* @param newAssignments The replica assignments for the new partitions.
*/
public static NewPartitions increaseTo(int totalCount, List<List<Integer>> newAssignments) {
return new NewPartitions(totalCount, newAssignments);
}
/**
* The total number of partitions after the operation succeeds.
*/
public int totalCount() {
return totalCount;
}
/**
* The replica assignments for the new partitions, or null if the assignment will be done by the controller.
*/
public List<List<Integer>> assignments() {
return newAssignments;
}
@Override
public String toString() {
return "(totalCount=" + totalCount() + ", newAssignments=" + assignments() + ")";
}
}

View File

@@ -0,0 +1,178 @@
/*
* 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.admin;
import java.util.Optional;
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
import org.apache.kafka.common.message.CreateTopicsRequestData.CreateableTopicConfig;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Map.Entry;
/**
* A new topic to be created via {@link Admin#createTopics(Collection)}.
*/
public class NewTopic {
private static final int NO_PARTITIONS = -1;
private static final short NO_REPLICATION_FACTOR = -1;
private final String name;
private final Optional<Integer> numPartitions;
private final Optional<Short> replicationFactor;
private final Map<Integer, List<Integer>> replicasAssignments;
private Map<String, String> configs = null;
/**
* A new topic with the specified replication factor and number of partitions.
*/
public NewTopic(String name, int numPartitions, short replicationFactor) {
this(name, Optional.of(numPartitions), Optional.of(replicationFactor));
}
/**
* A new topic that optionally defaults {@code numPartitions} and {@code replicationFactor} to
* the broker configurations for {@code num.partitions} and {@code default.replication.factor}
* respectively.
*/
public NewTopic(String name, Optional<Integer> numPartitions, Optional<Short> replicationFactor) {
this.name = name;
this.numPartitions = numPartitions;
this.replicationFactor = replicationFactor;
this.replicasAssignments = null;
}
/**
* A new topic with the specified replica assignment configuration.
*
* @param name the topic name.
* @param replicasAssignments a map from partition id to replica ids (i.e. broker ids). Although not enforced, it is
* generally a good idea for all partitions to have the same number of replicas.
*/
public NewTopic(String name, Map<Integer, List<Integer>> replicasAssignments) {
this.name = name;
this.numPartitions = Optional.empty();
this.replicationFactor = Optional.empty();
this.replicasAssignments = Collections.unmodifiableMap(replicasAssignments);
}
/**
* The name of the topic to be created.
*/
public String name() {
return name;
}
/**
* The number of partitions for the new topic or -1 if a replica assignment has been specified.
*/
public int numPartitions() {
return numPartitions.orElse(NO_PARTITIONS);
}
/**
* The replication factor for the new topic or -1 if a replica assignment has been specified.
*/
public short replicationFactor() {
return replicationFactor.orElse(NO_REPLICATION_FACTOR);
}
/**
* A map from partition id to replica ids (i.e. broker ids) or null if the number of partitions and replication
* factor have been specified instead.
*/
public Map<Integer, List<Integer>> replicasAssignments() {
return replicasAssignments;
}
/**
* Set the configuration to use on the new topic.
*
* @param configs The configuration map.
* @return This NewTopic object.
*/
public NewTopic configs(Map<String, String> configs) {
this.configs = configs;
return this;
}
/**
* The configuration for the new topic or null if no configs ever specified.
*/
public Map<String, String> configs() {
return configs;
}
CreatableTopic convertToCreatableTopic() {
CreatableTopic creatableTopic = new CreatableTopic().
setName(name).
setNumPartitions(numPartitions.orElse(NO_PARTITIONS)).
setReplicationFactor(replicationFactor.orElse(NO_REPLICATION_FACTOR));
if (replicasAssignments != null) {
for (Entry<Integer, List<Integer>> entry : replicasAssignments.entrySet()) {
creatableTopic.assignments().add(
new CreatableReplicaAssignment().
setPartitionIndex(entry.getKey()).
setBrokerIds(entry.getValue()));
}
}
if (configs != null) {
for (Entry<String, String> entry : configs.entrySet()) {
creatableTopic.configs().add(
new CreateableTopicConfig().
setName(entry.getKey()).
setValue(entry.getValue()));
}
}
return creatableTopic;
}
@Override
public String toString() {
StringBuilder bld = new StringBuilder();
bld.append("(name=").append(name).
append(", numPartitions=").append(numPartitions.map(String::valueOf).orElse("default")).
append(", replicationFactor=").append(replicationFactor.map(String::valueOf).orElse("default")).
append(", replicasAssignments=").append(replicasAssignments).
append(", configs=").append(configs).
append(")");
return bld.toString();
}
@Override
public boolean equals(final Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
final NewTopic that = (NewTopic) o;
return Objects.equals(name, that.name) &&
Objects.equals(numPartitions, that.numPartitions) &&
Objects.equals(replicationFactor, that.replicationFactor) &&
Objects.equals(replicasAssignments, that.replicasAssignments) &&
Objects.equals(configs, that.configs);
}
@Override
public int hashCode() {
return Objects.hash(name, numPartitions, replicationFactor, replicasAssignments, configs);
}
}

View File

@@ -0,0 +1,62 @@
/*
* 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.admin;
/**
* This class allows to specify the desired offsets when using {@link KafkaAdminClient#listOffsets(Map, ListOffsetsOptions)}
*/
public class OffsetSpec {
static class EarliestSpec extends OffsetSpec { }
static class LatestSpec extends OffsetSpec { }
static class TimestampSpec extends OffsetSpec {
private final long timestamp;
TimestampSpec(long timestamp) {
this.timestamp = timestamp;
}
long timestamp() {
return timestamp;
}
}
/**
* Used to retrieve the latest offset of a partition
*/
public static OffsetSpec latest() {
return new LatestSpec();
}
/**
* Used to retrieve the earliest offset of a partition
*/
public static OffsetSpec earliest() {
return new EarliestSpec();
}
/**
* Used to retrieve the the earliest offset whose timestamp is greater than
* or equal to the given timestamp in the corresponding partition
* @param timestamp in milliseconds
*/
public static OffsetSpec forTimestamp(long timestamp) {
return new TimestampSpec(timestamp);
}
}

View File

@@ -0,0 +1,69 @@
/*
* 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.admin;
import java.util.Collections;
import java.util.List;
/**
* A partition reassignment, which has been listed via {@link AdminClient#listPartitionReassignments()}.
*/
public class PartitionReassignment {
private final List<Integer> replicas;
private final List<Integer> addingReplicas;
private final List<Integer> removingReplicas;
public PartitionReassignment(List<Integer> replicas, List<Integer> addingReplicas, List<Integer> removingReplicas) {
this.replicas = Collections.unmodifiableList(replicas);
this.addingReplicas = Collections.unmodifiableList(addingReplicas);
this.removingReplicas = Collections.unmodifiableList(removingReplicas);
}
/**
* The brokers which this partition currently resides on.
*/
public List<Integer> replicas() {
return replicas;
}
/**
* The brokers that we are adding this partition to as part of a reassignment.
* A subset of replicas.
*/
public List<Integer> addingReplicas() {
return addingReplicas;
}
/**
* The brokers that we are removing this partition from as part of a reassignment.
* A subset of replicas.
*/
public List<Integer> removingReplicas() {
return removingReplicas;
}
@Override
public String toString() {
return "PartitionReassignment(" +
"replicas=" + replicas +
", addingReplicas=" + addingReplicas +
", removingReplicas=" + removingReplicas +
')';
}
}

Some files were not shown because too many files have changed in this diff Show More