Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Handle Kafka API_VERSIONS request #547

Open
wants to merge 4 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ApiMessage;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Message;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.AbstractResponse;
Expand Down Expand Up @@ -132,14 +133,17 @@ public ByteBuf serialize() {
private ByteBuf serialize(AbstractResponse response) {
final ObjectSerializationCache cache = new ObjectSerializationCache();
ResponseHeader responseHeader = header.toResponseHeader();
int headerSize = responseHeader.size();
short headerVersion = responseHeader.headerVersion();
short apiVersion = request.version();
Message headerData = responseHeader.data();
int headerSize = headerData.size(cache, headerVersion);
ApiMessage apiMessage = response.data();
int messageSize = apiMessage.size(cache, apiVersion);
final ByteBuf buffer = ctx.alloc().buffer(headerSize + messageSize);
buffer.writerIndex(headerSize + messageSize);
final ByteBuffer nioBuffer = buffer.nioBuffer();
final ByteBufferAccessor writable = new ByteBufferAccessor(nioBuffer);
responseHeader.data().write(writable, cache, apiVersion);
headerData.write(writable, cache, headerVersion);
apiMessage.write(writable, cache, apiVersion);
return buffer;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,11 @@
import com.alibaba.fluss.shaded.netty4.io.netty.channel.ChannelHandlerContext;

import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.message.ApiVersionsResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.requests.ApiVersionsResponse;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -52,7 +57,34 @@ protected void handleInactive(KafkaRequest request) {
}

@Override
protected void handleApiVersionsRequest(KafkaRequest request) {}
protected void handleApiVersionsRequest(KafkaRequest request) {
short apiVersion = request.apiVersion();
if (!ApiKeys.API_VERSIONS.isVersionSupported(apiVersion)) {
request.fail(Errors.UNSUPPORTED_VERSION.exception());
return;
}
ApiVersionsResponseData data = new ApiVersionsResponseData();
for (ApiKeys apiKey : ApiKeys.values()) {
if (apiKey.minRequiredInterBrokerMagic <= RecordBatch.CURRENT_MAGIC_VALUE) {
ApiVersionsResponseData.ApiVersion apiVersionData =
new ApiVersionsResponseData.ApiVersion()
.setApiKey(apiKey.id)
.setMinVersion(apiKey.oldestVersion())
.setMaxVersion(apiKey.latestVersion());
if (apiKey.equals(ApiKeys.METADATA)) {
// Not support TopicId
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fluss uses tableId, a globally unique identifier of type long, to represent a table. Would it be feasible to use tableId as the topicId for Kafka in the future? For instance, by converting the long value into the leastSignificantBits and using a constant value for the mostSignificantBits.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Support topic id will make it more complex, we have to handle the case of topic ids is not empty and topic names not empty.
But we can support it in the future, at the current stage, make it as easier as we can.

short v = apiKey.latestVersion() > 11 ? 11 : apiKey.latestVersion();
apiVersionData.setMaxVersion(v);
} else if (apiKey.equals(ApiKeys.FETCH)) {
// Not support TopicId
short v = apiKey.latestVersion() > 12 ? 12 : apiKey.latestVersion();
apiVersionData.setMaxVersion(v);
}
data.apiKeys().add(apiVersionData);
}
}
request.complete(new ApiVersionsResponse(data));
}

@Override
protected void handleProducerRequest(KafkaRequest request) {}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
/*
* Copyright (c) 2024 Alibaba Group Holding Ltd.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package com.alibaba.fluss.kafka;

import com.alibaba.fluss.shaded.netty4.io.netty.buffer.ByteBuf;
import com.alibaba.fluss.shaded.netty4.io.netty.buffer.ByteBufAllocator;
import com.alibaba.fluss.shaded.netty4.io.netty.channel.ChannelHandlerContext;

import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.AbstractResponse;
import org.apache.kafka.common.requests.ApiVersionsRequest;
import org.apache.kafka.common.requests.ApiVersionsResponse;
import org.apache.kafka.common.requests.RequestHeader;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.mockito.Mockito;

import java.util.Map;
import java.util.concurrent.CompletableFuture;

public class KafkaRequestHandlerTest {

@Test
public void testKafkaApiVersionsNotSupported() {
KafkaRequestHandler handler = new KafkaRequestHandler();
short latestVersion = ApiKeys.API_VERSIONS.latestVersion();
ApiVersionsRequest apiVersionsRequest =
new ApiVersionsRequest.Builder().build(latestVersion);
ChannelHandlerContext ctx = Mockito.mock(ChannelHandlerContext.class);
Mockito.doReturn(ByteBufAllocator.DEFAULT).when(ctx).alloc();
KafkaRequest request =
new KafkaRequest(
ApiKeys.API_VERSIONS,
(short) (latestVersion + 1), // unsupported version
new RequestHeader(ApiKeys.API_VERSIONS, latestVersion, "client-id", 0),
apiVersionsRequest,
ByteBufAllocator.DEFAULT.buffer(),
ctx,
new CompletableFuture<>());
handler.handleApiVersionsRequest(request);

ByteBuf responseBuffer = request.serialize();
ApiVersionsResponse response =
(ApiVersionsResponse)
AbstractResponse.parseResponse(
responseBuffer.nioBuffer(), request.header());
Map<Errors, Integer> errorCounts = response.errorCounts();
Assertions.assertEquals(1, errorCounts.size());
Assertions.assertEquals(1, errorCounts.get(Errors.UNSUPPORTED_VERSION));
}

@Test
public void testKafkaApiVersionsRequest() {
KafkaRequestHandler handler = new KafkaRequestHandler();
short latestVersion = ApiKeys.API_VERSIONS.latestVersion();
ApiVersionsRequest apiVersionsRequest =
new ApiVersionsRequest.Builder().build(latestVersion);
ChannelHandlerContext ctx = Mockito.mock(ChannelHandlerContext.class);
Mockito.doReturn(ByteBufAllocator.DEFAULT).when(ctx).alloc();
KafkaRequest request =
new KafkaRequest(
ApiKeys.API_VERSIONS,
latestVersion,
new RequestHeader(ApiKeys.API_VERSIONS, latestVersion, "client-id", 0),
apiVersionsRequest,
ByteBufAllocator.DEFAULT.buffer(),
ctx,
new CompletableFuture<>());
handler.handleApiVersionsRequest(request);

ByteBuf responseBuffer = request.serialize();
ApiVersionsResponse response =
(ApiVersionsResponse)
AbstractResponse.parseResponse(
responseBuffer.nioBuffer(), request.header());
Map<Errors, Integer> errorCounts = response.errorCounts();
Assertions.assertEquals(1, errorCounts.size());
Assertions.assertEquals(1, errorCounts.get(Errors.NONE));
response.data()
.apiKeys()
.forEach(
apiVersion -> {
if (ApiKeys.METADATA.id == apiVersion.apiKey()) {
Assertions.assertTrue((short) 11 >= apiVersion.maxVersion());
} else if (ApiKeys.FETCH.id == apiVersion.apiKey()) {
Assertions.assertTrue((short) 12 >= apiVersion.maxVersion());
} else {
Assertions.assertEquals(
apiVersion.minVersion(), apiVersion.minVersion());
Assertions.assertEquals(
apiVersion.maxVersion(), apiVersion.maxVersion());
}
});
}
}
Loading