Skip to content

Commit

Permalink
KAFKA-12369; Implement ListTransactions API (#10206)
Browse files Browse the repository at this point in the history
This patch implements the `ListTransactions` API as documented in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.

Reviewers: Tom Bentley <tbentley@redhat.com>, Chia-Ping Tsai <chia7712@gmail.com>
  • Loading branch information
Jason Gustafson authored Mar 2, 2021
1 parent a848e0c commit 3708a7c
Show file tree
Hide file tree
Showing 20 changed files with 587 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,8 @@ public enum ApiKeys {
BROKER_REGISTRATION(ApiMessageType.BROKER_REGISTRATION, true, RecordBatch.MAGIC_VALUE_V0, false),
BROKER_HEARTBEAT(ApiMessageType.BROKER_HEARTBEAT, true, RecordBatch.MAGIC_VALUE_V0, false),
UNREGISTER_BROKER(ApiMessageType.UNREGISTER_BROKER, false, RecordBatch.MAGIC_VALUE_V0, true),
DESCRIBE_TRANSACTIONS(ApiMessageType.DESCRIBE_TRANSACTIONS);
DESCRIBE_TRANSACTIONS(ApiMessageType.DESCRIBE_TRANSACTIONS),
LIST_TRANSACTIONS(ApiMessageType.LIST_TRANSACTIONS);

private static final Map<ApiMessageType.ListenerType, EnumSet<ApiKeys>> APIS_BY_LISTENER =
new EnumMap<>(ApiMessageType.ListenerType.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -286,6 +286,8 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion,
return UnregisterBrokerRequest.parse(buffer, apiVersion);
case DESCRIBE_TRANSACTIONS:
return DescribeTransactionsRequest.parse(buffer, apiVersion);
case LIST_TRANSACTIONS:
return ListTransactionsRequest.parse(buffer, apiVersion);
default:
throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " +
"code should be updated to do so.", apiKey));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response
return UnregisterBrokerResponse.parse(responseBuffer, version);
case DESCRIBE_TRANSACTIONS:
return DescribeTransactionsResponse.parse(responseBuffer, version);
case LIST_TRANSACTIONS:
return ListTransactionsResponse.parse(responseBuffer, version);
default:
throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " +
"code should be updated to do so.", apiKey));
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.common.requests;

import org.apache.kafka.common.message.ListTransactionsRequestData;
import org.apache.kafka.common.message.ListTransactionsResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;

import java.nio.ByteBuffer;

public class ListTransactionsRequest extends AbstractRequest {
public static class Builder extends AbstractRequest.Builder<ListTransactionsRequest> {
public final ListTransactionsRequestData data;

public Builder(ListTransactionsRequestData data) {
super(ApiKeys.LIST_TRANSACTIONS);
this.data = data;
}

@Override
public ListTransactionsRequest build(short version) {
return new ListTransactionsRequest(data, version);
}

@Override
public String toString() {
return data.toString();
}
}

private final ListTransactionsRequestData data;

private ListTransactionsRequest(ListTransactionsRequestData data, short version) {
super(ApiKeys.LIST_TRANSACTIONS, version);
this.data = data;
}

public ListTransactionsRequestData data() {
return data;
}

@Override
public ListTransactionsResponse getErrorResponse(int throttleTimeMs, Throwable e) {
Errors error = Errors.forException(e);
ListTransactionsResponseData response = new ListTransactionsResponseData()
.setErrorCode(error.code())
.setThrottleTimeMs(throttleTimeMs);
return new ListTransactionsResponse(response);
}

public static ListTransactionsRequest parse(ByteBuffer buffer, short version) {
return new ListTransactionsRequest(new ListTransactionsRequestData(
new ByteBufferAccessor(buffer), version), version);
}

@Override
public String toString(boolean verbose) {
return data.toString();
}

}
Original file line number Diff line number Diff line change
@@ -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.common.requests;

import org.apache.kafka.common.message.ListTransactionsResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;

import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.Map;

public class ListTransactionsResponse extends AbstractResponse {
private final ListTransactionsResponseData data;

public ListTransactionsResponse(ListTransactionsResponseData data) {
super(ApiKeys.LIST_TRANSACTIONS);
this.data = data;
}

public ListTransactionsResponseData data() {
return data;
}

@Override
public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> errorCounts = new HashMap<>();
updateErrorCounts(errorCounts, Errors.forCode(data.errorCode()));
return errorCounts;
}

public static ListTransactionsResponse parse(ByteBuffer buffer, short version) {
return new ListTransactionsResponse(new ListTransactionsResponseData(
new ByteBufferAccessor(buffer), version));
}

@Override
public String toString() {
return data.toString();
}

@Override
public int throttleTimeMs() {
return data.throttleTimeMs();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
"validVersions": "0",
"flexibleVersions": "0+",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true,
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
"about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
{ "name": "TransactionStates", "type": "[]TransactionState", "versions": "0+", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+" },
Expand Down
Original file line number Diff line number Diff line change
@@ -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.

{
"apiKey": 66,
"type": "request",
"listeners": ["zkBroker", "broker"],
"name": "ListTransactionsRequest",
"validVersions": "0",
"flexibleVersions": "0+",
"fields": [
{ "name": "StateFilters", "type": "[]string", "versions": "0+",
"about": "The transaction states to filter by: if empty, all transactions are returned; if non-empty, then only transactions matching one of the filtered states will be returned"
},
{ "name": "ProducerIdFilters", "type": "[]int64", "versions": "0+",
"about": "The producerIds to filter by: if empty, all transactions will be returned; if non-empty, only transactions which match one of the filtered producerIds will be returned"
}
]
}
Original file line number Diff line number Diff line change
@@ -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.

{
"apiKey": 66,
"type": "response",
"name": "ListTransactionsResponse",
"validVersions": "0",
"flexibleVersions": "0+",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
"about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+" },
{ "name": "UnknownStateFilters", "type": "[]string", "versions": "0+",
"about": "Set of state filters provided in the request which were unknown to the transaction coordinator" },
{ "name": "TransactionStates", "type": "[]TransactionState", "versions": "0+", "fields": [
{ "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId" },
{ "name": "ProducerId", "type": "int64", "versions": "0+", "entityType": "producerId" },
{ "name": "TransactionState", "type": "string", "versions": "0+",
"about": "The current transaction state of the producer" }
]}
]
}
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,8 @@
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData;
import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData;
import org.apache.kafka.common.message.ListTransactionsRequestData;
import org.apache.kafka.common.message.ListTransactionsResponseData;
import org.apache.kafka.common.message.OffsetCommitRequestData;
import org.apache.kafka.common.message.OffsetCommitResponseData;
import org.apache.kafka.common.message.OffsetDeleteRequestData;
Expand Down Expand Up @@ -560,6 +562,15 @@ public void testDescribeTransactionsSerialization() {
}
}

@Test
public void testListTransactionsSerialization() {
for (short v : ApiKeys.LIST_TRANSACTIONS.allVersions()) {
checkRequest(createListTransactionsRequest(v), true);
checkErrorResponse(createListTransactionsRequest(v), unknownServerException, true);
checkResponse(createListTransactionsResponse(), v, true);
}
}

@Test
public void testDescribeClusterSerialization() {
for (short v : ApiKeys.DESCRIBE_CLUSTER.allVersions()) {
Expand Down Expand Up @@ -2806,4 +2817,27 @@ private DescribeTransactionsResponse createDescribeTransactionsResponse() {
return new DescribeTransactionsResponse(data);
}

private ListTransactionsRequest createListTransactionsRequest(short version) {
return new ListTransactionsRequest.Builder(new ListTransactionsRequestData()
.setStateFilters(singletonList("Ongoing"))
.setProducerIdFilters(asList(1L, 2L, 15L))
).build(version);
}

private ListTransactionsResponse createListTransactionsResponse() {
ListTransactionsResponseData response = new ListTransactionsResponseData();
response.setErrorCode(Errors.NONE.code());
response.setTransactionStates(Arrays.asList(
new ListTransactionsResponseData.TransactionState()
.setTransactionalId("foo")
.setProducerId(12345L)
.setTransactionState("Ongoing"),
new ListTransactionsResponseData.TransactionState()
.setTransactionalId("bar")
.setProducerId(98765L)
.setTransactionState("PrepareAbort")
));
return new ListTransactionsResponse(response);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import kafka.server.{KafkaConfig, MetadataCache, ReplicaManager}
import kafka.utils.{Logging, Scheduler}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.message.DescribeTransactionsResponseData
import org.apache.kafka.common.message.{DescribeTransactionsResponseData, ListTransactionsResponseData}
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.RecordBatch
Expand Down Expand Up @@ -251,11 +251,21 @@ class TransactionCoordinator(brokerId: Int,
s"This is illegal as we should never have transitioned to this state."
fatal(errorMsg)
throw new IllegalStateException(errorMsg)

}
}
}

def handleListTransactions(
filteredProducerIds: Set[Long],
filteredStates: Set[String]
): ListTransactionsResponseData = {
if (!isActive.get()) {
new ListTransactionsResponseData().setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code)
} else {
txnManager.listTransactionStates(filteredProducerIds, filteredStates)
}
}

def handleDescribeTransactions(
transactionalId: String
): DescribeTransactionsResponseData.TransactionState = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ object TransactionLog {
.setProducerId(txnMetadata.producerId)
.setProducerEpoch(txnMetadata.producerEpoch)
.setTransactionTimeoutMs(txnMetadata.txnTimeoutMs)
.setTransactionStatus(txnMetadata.txnState.byte)
.setTransactionStatus(txnMetadata.txnState.id)
.setTransactionLastUpdateTimestampMs(txnMetadata.txnLastUpdateTimestamp)
.setTransactionStartTimestampMs(txnMetadata.txnStartTimestamp)
.setTransactionPartitions(transactionPartitions))
Expand Down Expand Up @@ -128,7 +128,7 @@ object TransactionLog {
producerEpoch = value.producerEpoch,
lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH,
txnTimeoutMs = value.transactionTimeoutMs,
state = TransactionMetadata.byteToState(value.transactionStatus),
state = TransactionState.fromId(value.transactionStatus),
topicPartitions = mutable.Set.empty[TopicPartition],
txnStartTimestamp = value.transactionStartTimestampMs,
txnLastUpdateTimestamp = value.transactionLastUpdateTimestampMs)
Expand Down
Loading

0 comments on commit 3708a7c

Please sign in to comment.