KAFKA-7609; Add Protocol Generator for Kafka (#5893)

This patch adds a framework to automatically generate the request/response classes for Kafka's protocol. The code will be updated to use the generated classes in follow-up patches. Below is a brief summary of the included components:

**buildSrc/src**
The message generator code is here.  This code is automatically re-run by gradle when one of the schema files changes.  The entire directory is processed at once to minimize the number of times we have to start a new JVM.  We use Jackson to translate the JSON files into Java objects.

**clients/src/main/java/org/apache/kafka/common/protocol/Message.java**
This is the interface implemented by all automatically generated messages.

**clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java**
Some utility functions used by the generated message code.

**clients/src/main/java/org/apache/kafka/common/protocol/Readable.java, Writable.java, ByteBufferAccessor.java**
The generated message code uses these classes for writing to a buffer.

**clients/src/main/message/README.md**
This README file explains how the JSON schemas work.

**clients/src/main/message/\*.json**
The JSON files in this directory implement every supported version of every Kafka API.  The unit tests automatically validate that the generated schemas match the hand-written schemas in our code.  Additionally, there are some things like request and response headers that have schemas here.

**clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashSet.java**
I added an optimization here for empty sets.  This is useful here because I want all messages to start with empty sets by default prior to being loaded with data.  This is similar to the "empty list" optimizations in the `java.util.ArrayList` class.

Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>, Bob Barrett <bob.barrett@outlook.com>, Jason Gustafson <jason@confluent.io>
This commit is contained in:
Colin Patrick McCabe 2019-01-11 16:40:21 -08:00 committed by Jason Gustafson
parent cc33511e9a
commit 71e85f5e84
124 changed files with 8037 additions and 72 deletions

1
.gitignore vendored
View File

@ -52,3 +52,4 @@ docs/generated/
kafkatest.egg-info/ kafkatest.egg-info/
systest/ systest/
*.swp *.swp
clients/src/generated

View File

@ -839,6 +839,7 @@ project(':clients') {
testRuntime libs.slf4jlog4j testRuntime libs.slf4jlog4j
testRuntime libs.jacksonDatabind testRuntime libs.jacksonDatabind
testRuntime libs.jacksonJDK8Datatypes testRuntime libs.jacksonJDK8Datatypes
testCompile libs.jacksonJaxrsJsonProvider
} }
task determineCommitId { task determineCommitId {
@ -887,6 +888,27 @@ project(':clients') {
delete "$buildDir/kafka/" delete "$buildDir/kafka/"
} }
task processMessages(type:org.apache.kafka.task.ProcessMessagesTask) {
inputDirectory = file("src/main/resources/common/message")
outputDirectory = file("src/generated/java/org/apache/kafka/common/message")
}
sourceSets {
main {
java {
srcDirs = ["src/generated/java", "src/main/java"]
}
}
test {
java {
srcDirs = ["src/generated/java", "src/test/java",
"$rootDir/buildSrc/src/main/java/org/apache/kafka/message/"]
}
}
}
compileJava.dependsOn 'processMessages'
javadoc { javadoc {
include "**/org/apache/kafka/clients/admin/*" include "**/org/apache/kafka/clients/admin/*"
include "**/org/apache/kafka/clients/consumer/*" include "**/org/apache/kafka/clients/consumer/*"

26
buildSrc/build.gradle Normal file
View File

@ -0,0 +1,26 @@
// 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.
repositories {
mavenCentral()
}
dependencies {
compile "com.fasterxml.jackson.core:jackson-databind:2.9.6"
compile "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:2.9.6"
compile "com.fasterxml.jackson.datatype:jackson-datatype-jdk8:2.9.6"
}
test.enabled=false

View File

@ -0,0 +1,123 @@
/*
* 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.message;
import java.io.BufferedWriter;
import java.io.IOException;
import java.util.Map;
import java.util.TreeMap;
public final class ApiMessageFactoryGenerator {
private final TreeMap<Short, String> requestApis;
private final TreeMap<Short, String> responseApis;
private final HeaderGenerator headerGenerator;
private final CodeBuffer buffer;
public void registerMessageType(MessageSpec spec) {
if (spec.type() == MessageSpecType.REQUEST) {
if (requestApis.containsKey(spec.apiKey().get())) {
throw new RuntimeException("Found more than one request with " +
"API key " + spec.apiKey().get());
}
requestApis.put(spec.apiKey().get(), spec.generatedClassName());
} else if (spec.type() == MessageSpecType.RESPONSE) {
if (responseApis.containsKey(spec.apiKey().get())) {
throw new RuntimeException("Found more than one response with " +
"API key " + spec.apiKey().get());
}
responseApis.put(spec.apiKey().get(), spec.generatedClassName());
}
}
public ApiMessageFactoryGenerator() {
this.requestApis = new TreeMap<>();
this.responseApis = new TreeMap<>();
this.headerGenerator = new HeaderGenerator();
this.buffer = new CodeBuffer();
}
public void generate() {
buffer.printf("public final class ApiMessageFactory {%n");
buffer.incrementIndent();
generateFactoryMethod("request", requestApis);
buffer.printf("%n");
generateFactoryMethod("response", responseApis);
buffer.printf("%n");
generateSchemasAccessor("request", requestApis);
buffer.printf("%n");
generateSchemasAccessor("response", responseApis);
buffer.decrementIndent();
buffer.printf("}%n");
headerGenerator.generate();
}
public void generateFactoryMethod(String type, TreeMap<Short, String> apis) {
headerGenerator.addImport(MessageGenerator.MESSAGE_CLASS);
buffer.printf("public static Message new%s(short apiKey) {%n",
MessageGenerator.capitalizeFirst(type));
buffer.incrementIndent();
buffer.printf("switch (apiKey) {%n");
buffer.incrementIndent();
for (Map.Entry<Short, String> entry : apis.entrySet()) {
buffer.printf("case %d:%n", entry.getKey());
buffer.incrementIndent();
buffer.printf("return new %s();%n", entry.getValue());
buffer.decrementIndent();
}
buffer.printf("default:%n");
buffer.incrementIndent();
headerGenerator.addImport(MessageGenerator.UNSUPPORTED_VERSION_EXCEPTION_CLASS);
buffer.printf("throw new UnsupportedVersionException(\"Unsupported %s API key \"" +
" + apiKey);%n", type);
buffer.decrementIndent();
buffer.decrementIndent();
buffer.printf("}%n");
buffer.decrementIndent();
buffer.printf("}%n");
}
public void generateSchemasAccessor(String type, TreeMap<Short, String> apis) {
headerGenerator.addImport(MessageGenerator.SCHEMA_CLASS);
buffer.printf("public static Schema[] %sSchemas(short apiKey) {%n",
MessageGenerator.lowerCaseFirst(type));
buffer.incrementIndent();
buffer.printf("switch (apiKey) {%n");
buffer.incrementIndent();
for (Map.Entry<Short, String> entry : apis.entrySet()) {
buffer.printf("case %d:%n", entry.getKey());
buffer.incrementIndent();
buffer.printf("return %s.SCHEMAS;%n", entry.getValue());
buffer.decrementIndent();
}
buffer.printf("default:%n");
buffer.incrementIndent();
headerGenerator.addImport(MessageGenerator.UNSUPPORTED_VERSION_EXCEPTION_CLASS);
buffer.printf("throw new UnsupportedVersionException(\"Unsupported %s API key \"" +
" + apiKey);%n", type);
buffer.decrementIndent();
buffer.decrementIndent();
buffer.printf("}%n");
buffer.decrementIndent();
buffer.printf("}%n");
}
public void write(BufferedWriter writer) throws IOException {
headerGenerator.buffer().write(writer);
buffer.write(writer);
}
}

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.message;
import java.io.IOException;
import java.io.Writer;
import java.util.ArrayList;
public class CodeBuffer {
private final ArrayList<String> lines;
private int indent;
public CodeBuffer() {
this.lines = new ArrayList<>();
this.indent = 0;
}
public void incrementIndent() {
indent++;
}
public void decrementIndent() {
indent--;
if (indent < 0) {
throw new RuntimeException("Indent < 0");
}
}
public void printf(String format, Object... args) {
lines.add(String.format(indentSpaces() + format, args));
}
public void write(Writer writer) throws IOException {
for (String line : lines) {
writer.write(line);
}
}
public void write(CodeBuffer other) {
for (String line : lines) {
other.lines.add(other.indentSpaces() + line);
}
}
private String indentSpaces() {
StringBuilder bld = new StringBuilder();
for (int i = 0; i < indent; i++) {
bld.append(" ");
}
return bld.toString();
}
@Override
public boolean equals(Object other) {
if (!(other instanceof CodeBuffer)) {
return false;
}
CodeBuffer o = (CodeBuffer) other;
return lines.equals(o.lines);
}
@Override
public int hashCode() {
return lines.hashCode();
}
}

View File

@ -0,0 +1,158 @@
/*
* 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.message;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
public final class FieldSpec {
private final String name;
private final Versions versions;
private final List<FieldSpec> fields;
private final FieldType type;
private final boolean mapKey;
private final Versions nullableVersions;
private final String fieldDefault;
private final boolean ignorable;
private final String about;
@JsonCreator
public FieldSpec(@JsonProperty("name") String name,
@JsonProperty("versions") String versions,
@JsonProperty("fields") List<FieldSpec> fields,
@JsonProperty("type") String type,
@JsonProperty("mapKey") boolean mapKey,
@JsonProperty("nullableVersions") String nullableVersions,
@JsonProperty("default") String fieldDefault,
@JsonProperty("ignorable") boolean ignorable,
@JsonProperty("about") String about) {
this.name = Objects.requireNonNull(name);
this.versions = Versions.parse(versions, null);
if (this.versions == null) {
throw new RuntimeException("You must specify the version of the " +
name + " structure.");
}
this.fields = Collections.unmodifiableList(fields == null ?
Collections.emptyList() : new ArrayList<>(fields));
this.type = FieldType.parse(Objects.requireNonNull(type));
this.mapKey = mapKey;
this.nullableVersions = Versions.parse(nullableVersions, Versions.NONE);
if (!this.nullableVersions.empty()) {
if (!this.type.canBeNullable()) {
throw new RuntimeException("Type " + this.type + " cannot be nullable.");
}
}
this.fieldDefault = fieldDefault == null ? "" : fieldDefault;
this.ignorable = ignorable;
this.about = about == null ? "" : about;
if (!this.fields().isEmpty()) {
if (!this.type.isArray()) {
throw new RuntimeException("Non-array field " + name + " cannot have fields");
}
}
}
public StructSpec toStruct() {
if ((!this.type.isArray()) && (this.type.isStruct())) {
throw new RuntimeException("Field " + name + " cannot be treated as a structure.");
}
return new StructSpec(name, versions.toString(), fields);
}
@JsonProperty("name")
public String name() {
return name;
}
String capitalizedCamelCaseName() {
return MessageGenerator.capitalizeFirst(name);
}
String camelCaseName() {
return MessageGenerator.lowerCaseFirst(name);
}
String snakeCaseName() {
return MessageGenerator.toSnakeCase(name);
}
public Versions versions() {
return versions;
}
@JsonProperty("versions")
public String versionsString() {
return versions.toString();
}
@JsonProperty("fields")
public List<FieldSpec> fields() {
return fields;
}
@JsonProperty("type")
public String typeString() {
return type.toString();
}
public FieldType type() {
return type;
}
@JsonProperty("mapKey")
public boolean mapKey() {
return mapKey;
}
public Versions nullableVersions() {
return nullableVersions;
}
@JsonProperty("nullableVersions")
public String nullableVersionsString() {
return nullableVersions.toString();
}
@JsonProperty("default")
public String defaultString() {
return fieldDefault;
}
@JsonProperty("ignorable")
public boolean ignorable() {
return ignorable;
}
@JsonProperty("about")
public String about() {
return about;
}
}

View File

@ -0,0 +1,281 @@
/*
* 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.message;
import java.util.Optional;
public interface FieldType {
String STRUCT_PREFIX = "[]";
final class BoolFieldType implements FieldType {
private static final BoolFieldType INSTANCE = new BoolFieldType();
private static final String NAME = "bool";
@Override
public Optional<Integer> fixedLength() {
return Optional.of(1);
}
@Override
public String toString() {
return NAME;
}
}
final class Int8FieldType implements FieldType {
private static final Int8FieldType INSTANCE = new Int8FieldType();
private static final String NAME = "int8";
@Override
public Optional<Integer> fixedLength() {
return Optional.of(1);
}
@Override
public String toString() {
return NAME;
}
}
final class Int16FieldType implements FieldType {
private static final Int16FieldType INSTANCE = new Int16FieldType();
private static final String NAME = "int16";
@Override
public Optional<Integer> fixedLength() {
return Optional.of(2);
}
@Override
public String toString() {
return NAME;
}
}
final class Int32FieldType implements FieldType {
private static final Int32FieldType INSTANCE = new Int32FieldType();
private static final String NAME = "int32";
@Override
public Optional<Integer> fixedLength() {
return Optional.of(4);
}
@Override
public String toString() {
return NAME;
}
}
final class Int64FieldType implements FieldType {
private static final Int64FieldType INSTANCE = new Int64FieldType();
private static final String NAME = "int64";
@Override
public Optional<Integer> fixedLength() {
return Optional.of(8);
}
@Override
public String toString() {
return NAME;
}
}
final class StringFieldType implements FieldType {
private static final StringFieldType INSTANCE = new StringFieldType();
private static final String NAME = "string";
@Override
public boolean isString() {
return true;
}
@Override
public boolean canBeNullable() {
return true;
}
@Override
public String toString() {
return NAME;
}
}
final class BytesFieldType implements FieldType {
private static final BytesFieldType INSTANCE = new BytesFieldType();
private static final String NAME = "bytes";
@Override
public boolean isBytes() {
return true;
}
@Override
public boolean canBeNullable() {
return true;
}
@Override
public String toString() {
return NAME;
}
}
final class StructType implements FieldType {
private final String type;
StructType(String type) {
this.type = type;
}
@Override
public boolean isStruct() {
return true;
}
@Override
public String toString() {
return type;
}
}
final class ArrayType implements FieldType {
private final FieldType elementType;
ArrayType(FieldType elementType) {
this.elementType = elementType;
}
@Override
public boolean isArray() {
return true;
}
@Override
public boolean isStructArray() {
return elementType.isStruct();
}
@Override
public boolean canBeNullable() {
return true;
}
public FieldType elementType() {
return elementType;
}
@Override
public String toString() {
return "[]" + elementType.toString();
}
}
static FieldType parse(String string) {
string = string.trim();
switch (string) {
case BoolFieldType.NAME:
return BoolFieldType.INSTANCE;
case Int8FieldType.NAME:
return Int8FieldType.INSTANCE;
case Int16FieldType.NAME:
return Int16FieldType.INSTANCE;
case Int32FieldType.NAME:
return Int32FieldType.INSTANCE;
case Int64FieldType.NAME:
return Int64FieldType.INSTANCE;
case StringFieldType.NAME:
return StringFieldType.INSTANCE;
case BytesFieldType.NAME:
return BytesFieldType.INSTANCE;
default:
if (string.startsWith(STRUCT_PREFIX)) {
String elementTypeString = string.substring(STRUCT_PREFIX.length());
if (elementTypeString.length() == 0) {
throw new RuntimeException("Can't parse array type " + string +
". No element type found.");
}
FieldType elementType = parse(elementTypeString);
if (elementType.isArray()) {
throw new RuntimeException("Can't have an array of arrays. " +
"Use an array of structs containing an array instead.");
}
return new ArrayType(elementType);
} else if (MessageGenerator.firstIsCapitalized(string)) {
return new StructType(string);
} else {
throw new RuntimeException("Can't parse type " + string);
}
}
}
/**
* Returns true if this is an array type.
*/
default boolean isArray() {
return false;
}
/**
* Returns true if this is an array of structures.
*/
default boolean isStructArray() {
return false;
}
/**
* Returns true if this is a string type.
*/
default boolean isString() {
return false;
}
/**
* Returns true if this is a bytes type.
*/
default boolean isBytes() {
return false;
}
/**
* Returns true if this is a struct type.
*/
default boolean isStruct() {
return false;
}
/**
* Returns true if this field type is compatible with nullability.
*/
default boolean canBeNullable() {
return false;
}
/**
* Gets the fixed length of the field, or None if the field is variable-length.
*/
default Optional<Integer> fixedLength() {
return Optional.empty();
}
/**
* Convert the field type to a JSON string.
*/
String toString();
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.message;
import java.util.TreeSet;
/**
* The Kafka header generator.
*/
public final class HeaderGenerator {
private static final String[] HEADER = new String[] {
"/*",
" * 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.",
" */",
"",
"// THIS CODE IS AUTOMATICALLY GENERATED. DO NOT EDIT.",
""
};
static final String PACKAGE = "org.apache.kafka.common.message";
private final CodeBuffer buffer;
private final TreeSet<String> imports;
public HeaderGenerator() {
this.buffer = new CodeBuffer();
this.imports = new TreeSet<>();
}
public void addImport(String newImport) {
this.imports.add(newImport);
}
public void generate() {
for (int i = 0; i < HEADER.length; i++) {
buffer.printf("%s%n", HEADER[i]);
}
buffer.printf("package %s;%n", PACKAGE);
buffer.printf("%n");
for (String newImport : imports) {
buffer.printf("import %s;%n", newImport);
}
buffer.printf("%n");
}
public CodeBuffer buffer() {
return buffer;
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,202 @@
/*
* 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.message;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
import java.io.BufferedWriter;
import java.io.File;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Locale;
import java.util.Map;
/**
* The Kafka message generator.
*/
public final class MessageGenerator {
static final String JSON_SUFFIX = ".json";
static final String JSON_GLOB = "*" + JSON_SUFFIX;
static final String JAVA_SUFFIX = ".java";
static final String API_MESSAGE_FACTORY_JAVA = "ApiMessageFactory.java";
static final String API_MESSAGE_CLASS = "org.apache.kafka.common.protocol.ApiMessage";
static final String MESSAGE_CLASS = "org.apache.kafka.common.protocol.Message";
static final String MESSAGE_UTIL_CLASS = "org.apache.kafka.common.protocol.MessageUtil";
static final String READABLE_CLASS = "org.apache.kafka.common.protocol.Readable";
static final String WRITABLE_CLASS = "org.apache.kafka.common.protocol.Writable";
static final String ARRAYS_CLASS = "java.util.Arrays";
static final String LIST_CLASS = "java.util.List";
static final String ARRAYLIST_CLASS = "java.util.ArrayList";
static final String IMPLICIT_LINKED_HASH_MULTI_SET_CLASS =
"org.apache.kafka.common.utils.ImplicitLinkedHashMultiSet";
static final String UNSUPPORTED_VERSION_EXCEPTION_CLASS =
"org.apache.kafka.common.errors.UnsupportedVersionException";
static final String ITERATOR_CLASS = "java.util.Iterator";
static final String TYPE_CLASS = "org.apache.kafka.common.protocol.types.Type";
static final String FIELD_CLASS = "org.apache.kafka.common.protocol.types.Field";
static final String SCHEMA_CLASS = "org.apache.kafka.common.protocol.types.Schema";
static final String ARRAYOF_CLASS = "org.apache.kafka.common.protocol.types.ArrayOf";
static final String STRUCT_CLASS = "org.apache.kafka.common.protocol.types.Struct";
static final String BYTES_CLASS = "org.apache.kafka.common.utils.Bytes";
/**
* The Jackson serializer we use for JSON objects.
*/
static final ObjectMapper JSON_SERDE;
static {
JSON_SERDE = new ObjectMapper();
JSON_SERDE.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
JSON_SERDE.configure(DeserializationFeature.ACCEPT_SINGLE_VALUE_AS_ARRAY, true);
JSON_SERDE.configure(JsonParser.Feature.ALLOW_COMMENTS, true);
JSON_SERDE.setSerializationInclusion(JsonInclude.Include.NON_EMPTY);
}
public static Map<String, File> getOutputFiles(String outputDir, String inputDir) throws Exception {
HashMap<String, File> outputFiles = new HashMap<>();
for (Path inputPath : Files.newDirectoryStream(Paths.get(inputDir), JSON_GLOB)) {
String jsonName = inputPath.getFileName().toString();
String javaName = jsonName.substring(0, jsonName.length() - JSON_SUFFIX.length()) + "Data.java";
File outputFile = new File(outputDir, javaName);
outputFiles.put(outputFile.toString(), outputFile);
}
File factoryFile = new File(outputDir, API_MESSAGE_FACTORY_JAVA);
outputFiles.put(factoryFile.toString(), factoryFile);
return outputFiles;
}
public static void processDirectories(String outputDir, String inputDir) throws Exception {
Files.createDirectories(Paths.get(outputDir));
int numProcessed = 0;
ApiMessageFactoryGenerator messageFactoryGenerator = new ApiMessageFactoryGenerator();
HashSet<String> outputFileNames = new HashSet<>();
for (Path inputPath : Files.newDirectoryStream(Paths.get(inputDir), JSON_GLOB)) {
try {
MessageSpec spec = JSON_SERDE.
readValue(inputPath.toFile(), MessageSpec.class);
String javaName = spec.generatedClassName() + JAVA_SUFFIX;
outputFileNames.add(javaName);
Path outputPath = Paths.get(outputDir, javaName);
try (BufferedWriter writer = Files.newBufferedWriter(outputPath)) {
MessageDataGenerator generator = new MessageDataGenerator();
generator.generate(spec);
generator.write(writer);
}
numProcessed++;
messageFactoryGenerator.registerMessageType(spec);
} catch (Exception e) {
throw new RuntimeException("Exception while processing " + inputPath.toString(), e);
}
}
Path factoryOutputPath = Paths.get(outputDir, API_MESSAGE_FACTORY_JAVA);
outputFileNames.add(API_MESSAGE_FACTORY_JAVA);
try (BufferedWriter writer = Files.newBufferedWriter(factoryOutputPath)) {
messageFactoryGenerator.generate();
messageFactoryGenerator.write(writer);
}
numProcessed++;
for (Path outputPath : Files.newDirectoryStream(Paths.get(outputDir))) {
if (!outputFileNames.contains(outputPath.getFileName().toString())) {
Files.delete(outputPath);
}
}
System.out.printf("MessageGenerator: processed %d Kafka message JSON files(s).%n", numProcessed);
}
static String capitalizeFirst(String string) {
if (string.isEmpty()) {
return string;
}
return string.substring(0, 1).toUpperCase(Locale.ENGLISH) +
string.substring(1);
}
static String lowerCaseFirst(String string) {
if (string.isEmpty()) {
return string;
}
return string.substring(0, 1).toLowerCase(Locale.ENGLISH) +
string.substring(1);
}
static boolean firstIsCapitalized(String string) {
if (string.isEmpty()) {
return false;
}
return Character.isUpperCase(string.charAt(0));
}
static String toSnakeCase(String string) {
StringBuilder bld = new StringBuilder();
boolean prevWasCapitalized = true;
for (int i = 0; i < string.length(); i++) {
char c = string.charAt(i);
if (Character.isUpperCase(c)) {
if (!prevWasCapitalized) {
bld.append('_');
}
bld.append(Character.toLowerCase(c));
prevWasCapitalized = true;
} else {
bld.append(c);
prevWasCapitalized = false;
}
}
return bld.toString();
}
private final static String USAGE = "MessageGenerator: [output Java file] [input JSON file]";
public static void main(String[] args) throws Exception {
if (args.length == 0) {
System.out.println(USAGE);
System.exit(0);
} else if (args.length != 2) {
System.out.println(USAGE);
System.exit(1);
}
processDirectories(args[0], args[1]);
}
}

View File

@ -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.message;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
public final class MessageSpec {
private final StructSpec struct;
private final Optional<Short> apiKey;
private final MessageSpecType type;
@JsonCreator
public MessageSpec(@JsonProperty("name") String name,
@JsonProperty("validVersions") String validVersions,
@JsonProperty("fields") List<FieldSpec> fields,
@JsonProperty("apiKey") Short apiKey,
@JsonProperty("type") MessageSpecType type) {
this.struct = new StructSpec(name, validVersions, fields);
this.apiKey = apiKey == null ? Optional.empty() : Optional.of(apiKey);
this.type = Objects.requireNonNull(type);
}
public StructSpec struct() {
return struct;
}
@JsonProperty("name")
public String name() {
return struct.name();
}
@JsonProperty("validVersions")
public String validVersionsString() {
return struct.versionsString();
}
@JsonProperty("fields")
public List<FieldSpec> fields() {
return struct.fields();
}
@JsonProperty("apiKey")
public Optional<Short> apiKey() {
return apiKey;
}
@JsonProperty("type")
public MessageSpecType type() {
return type;
}
public String generatedClassName() {
return struct.name() + "Data";
}
}

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.message;
import com.fasterxml.jackson.annotation.JsonProperty;
public enum MessageSpecType {
@JsonProperty("request")
REQUEST,
@JsonProperty("response")
RESPONSE,
@JsonProperty("header")
HEADER;
}

View File

@ -0,0 +1,243 @@
/*
* 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.message;
import java.util.HashMap;
import java.util.Map;
import java.util.TreeMap;
/**
* Generates Schemas for Kafka MessageData classes.
*/
final class SchemaGenerator {
/**
* Schema information for a particular message.
*/
static class MessageInfo {
/**
* The versions of this message that we want to generate a schema for.
* This will be constrained by the valid versions for the parent objects.
* For example, if the parent message is valid for versions 0 and 1,
* we will only generate a version 0 and version 1 schema for child classes,
* even if their valid versions are "0+".
*/
private final Versions versions;
/**
* Maps versions to schema declaration code. If the schema for a
* particular version is the same as that of a previous version,
* there will be no entry in the map for it.
*/
private final TreeMap<Short, CodeBuffer> schemaForVersion;
MessageInfo(Versions versions) {
this.versions = versions;
this.schemaForVersion = new TreeMap<>();
}
}
/**
* The header file generator. This is shared with the MessageDataGenerator
* instance that owns this SchemaGenerator.
*/
private final HeaderGenerator headerGenerator;
/**
* Maps message names to message information.
*/
private final Map<String, MessageInfo> messages;
SchemaGenerator(HeaderGenerator headerGenerator) {
this.headerGenerator = headerGenerator;
this.messages = new HashMap<>();
}
void generateSchemas(MessageSpec message) throws Exception {
generateSchemas(message.generatedClassName(), message.struct(),
message.struct().versions());
}
void generateSchemas(String className, StructSpec struct,
Versions parentVersions) throws Exception {
Versions versions = parentVersions.intersect(struct.versions());
MessageInfo messageInfo = messages.get(className);
if (messageInfo != null) {
return;
}
messageInfo = new MessageInfo(versions);
messages.put(className, messageInfo);
// Process the leaf classes first.
for (FieldSpec field : struct.fields()) {
if (field.type().isStructArray()) {
FieldType.ArrayType arrayType = (FieldType.ArrayType) field.type();
generateSchemas(arrayType.elementType().toString(), field.toStruct(), versions);
} else if (field.type().isStruct()) {
generateSchemas(field.type().toString(), field.toStruct(), versions);
}
}
CodeBuffer prev = null;
for (short v = versions.lowest(); v <= versions.highest(); v++) {
CodeBuffer cur = new CodeBuffer();
generateSchemaForVersion(struct, v, cur);
// If this schema version is different from the previous one,
// create a new map entry.
if (!cur.equals(prev)) {
messageInfo.schemaForVersion.put(v, cur);
}
prev = cur;
}
}
private void generateSchemaForVersion(StructSpec struct, short version,
CodeBuffer buffer) throws Exception {
// Find the last valid field index.
int lastValidIndex = struct.fields().size() - 1;
while (true) {
if (lastValidIndex < 0) {
break;
}
FieldSpec field = struct.fields().get(lastValidIndex);
if (field.versions().contains(version)) {
break;
}
lastValidIndex--;
}
headerGenerator.addImport(MessageGenerator.SCHEMA_CLASS);
buffer.printf("new Schema(%n");
buffer.incrementIndent();
for (int i = 0; i <= lastValidIndex; i++) {
FieldSpec field = struct.fields().get(i);
if (!field.versions().contains(version)) {
continue;
}
headerGenerator.addImport(MessageGenerator.FIELD_CLASS);
buffer.printf("new Field(\"%s\", %s, \"%s\")%s%n",
field.snakeCaseName(),
fieldTypeToSchemaType(field, version),
field.about(),
i == lastValidIndex ? "" : ",");
}
buffer.decrementIndent();
buffer.printf(");%n");
}
private String fieldTypeToSchemaType(FieldSpec field, short version) {
return fieldTypeToSchemaType(field.type(),
field.nullableVersions().contains(version),
version);
}
private String fieldTypeToSchemaType(FieldType type, boolean nullable, short version) {
if (type instanceof FieldType.BoolFieldType) {
headerGenerator.addImport(MessageGenerator.TYPE_CLASS);
if (nullable) {
throw new RuntimeException("Type " + type + " cannot be nullable.");
}
return "Type.BOOLEAN";
} else if (type instanceof FieldType.Int8FieldType) {
headerGenerator.addImport(MessageGenerator.TYPE_CLASS);
if (nullable) {
throw new RuntimeException("Type " + type + " cannot be nullable.");
}
return "Type.INT8";
} else if (type instanceof FieldType.Int16FieldType) {
headerGenerator.addImport(MessageGenerator.TYPE_CLASS);
if (nullable) {
throw new RuntimeException("Type " + type + " cannot be nullable.");
}
return "Type.INT16";
} else if (type instanceof FieldType.Int32FieldType) {
headerGenerator.addImport(MessageGenerator.TYPE_CLASS);
if (nullable) {
throw new RuntimeException("Type " + type + " cannot be nullable.");
}
return "Type.INT32";
} else if (type instanceof FieldType.Int64FieldType) {
headerGenerator.addImport(MessageGenerator.TYPE_CLASS);
if (nullable) {
throw new RuntimeException("Type " + type + " cannot be nullable.");
}
return "Type.INT64";
} else if (type instanceof FieldType.StringFieldType) {
headerGenerator.addImport(MessageGenerator.TYPE_CLASS);
return nullable ? "Type.NULLABLE_STRING" : "Type.STRING";
} else if (type instanceof FieldType.BytesFieldType) {
headerGenerator.addImport(MessageGenerator.TYPE_CLASS);
return nullable ? "Type.NULLABLE_BYTES" : "Type.BYTES";
} else if (type.isArray()) {
headerGenerator.addImport(MessageGenerator.ARRAYOF_CLASS);
FieldType.ArrayType arrayType = (FieldType.ArrayType) type;
String prefix = nullable ? "ArrayOf.nullable" : "new ArrayOf";
return String.format("%s(%s)", prefix,
fieldTypeToSchemaType(arrayType.elementType(), false, version));
} else if (type.isStruct()) {
if (nullable) {
throw new RuntimeException("Type " + type + " cannot be nullable.");
}
return String.format("%s.SCHEMA_%d", type.toString(),
floorVersion(type.toString(), version));
} else {
throw new RuntimeException("Unsupported type " + type);
}
}
/**
* Find the lowest schema version for a given class that is the same as the
* given version.
*/
private short floorVersion(String className, short v) {
MessageInfo message = messages.get(className);
return message.schemaForVersion.floorKey(v);
}
/**
* Write the message schema to the provided buffer.
*
* @param className The class name.
* @param buffer The destination buffer.
*/
void writeSchema(String className, CodeBuffer buffer) throws Exception {
MessageInfo messageInfo = messages.get(className);
Versions versions = messageInfo.versions;
for (short v = versions.lowest(); v <= versions.highest(); v++) {
CodeBuffer declaration = messageInfo.schemaForVersion.get(v);
if (declaration == null) {
buffer.printf("public static final Schema SCHEMA_%d = SCHEMA_%d;%n", v, v - 1);
} else {
buffer.printf("public static final Schema SCHEMA_%d =%n", v);
buffer.incrementIndent();
declaration.write(buffer);
buffer.decrementIndent();
}
buffer.printf("%n");
}
buffer.printf("public static final Schema[] SCHEMAS = new Schema[] {%n");
buffer.incrementIndent();
for (short v = 0; v < versions.lowest(); v++) {
buffer.printf("null%s%n", (v == versions.highest()) ? "" : ",");
}
for (short v = versions.lowest(); v <= versions.highest(); v++) {
buffer.printf("SCHEMA_%d%s%n", v, (v == versions.highest()) ? "" : ",");
}
buffer.decrementIndent();
buffer.printf("};%n");
buffer.printf("%n");
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.message;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
public final class StructSpec {
private final String name;
private final Versions versions;
private final List<FieldSpec> fields;
private final boolean hasKeys;
@JsonCreator
public StructSpec(@JsonProperty("name") String name,
@JsonProperty("versions") String versions,
@JsonProperty("fields") List<FieldSpec> fields) {
this.name = Objects.requireNonNull(name);
this.versions = Versions.parse(versions, null);
if (this.versions == null) {
throw new RuntimeException("You must specify the version of the " +
name + " structure.");
}
this.fields = Collections.unmodifiableList(fields == null ?
Collections.emptyList() : new ArrayList<>(fields));
this.hasKeys = this.fields.stream().anyMatch(f -> f.mapKey());
}
@JsonProperty
public String name() {
return name;
}
public Versions versions() {
return versions;
}
@JsonProperty
public String versionsString() {
return versions.toString();
}
@JsonProperty
public List<FieldSpec> fields() {
return fields;
}
boolean hasKeys() {
return hasKeys;
}
}

View File

@ -0,0 +1,139 @@
/*
* 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.message;
import java.util.Objects;
/**
* A version range.
*
* A range consists of two 16-bit numbers: the lowest version which is accepted, and the highest.
* Ranges are inclusive, meaning that both the lowest and the highest version are valid versions.
* The only exception to this is the NONE range, which contains no versions at all.
*
* Version ranges can be represented as strings.
*
* A single supported version V is represented as "V".
* A bounded range from A to B is represented as "A-B".
* All versions greater than A is represented as "A+".
* The NONE range is represented as an the string "none".
*/
public final class Versions {
private final short lowest;
private final short highest;
public static Versions parse(String input, Versions defaultVersions) {
if (input == null) {
return defaultVersions;
}
String trimmedInput = input.trim();
if (trimmedInput.length() == 0) {
return defaultVersions;
}
if (trimmedInput.equals(NONE_STRING)) {
return NONE;
}
if (trimmedInput.endsWith("+")) {
return new Versions(Short.parseShort(
trimmedInput.substring(0, trimmedInput.length() - 1)),
Short.MAX_VALUE);
} else {
int dashIndex = trimmedInput.indexOf("-");
if (dashIndex < 0) {
short version = Short.parseShort(trimmedInput);
return new Versions(version, version);
}
return new Versions(
Short.parseShort(trimmedInput.substring(0, dashIndex)),
Short.parseShort(trimmedInput.substring(dashIndex + 1)));
}
}
public static final Versions ALL = new Versions((short) 0, Short.MAX_VALUE);
public static final Versions NONE = new Versions();
public static final String NONE_STRING = "none";
private Versions() {
this.lowest = 0;
this.highest = -1;
}
public Versions(short lowest, short highest) {
if ((lowest < 0) || (highest < 0)) {
throw new RuntimeException("Invalid version range " +
lowest + " to " + highest);
}
this.lowest = lowest;
this.highest = highest;
}
public short lowest() {
return lowest;
}
public short highest() {
return highest;
}
public boolean empty() {
return lowest > highest;
}
@Override
public String toString() {
if (empty()) {
return NONE_STRING;
} else if (lowest == highest) {
return String.valueOf(lowest);
} else if (highest == Short.MAX_VALUE) {
return String.format("%d+", lowest);
} else {
return String.format("%d-%d", lowest, highest);
}
}
public Versions intersect(Versions other) {
short newLowest = lowest > other.lowest ? lowest : other.lowest;
short newHighest = highest < other.highest ? highest : other.highest;
if (newLowest > newHighest) {
return Versions.NONE;
}
return new Versions(newLowest, newHighest);
}
public boolean contains(short version) {
return version >= lowest && version <= highest;
}
@Override
public int hashCode() {
return Objects.hash(lowest, highest);
}
@Override
public boolean equals(Object other) {
if (!(other instanceof Versions)) {
return false;
}
Versions otherVersions = (Versions) other;
return lowest == otherVersions.lowest &&
highest == otherVersions.highest;
}
}

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.task;
import org.apache.kafka.message.MessageGenerator;
import org.gradle.api.DefaultTask;
import org.gradle.api.tasks.InputDirectory;
import org.gradle.api.tasks.OutputFiles;
import org.gradle.api.tasks.TaskAction;
import java.io.File;
import java.util.Map;
/**
* A gradle task which processes a directory full of JSON files into an output directory.
*/
public class ProcessMessagesTask extends DefaultTask {
/**
* The directory where we should read the input JSON from.
*/
public File inputDirectory;
/**
* The directory that we should write output JSON to.
*/
public File outputDirectory;
@InputDirectory
public File getInputDirectory() {
return inputDirectory;
}
/**
* Define the task outputs.
*
* Gradle consults this to see if the task is up-to-date.
*/
@OutputFiles
public Map<String, File> getOutputFiles() throws Exception {
return MessageGenerator.getOutputFiles(
outputDirectory.toString(), inputDirectory.toString());
}
@TaskAction
public void run() {
try {
MessageGenerator.processDirectories(
outputDirectory.toString(), inputDirectory.toString());
} catch (Exception e) {
throw new RuntimeException(e);
}
}
}

View File

@ -67,6 +67,13 @@
<allow pkg="org.apache.kafka.common.metrics" /> <allow pkg="org.apache.kafka.common.metrics" />
</subpackage> </subpackage>
<subpackage name="message">
<allow pkg="com.fasterxml.jackson" />
<allow pkg="org.apache.kafka.common.protocol" />
<allow pkg="org.apache.kafka.common.protocol.types" />
<allow pkg="org.apache.kafka.common.message" />
</subpackage>
<subpackage name="metrics"> <subpackage name="metrics">
<allow pkg="org.apache.kafka.common.metrics" /> <allow pkg="org.apache.kafka.common.metrics" />
</subpackage> </subpackage>
@ -206,6 +213,11 @@
<allow pkg="org.glassfish.jersey" /> <allow pkg="org.glassfish.jersey" />
</subpackage> </subpackage>
<subpackage name="message">
<allow pkg="com.fasterxml.jackson" />
<allow pkg="com.fasterxml.jackson.annotation" />
</subpackage>
<subpackage name="streams"> <subpackage name="streams">
<allow pkg="org.apache.kafka.common"/> <allow pkg="org.apache.kafka.common"/>
<allow pkg="org.apache.kafka.test"/> <allow pkg="org.apache.kafka.test"/>

View File

@ -51,7 +51,7 @@
files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData).java"/> files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData).java"/>
<suppress checks="CyclomaticComplexity" <suppress checks="CyclomaticComplexity"
files="(ConsumerCoordinator|Fetcher|Sender|KafkaProducer|BufferPool|ConfigDef|RecordAccumulator|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator).java"/> files="(ConsumerCoordinator|Fetcher|Sender|KafkaProducer|BufferPool|ConfigDef|RecordAccumulator|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|SchemaGenerator).java"/>
<suppress checks="JavaNCSS" <suppress checks="JavaNCSS"
files="AbstractRequest.java|KerberosLogin.java|WorkerSinkTaskTest.java|TransactionManagerTest.java"/> files="AbstractRequest.java|KerberosLogin.java|WorkerSinkTaskTest.java|TransactionManagerTest.java"/>
@ -218,4 +218,11 @@
<suppress checks="JavaNCSS" <suppress checks="JavaNCSS"
files="RequestResponseTest.java"/> files="RequestResponseTest.java"/>
<suppress checks="(NPathComplexity|ClassFanOutComplexity|CyclomaticComplexity|ClassDataAbstractionCoupling)"
files="clients/src/generated/.+.java$"/>
<suppress checks="NPathComplexity"
files="MessageTest.java"/>
<suppress checks="CyclomaticComplexity" files="MessageDataGenerator.java"/>
</suppressions> </suppressions>

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.common.protocol;
/**
* A Message which is part of the top-level Kafka API.
*/
public interface ApiMessage extends Message {
/**
* Returns the API key of this message, or -1 if there is none.
*/
short apiKey();
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.protocol;
import java.nio.ByteBuffer;
public class ByteBufferAccessor implements Readable, Writable {
private final ByteBuffer buf;
public ByteBufferAccessor(ByteBuffer buf) {
this.buf = buf;
}
@Override
public byte readByte() {
return buf.get();
}
@Override
public short readShort() {
return buf.getShort();
}
@Override
public int readInt() {
return buf.getInt();
}
@Override
public long readLong() {
return buf.getLong();
}
@Override
public void readArray(byte[] arr) {
buf.get(arr);
}
@Override
public void writeByte(byte val) {
buf.put(val);
}
@Override
public void writeShort(short val) {
buf.putShort(val);
}
@Override
public void writeInt(int val) {
buf.putInt(val);
}
@Override
public void writeLong(long val) {
buf.putLong(val);
}
@Override
public void writeArray(byte[] arr) {
buf.put(arr);
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.protocol;
import org.apache.kafka.common.protocol.types.Struct;
/**
* An object that can serialize itself. The serialization protocol is versioned.
* Messages also implement toString, equals, and hashCode.
*/
public interface Message {
/**
* Returns the lowest supported API key of this message, inclusive.
*/
short lowestSupportedVersion();
/**
* Returns the highest supported API key of this message, inclusive.
*/
short highestSupportedVersion();
/**
* Returns the number of bytes it would take to write out this message.
*
* @param version The version to use.
*
* @throws {@see org.apache.kafka.common.errors.UnsupportedVersionException}
* If the specified version is too new to be supported
* by this software.
*/
int size(short version);
/**
* Writes out this message to the given ByteBuffer.
*
* @param writable The destination writable.
* @param version The version to use.
*
* @throws {@see org.apache.kafka.common.errors.UnsupportedVersionException}
* If the specified version is too new to be supported
* by this software.
*/
void write(Writable writable, short version);
/**
* Reads this message from the given ByteBuffer. This will overwrite all
* relevant fields with information from the byte buffer.
*
* @param readable The source readable.
* @param version The version to use.
*
* @throws {@see org.apache.kafka.common.errors.UnsupportedVersionException}
* If the specified version is too new to be supported
* by this software.
*/
void read(Readable readable, short version);
/**
* Reads this message from the a Struct object. This will overwrite all
* relevant fields with information from the Struct.
*
* @param struct The source struct.
* @param version The version to use.
*/
void fromStruct(Struct struct, short version);
/**
* Writes out this message to a Struct.
*
* @param version The version to use.
*
* @throws {@see org.apache.kafka.common.errors.UnsupportedVersionException}
* If the specified version is too new to be supported
* by this software.
*/
Struct toStruct(short version);
}

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.common.protocol;
import org.apache.kafka.common.utils.Utils;
import java.util.Iterator;
public final class MessageUtil {
/**
* Get the length of the UTF8 representation of a string, without allocating
* a byte buffer for the string.
*/
public static short serializedUtf8Length(CharSequence input) {
int count = Utils.utf8Length(input);
if (count > Short.MAX_VALUE) {
throw new RuntimeException("String " + input + " is too long to serialize.");
}
return (short) count;
}
public static String deepToString(Iterator<?> iter) {
StringBuilder bld = new StringBuilder("[");
String prefix = "";
while (iter.hasNext()) {
Object object = iter.next();
bld.append(prefix);
bld.append(object.toString());
prefix = ", ";
}
bld.append("]");
return bld.toString();
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.protocol;
import java.nio.charset.StandardCharsets;
public interface Readable {
byte readByte();
short readShort();
int readInt();
long readLong();
void readArray(byte[] arr);
/**
* Read a Kafka-delimited string from a byte buffer. The UTF-8 string
* length is stored in a two-byte short. If the length is negative, the
* string is null.
*/
default String readNullableString() {
int length = readShort();
if (length < 0) {
return null;
}
byte[] arr = new byte[length];
readArray(arr);
return new String(arr, StandardCharsets.UTF_8);
}
/**
* Read a Kafka-delimited array from a byte buffer. The array length is
* stored in a four-byte short.
*/
default byte[] readNullableBytes() {
int length = readInt();
if (length < 0) {
return null;
}
byte[] arr = new byte[length];
readArray(arr);
return arr;
}
}

View File

@ -0,0 +1,71 @@
/*
* 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.protocol;
import java.nio.charset.StandardCharsets;
public interface Writable {
void writeByte(byte val);
void writeShort(short val);
void writeInt(int val);
void writeLong(long val);
void writeArray(byte[] arr);
/**
* Write a nullable byte array delimited by a four-byte length prefix.
*/
default void writeNullableBytes(byte[] arr) {
if (arr == null) {
writeInt(-1);
} else {
writeBytes(arr);
}
}
/**
* Write a byte array delimited by a four-byte length prefix.
*/
default void writeBytes(byte[] arr) {
writeInt(arr.length);
writeArray(arr);
}
/**
* Write a nullable string delimited by a two-byte length prefix.
*/
default void writeNullableString(String string) {
if (string == null) {
writeShort((short) -1);
} else {
writeString(string);
}
}
/**
* Write a string delimited by a two-byte length prefix.
*/
default void writeString(String string) {
byte[] arr = string.getBytes(StandardCharsets.UTF_8);
if (arr.length > Short.MAX_VALUE) {
throw new RuntimeException("Can't store string longer than " +
Short.MAX_VALUE);
}
writeShort((short) arr.length);
writeArray(arr);
}
}

View File

@ -282,6 +282,16 @@ public class Struct {
return (ByteBuffer) result; return (ByteBuffer) result;
} }
public byte[] getByteArray(String name) {
Object result = get(name);
if (result instanceof byte[])
return (byte[]) result;
ByteBuffer buf = (ByteBuffer) result;
byte[] arr = new byte[buf.remaining()];
buf.get(arr);
return arr;
}
/** /**
* Set the given field to the specified value * Set the given field to the specified value
* *
@ -346,6 +356,11 @@ public class Struct {
return set(def.name, value); return set(def.name, value);
} }
public Struct setByteArray(String name, byte[] value) {
ByteBuffer buf = value == null ? null : ByteBuffer.wrap(value);
return set(name, buf);
}
public Struct setIfExists(Field.Array def, Object[] value) { public Struct setIfExists(Field.Array def, Object[] value) {
return setIfExists(def.name, value); return setIfExists(def.name, value);
} }

View File

@ -25,6 +25,8 @@ import java.util.Comparator;
*/ */
public class Bytes implements Comparable<Bytes> { public class Bytes implements Comparable<Bytes> {
public static final byte[] EMPTY = new byte[0];
private static final char[] HEX_CHARS_UPPER = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'}; private static final char[] HEX_CHARS_UPPER = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'};
private final byte[] bytes; private final byte[] bytes;

View File

@ -0,0 +1,142 @@
/*
* 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.utils;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
/**
* A memory-efficient hash multiset which tracks the order of insertion of elements.
* See org.apache.kafka.common.utils.ImplicitLinkedHashSet for implementation details.
*
* This class is a multi-set because it allows multiple elements to be inserted that are
* equal to each other.
*
* We use reference equality when adding elements to the set. A new element A can
* be added if there is no existing element B such that A == B. If an element B
* exists such that A.equals(B), A will still be added.
*
* When deleting an element A from the set, we will try to delete the element B such
* that A == B. If no such element can be found, we will try to delete an element B
* such that A.equals(B).
*
* contains() and find() are unchanged from the base class-- they will look for element
* based on object equality, not reference equality.
*
* This multiset does not allow null elements. It does not have internal synchronization.
*/
public class ImplicitLinkedHashMultiSet<E extends ImplicitLinkedHashSet.Element>
extends ImplicitLinkedHashSet<E> {
public ImplicitLinkedHashMultiSet() {
super(0);
}
public ImplicitLinkedHashMultiSet(int expectedNumElements) {
super(expectedNumElements);
}
public ImplicitLinkedHashMultiSet(Iterator<E> iter) {
super(iter);
}
/**
* Adds a new element to the appropriate place in the elements array.
*
* @param newElement The new element to add.
* @param addElements The elements array.
* @return The index at which the element was inserted, or INVALID_INDEX
* if the element could not be inserted.
*/
@Override
int addInternal(Element newElement, Element[] addElements) {
int slot = slot(addElements, newElement);
for (int seen = 0; seen < addElements.length; seen++) {
Element element = addElements[slot];
if (element == null) {
addElements[slot] = newElement;
return slot;
}
if (element == newElement) {
return INVALID_INDEX;
}
slot = (slot + 1) % addElements.length;
}
throw new RuntimeException("Not enough hash table slots to add a new element.");
}
/**
* Find an element matching an example element.
*
* @param key The element to match.
*
* @return The match index, or INVALID_INDEX if no match was found.
*/
@Override
int findElementToRemove(Object key) {
if (key == null) {
return INVALID_INDEX;
}
int slot = slot(elements, key);
int bestSlot = INVALID_INDEX;
for (int seen = 0; seen < elements.length; seen++) {
Element element = elements[slot];
if (element == null) {
return bestSlot;
}
if (key == element) {
return slot;
} else if (key.equals(element)) {
bestSlot = slot;
}
slot = (slot + 1) % elements.length;
}
return INVALID_INDEX;
}
/**
* Returns all of the elements e in the collection such that
* key.equals(e) and key.hashCode() == e.hashCode().
*
* @param key The element to match.
*
* @return All of the matching elements.
*/
final public List<E> findAll(E key) {
if (key == null) {
return Collections.<E>emptyList();
}
ArrayList<E> results = new ArrayList<>();
int slot = slot(elements, key);
for (int seen = 0; seen < elements.length; seen++) {
Element element = elements[slot];
if (element == null) {
break;
}
if (key.equals(element)) {
@SuppressWarnings("unchecked")
E result = (E) elements[slot];
results.add(result);
}
slot = (slot + 1) % elements.length;
}
return results;
}
}

View File

@ -22,35 +22,57 @@ import java.util.Iterator;
import java.util.NoSuchElementException; import java.util.NoSuchElementException;
/** /**
* A LinkedHashSet which is more memory-efficient than the standard implementation. * A memory-efficient hash set which tracks the order of insertion of elements.
* *
* This set preserves the order of insertion. The order of iteration will always be * Like java.util.LinkedHashSet, this collection maintains a linked list of elements.
* the order of insertion. * However, rather than using a separate linked list, this collection embeds the next
* and previous fields into the elements themselves. This reduces memory consumption,
* because it means that we only have to store one Java object per element, rather
* than multiple.
* *
* This collection requires previous and next indexes to be embedded into each * The next and previous fields are stored as array indices rather than pointers.
* element. Using array indices rather than pointers saves space on large heaps * This ensures that the fields only take 32 bits, even when pointers are 64 bits.
* where pointer compression is not in use. It also reduces the amount of time * It also makes the garbage collector's job easier, because it reduces the number of
* the garbage collector has to spend chasing pointers. * pointers that it must chase.
* *
* This class uses linear probing. Unlike HashMap (but like HashTable), we don't force * This class uses linear probing. Unlike HashMap (but like HashTable), we don't force
* the size to be a power of 2. This saves memory. * the size to be a power of 2. This saves memory.
* *
* This class does not have internal synchronization. * This set does not allow null elements. It does not have internal synchronization.
*/ */
@SuppressWarnings("unchecked")
public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> extends AbstractSet<E> { public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> extends AbstractSet<E> {
public interface Element { public interface Element {
int prev(); int prev();
void setPrev(int e); void setPrev(int prev);
int next(); int next();
void setNext(int e); void setNext(int next);
} }
/**
* A special index value used to indicate that the next or previous field is
* the head.
*/
private static final int HEAD_INDEX = -1; private static final int HEAD_INDEX = -1;
/**
* A special index value used for next and previous indices which have not
* been initialized.
*/
public static final int INVALID_INDEX = -2; public static final int INVALID_INDEX = -2;
/**
* The minimum new capacity for a non-empty implicit hash set.
*/
private static final int MIN_NONEMPTY_CAPACITY = 5;
/**
* A static empty array used to avoid object allocations when the capacity is zero.
*/
private static final Element[] EMPTY_ELEMENTS = new Element[0];
private static class HeadElement implements Element { private static class HeadElement implements Element {
static final HeadElement EMPTY = new HeadElement();
private int prev = HEAD_INDEX; private int prev = HEAD_INDEX;
private int next = HEAD_INDEX; private int next = HEAD_INDEX;
@ -122,7 +144,9 @@ public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> exte
} }
cur = next; cur = next;
next = indexToElement(head, elements, cur.next()); next = indexToElement(head, elements, cur.next());
return (E) cur; @SuppressWarnings("unchecked")
E returnValue = (E) cur;
return returnValue;
} }
@Override @Override
@ -137,16 +161,23 @@ public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> exte
private Element head; private Element head;
private Element[] elements; Element[] elements;
private int size; private int size;
/**
* Returns an iterator that will yield every element in the set.
* The elements will be returned in the order that they were inserted in.
*
* Do not modify the set while you are iterating over it (except by calling
* remove on the iterator itself, of course.)
*/
@Override @Override
public Iterator<E> iterator() { final public Iterator<E> iterator() {
return new ImplicitLinkedHashSetIterator(); return new ImplicitLinkedHashSetIterator();
} }
private static int slot(Element[] curElements, Element e) { final int slot(Element[] curElements, Object e) {
return (e.hashCode() & 0x7fffffff) % curElements.length; return (e.hashCode() & 0x7fffffff) % curElements.length;
} }
@ -158,17 +189,20 @@ public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> exte
* Therefore, we must search forward in the array until we hit a null, before * Therefore, we must search forward in the array until we hit a null, before
* concluding that the element is not present. * concluding that the element is not present.
* *
* @param example The element to match. * @param key The element to match.
* @return The match index, or INVALID_INDEX if no match was found. * @return The match index, or INVALID_INDEX if no match was found.
*/ */
private int findIndex(E example) { final private int findIndexOfEqualElement(Object key) {
int slot = slot(elements, example); if (key == null) {
return INVALID_INDEX;
}
int slot = slot(elements, key);
for (int seen = 0; seen < elements.length; seen++) { for (int seen = 0; seen < elements.length; seen++) {
Element element = elements[slot]; Element element = elements[slot];
if (element == null) { if (element == null) {
return INVALID_INDEX; return INVALID_INDEX;
} }
if (element.equals(example)) { if (key.equals(element)) {
return slot; return slot;
} }
slot = (slot + 1) % elements.length; slot = (slot + 1) % elements.length;
@ -177,43 +211,66 @@ public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> exte
} }
/** /**
* Find the element which equals() the given example element. * An element e in the collection such that e.equals(key) and
* e.hashCode() == key.hashCode().
* *
* @param example The example element. * @param key The element to match.
* @return Null if no element was found; the element, otherwise. * @return The matching element, or null if there were none.
*/ */
public E find(E example) { final public E find(E key) {
int index = findIndex(example); int index = findIndexOfEqualElement(key);
if (index == INVALID_INDEX) { if (index == INVALID_INDEX) {
return null; return null;
} }
return (E) elements[index]; @SuppressWarnings("unchecked")
E result = (E) elements[index];
return result;
} }
/** /**
* Returns the number of elements in the set. * Returns the number of elements in the set.
*/ */
@Override @Override
public int size() { final public int size() {
return size; return size;
} }
/**
* Returns true if there is at least one element e in the collection such
* that key.equals(e) and key.hashCode() == e.hashCode().
*
* @param key The object to try to match.
*/
@Override @Override
public boolean contains(Object o) { final public boolean contains(Object key) {
E example = null; return findIndexOfEqualElement(key) != INVALID_INDEX;
try {
example = (E) o;
} catch (ClassCastException e) {
return false;
}
return find(example) != null;
} }
@Override private static int calculateCapacity(int expectedNumElements) {
public boolean add(E newElement) {
if ((size + 1) >= elements.length / 2) {
// Avoid using even-sized capacities, to get better key distribution. // Avoid using even-sized capacities, to get better key distribution.
changeCapacity((2 * elements.length) + 1); int newCapacity = (2 * expectedNumElements) + 1;
// Don't use a capacity that is too small.
if (newCapacity < MIN_NONEMPTY_CAPACITY) {
return MIN_NONEMPTY_CAPACITY;
}
return newCapacity;
}
/**
* Add a new element to the collection.
*
* @param newElement The new element.
*
* @return True if the element was added to the collection;
* false if it was not, because there was an existing equal element.
*/
@Override
final public boolean add(E newElement) {
if (newElement == null) {
return false;
}
if ((size + 1) >= elements.length / 2) {
changeCapacity(calculateCapacity(elements.length));
} }
int slot = addInternal(newElement, elements); int slot = addInternal(newElement, elements);
if (slot >= 0) { if (slot >= 0) {
@ -224,7 +281,7 @@ public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> exte
return false; return false;
} }
public void mustAdd(E newElement) { final public void mustAdd(E newElement) {
if (!add(newElement)) { if (!add(newElement)) {
throw new RuntimeException("Unable to add " + newElement); throw new RuntimeException("Unable to add " + newElement);
} }
@ -236,10 +293,9 @@ public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> exte
* @param newElement The new element to add. * @param newElement The new element to add.
* @param addElements The elements array. * @param addElements The elements array.
* @return The index at which the element was inserted, or INVALID_INDEX * @return The index at which the element was inserted, or INVALID_INDEX
* if the element could not be inserted because there was already * if the element could not be inserted.
* an equivalent element.
*/ */
private static int addInternal(Element newElement, Element[] addElements) { int addInternal(Element newElement, Element[] addElements) {
int slot = slot(addElements, newElement); int slot = slot(addElements, newElement);
for (int seen = 0; seen < addElements.length; seen++) { for (int seen = 0; seen < addElements.length; seen++) {
Element element = addElements[slot]; Element element = addElements[slot];
@ -270,18 +326,35 @@ public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> exte
this.size = oldSize; this.size = oldSize;
} }
/**
* Remove the first element e such that key.equals(e)
* and key.hashCode == e.hashCode.
*
* @param key The object to try to match.
* @return True if an element was removed; false otherwise.
*/
@Override @Override
public boolean remove(Object o) { final public boolean remove(Object key) {
E example = null; int slot = findElementToRemove(key);
try {
example = (E) o;
} catch (ClassCastException e) {
return false;
}
int slot = findIndex(example);
if (slot == INVALID_INDEX) { if (slot == INVALID_INDEX) {
return false; return false;
} }
removeElementAtSlot(slot);
return true;
}
int findElementToRemove(Object key) {
return findIndexOfEqualElement(key);
}
/**
* Remove an element in a particular slot.
*
* @param slot The slot of the element to remove.
*
* @return True if an element was removed; false otherwise.
*/
private boolean removeElementAtSlot(int slot) {
size--; size--;
removeFromList(head, elements, slot); removeFromList(head, elements, slot);
slot = (slot + 1) % elements.length; slot = (slot + 1) % elements.length;
@ -328,27 +401,64 @@ public class ImplicitLinkedHashSet<E extends ImplicitLinkedHashSet.Element> exte
elements[newSlot] = element; elements[newSlot] = element;
} }
@Override /**
public void clear() { * Create a new ImplicitLinkedHashSet.
reset(elements.length); */
}
public ImplicitLinkedHashSet() { public ImplicitLinkedHashSet() {
this(5); this(0);
} }
public ImplicitLinkedHashSet(int initialCapacity) { /**
reset(initialCapacity); * Create a new ImplicitLinkedHashSet.
*
* @param expectedNumElements The number of elements we expect to have in this set.
* This is used to optimize by setting the capacity ahead
* of time rather than growing incrementally.
*/
public ImplicitLinkedHashSet(int expectedNumElements) {
clear(expectedNumElements);
} }
private void reset(int capacity) { /**
* Create a new ImplicitLinkedHashSet.
*
* @param iter We will add all the elements accessible through this iterator
* to the set.
*/
public ImplicitLinkedHashSet(Iterator<E> iter) {
clear(0);
while (iter.hasNext()) {
mustAdd(iter.next());
}
}
/**
* Removes all of the elements from this set.
*/
@Override
final public void clear() {
clear(elements.length);
}
/**
* Removes all of the elements from this set, and resets the set capacity
* based on the provided expected number of elements.
*/
final public void clear(int expectedNumElements) {
if (expectedNumElements == 0) {
// Optimize away object allocations for empty sets.
this.head = HeadElement.EMPTY;
this.elements = EMPTY_ELEMENTS;
this.size = 0;
} else {
this.head = new HeadElement(); this.head = new HeadElement();
// Avoid using even-sized capacities, to get better key distribution. this.elements = new Element[calculateCapacity(expectedNumElements)];
this.elements = new Element[(2 * capacity) + 1];
this.size = 0; this.size = 0;
} }
}
int numSlots() { // Visible for testing
final int numSlots() {
return elements.length; return elements.length;
} }
} }

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.
{
"apiKey": 25,
"type": "request",
"name": "AddOffsetsToTxnRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "TransactionalId", "type": "string", "versions": "0+",
"about": "The transactional id corresponding to the transaction."},
{ "name": "ProducerId", "type": "int64", "versions": "0+",
"about": "Current producer id in use by the transactional id." },
{ "name": "ProducerEpoch", "type": "int16", "versions": "0+",
"about": "Current epoch associated with the producer id." },
{ "name": "GroupId", "type": "string", "versions": "0+",
"about": "The unique group identifier." }
]
}

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.
{
"apiKey": 25,
"type": "response",
"name": "AddOffsetsToTxnResponse",
// Starting in version 1, on quota violation brokers send out responses before throttling.
"validVersions": "0-1",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
"about": "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+",
"about": "The response error code, or 0 if there was no error." }
]
}

View File

@ -0,0 +1,37 @@
// 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": 24,
"type": "request",
"name": "AddPartitionsToTxnRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "TransactionalId", "type": "string", "versions": "0+",
"about": "The transactional id corresponding to the transaction."},
{ "name": "ProducerId", "type": "int64", "versions": "0+",
"about": "Current producer id in use by the transactional id." },
{ "name": "ProducerEpoch", "type": "int16", "versions": "0+",
"about": "Current epoch associated with the producer id." },
{ "name": "Topics", "type": "[]AddPartitionsToTxnTopic", "versions": "0+",
"about": "The partitions to add to the transation.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The name of the topic." },
{ "name": "Partitions", "type": "[]int32", "versions": "0+",
"about": "The partition indexes to add to the transaction" }
]}
]
}

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.
{
"apiKey": 24,
"type": "response",
"name": "AddPartitionsToTxnResponse",
// Starting in version 1, on quota violation brokers send out responses before throttling.
"validVersions": "0-1",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
"about": "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": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0+",
"about": "The results for each topic.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The topic name." },
{ "name": "Results", "type": "[]AddPartitionsToTxnPartitionResult", "versions": "0+",
"about": "The results for each partition", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true,
"about": "The partition indexes." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The response error code."}
]}
]}
]
}

View File

@ -0,0 +1,40 @@
// 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": 33,
"type": "request",
"name": "AlterConfigsRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "Resources", "type": "[]AlterConfigsResource", "versions": "0+",
"about": "The updates for each resource.", "fields": [
{ "name": "ResourceType", "type": "int8", "versions": "0+", "mapKey": true,
"about": "The resource type." },
{ "name": "ResourceName", "type": "string", "versions": "0+", "mapKey": true,
"about": "The resource name." },
{ "name": "Configs", "type": "[]AlterableConfig", "versions": "0+",
"about": "The configurations.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The configuration key name." },
{ "name": "Value", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The value to set for the configuration key."}
]}
]},
{ "name": "ValidateOnly", "type": "bool", "versions": "0+",
"about": "True if we should validate the request, but not change the configurations."}
]
}

View File

@ -0,0 +1,37 @@
// 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": 33,
"type": "response",
"name": "AlterConfigsResponse",
// Starting in version 1, on quota violation brokers send out responses before throttling.
"validVersions": "0-1",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
"about": "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": "Resources", "type": "[]AlterConfigsResourceResponse", "versions": "0+",
"about": "The responses for each resource.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The resource error code." },
{ "name": "ErrorMessage", "type": "string", "nullableVersions": "0+", "versions": "0+",
"about": "The resource error message, or null if there was no error." },
{ "name": "ResourceType", "type": "int8", "versions": "0+",
"about": "The resource type." },
{ "name": "ResourceName", "type": "string", "versions": "0+",
"about": "The resource name." }
]}
]
}

View File

@ -0,0 +1,36 @@
// 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": 34,
"type": "request",
"name": "AlterReplicaLogDirsRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "Dirs", "type": "[]AlterReplicaLogDir", "versions": "0+",
"about": "The alterations to make for each directory.", "fields": [
{ "name": "Path", "type": "string", "versions": "0+", "mapKey": true,
"about": "The absolute directory path." },
{ "name": "Topics", "type": "[]AlterReplicaLogDirTopic", "versions": "0+",
"about": "The topics to add to the directory.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The topic name." },
{ "name": "Partitions", "type": "[]int32", "versions": "0+",
"about": "The partition indexes." }
]}
]}
]
}

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.
{
"apiKey": 34,
"type": "response",
"name": "AlterReplicaLogDirsResponse",
// Starting in version 1, on quota violation brokers send out responses before throttling.
"validVersions": "0-1",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
"about": "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": "Results", "type": "[]AlterReplicaLogDirTopicResult", "versions": "0+",
"about": "The results for each topic.", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0+",
"about": "The name of the topic." },
{ "name": "Partitions", "type": "[]AlterReplicaLogDirPartitionResult", "versions": "0+",
"about": "The results for each partition.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index."},
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." }
]}
]}
]
}

View File

@ -0,0 +1,24 @@
// 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": 18,
"type": "request",
"name": "ApiVersionsRequest",
// Versions 0 through 2 of ApiVersionsRequest are the same.
"validVersions": "0-2",
"fields": [
]
}

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.
{
"apiKey": 18,
"type": "response",
"name": "ApiVersionsResponse",
// Version 1 adds throttle time to the response.
// Starting in version 2, on quota violation, brokers send out responses before throttling.
"validVersions": "0-2",
"fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The top-level error code." },
{ "name": "ApiKeys", "type": "[]ApiVersionsResponseKey", "versions": "0+",
"about": "The APIs supported by the broker.", "fields": [
{ "name": "Index", "type": "int16", "versions": "0+", "mapKey": true,
"about": "The API index." },
{ "name": "MinVersion", "type": "int16", "versions": "0+",
"about": "The minimum supported version, inclusive." },
{ "name": "MaxVersion", "type": "int16", "versions": "0+",
"about": "The maximum supported version, inclusive." }
]},
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
"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." }
]
}

View File

@ -0,0 +1,34 @@
// 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": 7,
"type": "request",
"name": "ControlledShutdownRequest",
// Version 0 of ControlledShutdownRequest has a non-standard request header
// which does not include clientId. Version 1 and later use the standard
// request header.
//
// Version 1 is the same as version 0.
//
// Version 2 adds BrokerEpoch.
"validVersions": "0-2",
"fields": [
{ "name": "BrokerId", "type": "int32", "versions": "0+",
"about": "The id of the broker for which controlled shutdown has been requested." },
{ "name": "BrokerEpoch", "type": "int64", "versions": "2+", "default": "-1", "ignorable": true,
"about": "The broker epoch." }
]
}

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.
{
"apiKey": 7,
"type": "response",
"name": "ControlledShutdownResponse",
// Versions 1 and 2 are the same as version 0.
"validVersions": "0-2",
"fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The top-level error code." },
{ "name": "RemainingPartitions", "type": "[]RemainingPartition", "versions": "0+",
"about": "The partitions that the broker still leads.", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0+", "mapKey": true,
"about": "The name of the topic." },
{ "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true,
"about": "The index of the partition." }
]}
]
}

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.
{
"apiKey": 30,
"type": "request",
"name": "CreateAclsRequest",
// Version 1 adds resource pattern type.
"validVersions": "0-1",
"fields": [
{ "name": "Creations", "type": "[]CreatableAcl", "versions": "0+",
"about": "The ACLs that we want to create.", "fields": [
{ "name": "ResourceType", "type": "int8", "versions": "0+",
"about": "The type of the resource." },
{ "name": "ResourceName", "type": "string", "versions": "0+",
"about": "The resource name for the ACL." },
{ "name": "ResourcePatternType", "type": "int8", "versions": "1+", "default": "3",
"about": "The pattern type for the ACL." },
{ "name": "Principal", "type": "string", "versions": "0+",
"about": "The principal for the ACL." },
{ "name": "Host", "type": "string", "versions": "0+",
"about": "The host for the ACL." },
{ "name": "Operation", "type": "int8", "versions": "0+",
"about": "The operation type for the ACL (read, write, etc.)." },
{ "name": "PermissionType", "type": "int8", "versions": "0+",
"about": "The permission type for the ACL (allow, deny, etc.)." }
]}
]
}

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.
{
"apiKey": 30,
"type": "response",
"name": "CreateAclsResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"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": "Results", "type": "[]CreatableAclResult", "versions": "0+",
"about": "The results for each ACL creation.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The result error, or zero if there was no error." },
{ "name": "ErrorMessage", "type": "string", "nullableVersions": "0+", "versions": "0+",
"about": "The result message, or null if there was no error." }
]}
]
}

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.
{
"apiKey": 38,
"type": "request",
"name": "CreateDelegationTokenRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "Renewers", "type": "[]CreatableRenewers", "versions": "0+",
"about": "A list of those who are allowed to renew this token before it expires.", "fields": [
{ "name": "PrincipalType", "type": "string", "versions": "0+",
"about": "The type of the Kafka principal." },
{ "name": "PrincipalName", "type": "string", "versions": "0+",
"about": "The name of the Kafka principal." }
]},
{ "name": "MaxLifetimeMs", "type": "int64", "versions": "0+",
"about": "The maximum lifetime of the token in milliseconds, or -1 to use the server side default." }
]
}

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.
{
"apiKey": 38,
"type": "response",
"name": "CreateDelegationTokenResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The top-level error, or zero if there was no error."},
{ "name": "PrincipalType", "type": "string", "versions": "0+",
"about": "The principal type of the token owner." },
{ "name": "PrincipalName", "type": "string", "versions": "0+",
"about": "The name of the token owner." },
{ "name": "IssueTimestampMs", "type": "int64", "versions": "0+",
"about": "When this token was generated." },
{ "name": "ExpiryTimestampMs", "type": "int64", "versions": "0+",
"about": "When this token expires." },
{ "name": "MaxTimestampMs", "type": "int64", "versions": "0+",
"about": "The maximum lifetime of this token." },
{ "name": "TokenId", "type": "string", "versions": "0+",
"about": "The token UUID." },
{ "name": "Hmac", "type": "bytes", "versions": "0+",
"about": "HMAC of the delegation token." },
{ "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." }
]
}

View File

@ -0,0 +1,40 @@
// 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": 37,
"type": "request",
"name": "CreatePartitionsRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "Topics", "type": "[]CreatePartitionsTopic", "versions": "0+",
"about": "Each topic that we want to create new partitions inside.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Count", "type": "int32", "versions": "0+",
"about": "The new partition count." },
{ "name": "Assignments", "type": "[]CreatePartitionsAssignment", "versions": "0+", "nullableVersions": "0+",
"about": "The new partition assignments.", "fields": [
{ "name": "BrokerIds", "type": "[]int32", "versions": "0+",
"about": "The assigned broker IDs." }
]}
]},
{ "name": "TimeoutMs", "type": "int32", "versions": "0+",
"about": "The time in ms to wait for the partitions to be created." },
{ "name": "ValidateOnly", "type": "bool", "versions": "0+",
"about": "If true, then validate the request, but don't actually increase the number of partitions." }
]
}

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.
{
"apiKey": 37,
"type": "response",
"name": "CreatePartitionsResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"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": "Results", "type": "[]CreatePartitionsTopicResult", "versions": "0+",
"about": "The partition creation results for each topic.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The result error, or zero if there was no error."},
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The result message, or null if there was no error."}
]}
]
}

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.
{
"apiKey": 19,
"type": "request",
"name": "CreateTopicsRequest",
// Version 1 adds validateOnly.
"validVersions": "0-3",
"fields": [
{ "name": "Topics", "type": "[]CreatableTopic", "versions": "0+",
"about": "The topics to create.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "NumPartitions", "type": "int32", "versions": "0+",
"about": "The number of partitions to create in the topic, or -1 if we are specifying a manual partition assignment." },
{ "name": "ReplicationFactor", "type": "int16", "versions": "0+",
"about": "The number of replicas to create for each partition in the topic, or -1 if we are specifying a manual partition assignment." },
{ "name": "Assignments", "type": "[]CreatableReplicaAssignment", "versions": "0+",
"about": "The manual partition assignment, or the empty array if we are using automatic assignment.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true,
"about": "The partition index." },
{ "name": "BrokerIds", "type": "[]int32", "versions": "0+",
"about": "The brokers to place the partition on." }
]},
{ "name": "Configs", "type": "[]CreateableTopicConfig", "versions": "0+",
"about": "The custom topic configurations to set.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+" , "mapKey": true,
"about": "The configuration name." },
{ "name": "Value", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The configuration value." }
]}
]},
{ "name": "timeoutMs", "type": "int32", "versions": "0+",
"about": "How long to wait in milliseconds before timing out the request." },
{ "name": "validateOnly", "type": "bool", "versions": "1+", "default": "false", "ignorable": false,
"about": "If true, check that the topics can be created as specified, but don't create anything." }
]
}

View File

@ -0,0 +1,37 @@
// 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": 19,
"type": "response",
"name": "CreateTopicsResponse",
// Version 1 adds a per-topic error message string.
// Version 2 adds the throttle time.
// Starting in version 3, on quota violation, brokers send out responses before throttling.
"validVersions": "0-3",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
"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": "Topics", "type": "[]CreatableTopicResult", "versions": "0+",
"about": "Results for each topic we tried to create.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The topic name." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." },
{ "name": "ErrorMessage", "type": "string", "versions": "1+", "nullableVersions": "0+", "ignorable": true,
"about": "The error message, or null if there was no error." }
]}
]
}

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.
{
"apiKey": 31,
"type": "request",
"name": "DeleteAclsRequest",
// Version 1 adds the pattern type.
"validVersions": "0-1",
"fields": [
{ "name": "Filters", "type": "[]DeleteAclsFilter", "versions": "0+",
"about": "The filters to use when deleting ACLs.", "fields": [
{ "name": "ResourceTypeFilter", "type": "int8", "versions": "0+",
"about": "The resource type." },
{ "name": "ResourceNameFilter", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The resource name." },
{ "name": "PatternTypeFilter", "type": "int8", "versions": "1+", "default": "3", "ignorable": false,
"about": "The pattern type." },
{ "name": "PrincipalFilter", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The principal filter, or null to accept all principals." },
{ "name": "HostFilter", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The host filter, or null to accept all hosts." },
{ "name": "Operation", "type": "int8", "versions": "0+",
"about": "The ACL operation." },
{ "name": "PermissionType", "type": "int8", "versions": "0+",
"about": "The permission type." }
]}
]
}

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.
{
"apiKey": 31,
"type": "response",
"name": "DeleteAclsResponse",
// Version 1 adds the resource pattern type.
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"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": "FilterResults", "type": "[]DeleteAclsFilterResult", "versions": "0+",
"about": "The results for each filter.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if the filter succeeded." },
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The error message, or null if the filter succeeded." },
{ "name": "MatchingAcls", "type": "[]DeleteAclsMatchingAcl", "versions": "0+",
"about": "The ACLs which matched this filter.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The deletion error code, or 0 if the deletion succeeded." },
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The deletion error message, or null if the deletion succeeded." },
{ "name": "ResourceType", "type": "int8", "versions": "0+",
"about": "The ACL resource type." },
{ "name": "ResourceName", "type": "string", "versions": "0+",
"about": "The ACL resource name." },
{ "name": "PatternType", "type": "int8", "versions": "1+", "default": "3", "ignorable": false,
"about": "The ACL resource pattern type." },
{ "name": "Principal", "type": "string", "versions": "0+",
"about": "The ACL principal." },
{ "name": "Host", "type": "string", "versions": "0+",
"about": "The ACL host." },
{ "name": "Operation", "type": "int8", "versions": "0+",
"about": "The ACL operation." },
{ "name": "PermissionType", "type": "int8", "versions": "0+",
"about": "The ACL permission type." }
]}
]}
]
}

View File

@ -0,0 +1,26 @@
// 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": 42,
"type": "request",
"name": "DeleteGroupsRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "GroupsNames", "type": "[]string", "versions": "0+",
"about": "The group names to delete." }
]
}

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.
{
"apiKey": 42,
"type": "response",
"name": "DeleteGroupsResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"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": "Results", "type": "[]DeletableGroupResult", "versions": "0+",
"about": "The deletion results", "fields": [
{ "name": "GroupId", "type": "string", "versions": "0+", "mapKey": true,
"about": "The group id" },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The deletion error, or 0 if the deletion succeeded." }
]}
]
}

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.
{
"apiKey": 21,
"type": "request",
"name": "DeleteRecordsRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "Topics", "type": "[]DeleteRecordsTopic", "versions": "0+",
"about": "Each topic that we want to delete records from.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]DeleteRecordsPartition", "versions": "0+",
"about": "Each partition that we want to delete records from.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "Offset", "type": "int64", "versions": "0+",
"about": "The deletion offset." }
]}
]},
{ "name": "TimeoutMs", "type": "int32", "versions": "0+",
"about": "How long to wait for the deletion to complete, in milliseconds." }
]
}

View File

@ -0,0 +1,40 @@
// 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": 21,
"type": "response",
"name": "DeleteRecordsResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"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": "Topics", "type": "[]DeleteRecordsTopicResult", "versions": "0+",
"about": "Each topic that we wanted to delete records from.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]DeleteRecordsPartitionResult", "versions": "0+",
"about": "Each partition that we wanted to delete records from.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "LowWatermark", "type": "int64", "versions": "0+",
"about": "The partition low water mark." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The deletion error code, or 0 if the deletion succeeded." }
]}
]}
]
}

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.
{
"apiKey": 20,
"type": "request",
"name": "DeleteTopicsRequest",
// Versions 0, 1, 2, and 3 are the same.
"validVersions": "0-3",
"fields": [
{ "name": "TopicNames", "type": "[]string", "versions": "0+",
"about": "The names of the topics to delete" },
{ "name": "TimeoutMs", "type": "int32", "versions": "0+",
"about": "The length of time in milliseconds to wait for the deletions to complete." }
]
}

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.
{
"apiKey": 20,
"type": "response",
"name": "DeleteTopicsResponse",
// Version 1 adds the throttle time.
// Starting in version 2, on quota violation, brokers send out responses before throttling.
// Starting in version 3, a TOPIC_DELETION_DISABLED error code may be returned.
"validVersions": "0-3",
"fields": [
{ "name": "throttleTimeMs", "type": "int32", "versions": "1+",
"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": "Responses", "type": "[]DeletableTopicResult", "versions": "0+",
"about": "The results for each topic.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name" },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The deletion error, or 0 if the deletion succeeded." }
]}
]
}

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.
{
"apiKey": 29,
"type": "request",
"name": "DescribeAclsRequest",
// Version 1 adds resource pattern type.
"validVersions": "0-1",
"fields": [
{ "name": "ResourceType", "type": "int8", "versions": "0+",
"about": "The resource type." },
{ "name": "ResourceNameFilter", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The resource name, or null to match any resource name." },
{ "name": "ResourcePatternType", "type": "int8", "versions": "1+", "default": "3", "ignorable": false,
"about": "The resource pattern to match." },
{ "name": "PrincipalFilter", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The principal to match, or null to match any principal." },
{ "name": "HostFilter", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The host to match, or null to match any host." },
{ "name": "Operation", "type": "int8", "versions": "0+",
"about": "The operation to match." },
{ "name": "PermissionType", "type": "int8", "versions": "0+",
"about": "The permission type to match." }
]
}

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.
{
"apiKey": 29,
"type": "response",
"name": "DescribeAclsResponse",
// Version 1 adds PatternType.
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"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+",
"about": "The error code, or 0 if there was no error." },
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The error message, or null if there was no error." },
{ "name": "Resources", "type": "[]DescribeAclsResource", "versions": "0+",
"about": "Each Resource that is referenced in an ACL.", "fields": [
{ "name": "Type", "type": "int8", "versions": "0+",
"about": "The resource type." },
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The resource name." },
{ "name": "PatternType", "type": "int8", "versions": "1+", "default": "3", "ignorable": false,
"about": "The resource pattern type." },
{ "name": "Acls", "type": "[]AclDescription", "versions": "0+",
"about": "The ACLs.", "fields": [
{ "name": "Principal", "type": "string", "versions": "0+",
"about": "The ACL principal." },
{ "name": "Host", "type": "string", "versions": "0+",
"about": "The ACL host." },
{ "name": "Operation", "type": "int8", "versions": "0+",
"about": "The ACL operation." },
{ "name": "PermissionType", "type": "int8", "versions": "0+",
"about": "The ACL permission type." }
]}
]}
]
}

View File

@ -0,0 +1,36 @@
// 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": 32,
"type": "request",
"name": "DescribeConfigsRequest",
// Version 1 adds IncludeSynoyms.
// Version 2 is the same as version 1.
"validVersions": "0-2",
"fields": [
{ "name": "Resources", "type": "[]DescribeConfigsResource", "versions": "0+",
"about": "The resources whose configurations we want to describe.", "fields": [
{ "name": "ResourceType", "type": "int8", "versions": "0+",
"about": "The resource type." },
{ "name": "ResourceName", "type": "string", "versions": "0+",
"about": "The resource name." },
{ "name": "ConfigurationKeys", "type": "[]string", "versions": "0+", "nullableVersions": "0+",
"about": "The configuration keys to list, or null to list all configuration keys." }
]},
{ "name": "IncludeSynoyms", "type": "bool", "versions": "1+", "default": "false", "ignorable": false,
"about": "True if we should include all synonyms." }
]
}

View File

@ -0,0 +1,65 @@
// 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": 32,
"type": "response",
"name": "DescribeConfigsResponse",
// Version 1 adds ConfigSource and the synonyms.
// Starting in version 2, on quota violation, brokers send out responses before throttling.
"validVersions": "0-2",
"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": "Results", "type": "[]DescribeConfigsResult", "versions": "0+",
"about": "The results for each resource.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if we were able to successfully describe the configurations." },
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The error message, or null if we were able to successfully describe the configurations." },
{ "name": "ResourceType", "type": "int8", "versions": "0+",
"about": "The resource type." },
{ "name": "ResourceName", "type": "string", "versions": "0+",
"about": "The resource name." },
{ "name": "Configs", "type": "[]DescribeConfigsResourceResult", "versions": "0+",
"about": "Each listed configuration.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The configuration name." },
{ "name": "Value", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The configuration value." },
{ "name": "ReadOnly", "type": "bool", "versions": "0+",
"about": "True if the configuration is read-only." },
{ "name": "IsDefault", "type": "bool", "versions": "0",
"about": "True if the configuration is not set." },
// Note: the v0 default for this field that shouldd be exposed to callers is
// context-dependent. For example, if the resource is a broker, this should default to 4.
// -1 is just a placeholder value.
{ "name": "ConfigSource", "type": "int8", "versions": "1+", "default": "-1", "ignorable": true,
"about": "The configuration source." },
{ "name": "IsSensitive", "type": "bool", "versions": "0+",
"about": "True if this configuration is sensitive." },
{ "name": "Synonyms", "type": "[]DescribeConfigsSynonym", "versions": "1+", "ignorable": true,
"about": "The synonyms for this configuration key.", "fields": [
{ "name": "Name", "type": "string", "versions": "1+",
"about": "The synonym name." },
{ "name": "Value", "type": "string", "versions": "1+", "nullableVersions": "0+",
"about": "The synonym value." },
{ "name": "Source", "type": "int8", "versions": "1+",
"about": "The synonym source." }
]}
]}
]}
]
}

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.
{
"apiKey": 41,
"type": "request",
"name": "DescribeDelegationTokenRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "Owners", "type": "[]DescribeDelegationTokenOwner", "versions": "0+", "nullableVersions": "0+",
"about": "Each owner that we want to describe delegation tokens for, or null to describe all tokens.", "fields": [
{ "name": "PrincipalType", "type": "string", "versions": "0+",
"about": "The owner principal type." },
{ "name": "PrincipalName", "type": "string", "versions": "0+",
"about": "The owner principal name." }
]}
]
}

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.
{
"apiKey": 41,
"type": "response",
"name": "DescribeDelegationTokenResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." },
{ "name": "Tokens", "type": "[]DescribedDelegationToken", "versions": "0+",
"about": "The tokens.", "fields": [
{ "name": "PrincipalType", "type": "string", "versions": "0+",
"about": "The token principal type." },
{ "name": "PrincipalName", "type": "string", "versions": "0+",
"about": "The token principal name." },
{ "name": "IssueTimestamp", "type": "int64", "versions": "0+",
"about": "The token issue timestamp in milliseconds." },
{ "name": "ExpiryTimestamp", "type": "int64", "versions": "0+",
"about": "The token expiry timestamp in milliseconds." },
{ "name": "MaxTimestamp", "type": "int64", "versions": "0+",
"about": "The token maximum timestamp length in milliseconds." },
{ "name": "TokenId", "type": "string", "versions": "0+",
"about": "The token ID." },
{ "name": "Hmac", "type": "bytes", "versions": "0+",
"about": "The token HMAC." },
{ "name": "Renewers", "type": "[]DescribedDelegationTokenRenewer", "versions": "0+",
"about": "Those who are able to renew this token before it expires.", "fields": [
{ "name": "PrincipalType", "type": "string", "versions": "0+",
"about": "The renewer principal type" },
{ "name": "PrincipalName", "type": "string", "versions": "0+",
"about": "The renewer principal name" }
]}
]},
{ "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." }
]
}

View File

@ -0,0 +1,26 @@
// 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": 15,
"type": "request",
"name": "DescribeGroupsRequest",
// Versions 1 and 2 are the same as version 0.
"validVersions": "0-2",
"fields": [
{ "name": "Groups", "type": "[]string", "versions": "0+",
"about": "The names of the groups to describe" }
]
}

View File

@ -0,0 +1,57 @@
// 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": 15,
"type": "response",
"name": "DescribeGroupsResponse",
// Version 1 added throttle time.
// Starting in version 2, on quota violation, brokers send out responses before throttling.
"validVersions": "0-2",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
"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": "Groups", "type": "[]DescribedGroup", "versions": "0+",
"about": "Each described group.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The describe error, or 0 if there was no error." },
{ "name": "GroupId", "type": "string", "versions": "0+",
"about": "The group ID string." },
{ "name": "GroupState", "type": "string", "versions": "0+",
"about": "The group state string, or the empty string." },
{ "name": "ProtocolType", "type": "string", "versions": "0+",
"about": "The group protocol type, or the empty string." },
// ProtocolData is currently only filled in if the group state is in the Stable state.
{ "name": "ProtocolData", "type": "string", "versions": "0+",
"about": "The group protocol data, or the empty string." },
// N.B. If the group is in the Dead state, the members array will always be empty.
{ "name": "Members", "type": "[]DescribedGroupMember", "versions": "0+",
"about": "The group members.", "fields": [
{ "name": "MemberId", "type": "string", "versions": "0+",
"about": "The member ID assigned by the group coordinator." },
{ "name": "ClientId", "type": "string", "versions": "0+",
"about": "The client ID used in the member's latest join group request." },
{ "name": "ClientHost", "type": "string", "versions": "0+",
"about": "The client host." },
// This is currently only provided if the group is in the Stable state.
{ "name": "MemberMetadata", "type": "bytes", "versions": "0+",
"about": "The metadata corresponding to the current group protocol in use." },
// This is currently only provided if the group is in the Stable state.
{ "name": "MemberAssignment", "type": "bytes", "versions": "0+",
"about": "The current assignment provided by the group leader." }
]}
]}
]
}

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.
{
"apiKey": 35,
"type": "request",
"name": "DescribeLogDirsRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "Topics", "type": "[]DescribableLogDirTopic", "versions": "0+", "nullableVersions": "0+",
"about": "Each topic that we want to describe log directories for, or null for all topics.", "fields": [
{ "name": "Topic", "type": "string", "versions": "0+",
"about": "The topic name" },
{ "name": "PartitionIndex", "type": "[]int32", "versions": "0+",
"about": "The partition indxes." }
]}
]
}

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.
{
"apiKey": 35,
"type": "response",
"name": "DescribeLogDirsResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"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": "Results", "type": "[]DescribeLogDirsResult", "versions": "0+",
"about": "The log directories.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." },
{ "name": "LogDir", "type": "string", "versions": "0+",
"about": "The absolute log directory path." },
{ "name": "Topics", "type": "[]DescribeLogDirsTopic", "versions": "0+",
"about": "Each topic.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]DescribeLogDirsPartition", "versions": "0+", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "PartitionSize", "type": "int64", "versions": "0+",
"about": "The size of the log segments in this partition in bytes." },
{ "name": "OffsetLag", "type": "int64", "versions": "0+",
"about": "The lag of the log's LEO w.r.t. partition's HW (if it is the current log for the partition) or current replica's LEO (if it is the future log for the partition)" },
{ "name": "IsFutureKey", "type": "bool", "versions": "0+",
"about": "True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future." }
]}
]}
]}
]
}

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.
{
"apiKey": 26,
"type": "request",
"name": "EndTxnRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "TransactionalId", "type": "string", "versions": "0+",
"about": "The ID of the transaction to end." },
{ "name": "ProducerId", "type": "int64", "versions": "0+",
"about": "The producer ID." },
{ "name": "ProducerEpoch", "type": "int16", "versions": "0+",
"about": "The current epoch associated with the producer." },
{ "name": "Committed", "type": "bool", "versions": "0+",
"about": "True if the transaction was committed, false if it was aborted." }
]
}

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.
{
"apiKey": 26,
"type": "response",
"name": "EndTxnResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"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+",
"about": "The error code, or 0 if there was no error." }
]
}

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.
{
"apiKey": 40,
"type": "request",
"name": "ExpireDelegationTokenRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "Hmac", "type": "bytes", "versions": "0+",
"about": "The HMAC of the delegation token to be expired." },
{ "name": "ExpiryTimePeriodMs", "type": "int64", "versions": "0+",
"about": "The expiry time period in milliseconds." }
]
}

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.
{
"apiKey": 40,
"type": "response",
"name": "ExpireDelegationTokenResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." },
{ "name": "ExpiryTimestampMs", "type": "int64", "versions": "0+",
"about": "The timestamp in milliseconds at which this token expires." },
{ "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." }
]
}

View File

@ -0,0 +1,89 @@
// 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": 1,
"type": "request",
"name": "FetchRequest",
//
// Version 1 is the same as version 0.
//
// Starting in Version 2, the requestor must be able to handle Kafka Log
// Message format version 1.
//
// Version 3 adds MaxBytes. Starting in version 3, the partition ordering in
// the request is now relevant. Partitions will be processed in the order
// they appear in the request.
//
// Version 4 adds IsolationLevel. Starting in version 4, the reqestor must be
// able to handle Kafka log message format version 2.
//
// Version 5 adds LogStartOffset to indicate the earliest available offset of
// partition data that can be consumed.
//
// Version 6 is the same as version 5.
//
// Version 7 adds incremental fetch request support.
//
// Version 8 is the same as version 7.
//
// Version 9 adds CurrentLeaderEpoch, as described in KIP-320.
//
// Version 10 indicates that we can use the ZStd compression algorithm, as
// described in KIP-110.
//
"validVersions": "0-10",
"fields": [
{ "name": "ReplicaId", "type": "int32", "versions": "0+",
"about": "The broker ID of the follower, of -1 if this request is from a consumer." },
{ "name": "MaxWait", "type": "int32", "versions": "0+",
"about": "The maximum time in milliseconds to wait for the response." },
{ "name": "MinBytes", "type": "int32", "versions": "0+",
"about": "The minimum bytes to accumulate in the response." },
{ "name": "MaxBytes", "type": "int32", "versions": "3+", "default": "0x7fffffff", "ignorable": true,
"about": "The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored." },
{ "name": "IsolationLevel", "type": "int8", "versions": "4+", "default": "0", "ignorable": false,
"about": "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records" },
{ "name": "SessionId", "type": "int32", "versions": "7+", "default": "0", "ignorable": false,
"about": "The fetch session ID." },
{ "name": "Epoch", "type": "int32", "versions": "7+", "default": "-1", "ignorable": false,
"about": "The fetch session ID." },
{ "name": "Topics", "type": "[]FetchableTopic", "versions": "0+",
"about": "The topics to fetch.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The name of the topic to fetch." },
{ "name": "FetchPartitions", "type": "[]FetchPartition", "versions": "0+",
"about": "The partitions to fetch.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "CurrentLeaderEpoch", "type": "int32", "versions": "9+", "default": "-1", "ignorable": true,
"about": "The current leader epoch of the partition." },
{ "name": "FetchOffset", "type": "int64", "versions": "0+",
"about": "The message offset." },
{ "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": false,
"about": "The earliest available offset of the follower replica. The field is only used when the request is sent by the follower."},
{ "name": "MaxBytes", "type": "int32", "versions": "0+",
"about": "The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored." }
]}
]},
{ "name": "Forgotten", "type": "[]ForgottenTopic", "versions": "7+", "ignorable": false,
"about": "In an incremental fetch request, the partitions to remove.", "fields": [
{ "name": "Name", "type": "string", "versions": "7+",
"about": "The partition name." },
{ "name": "ForgottenPartitionIndexes", "type": "[]int32", "versions": "7+",
"about": "The partitions indexes to forget." }
]}
]
}

View File

@ -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.
{
"apiKey": 1,
"type": "response",
"name": "FetchResponse",
//
// Version 1 adds throttle time.
//
// Version 2 and 3 are the same as version 1.
//
// Version 4 adds features for transactional consumption.
//
// Version 5 adds LogStartOffset to indicate the earliest available offset of
// partition data that can be consumed.
//
// Starting in version 6, we may return KAFKA_STORAGE_ERROR as an error code.
//
// Version 7 adds incremental fetch request support.
//
// Starting in version 8, on quota violation, brokers send out responses before throttling.
//
// Version 9 is the same as version 8.
//
// Version 10 indicates that the response data can use the ZStd compression
// algorithm, as described in KIP-110.
//
"validVersions": "0-10",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
"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": "7+", "ignorable": false,
"about": "The top level response error code." },
{ "name": "SessionId", "type": "int32", "versions": "7+", "default": "0", "ignorable": false,
"about": "The fetch session ID, or 0 if this is not part of a fetch session." },
{ "name": "Topics", "type": "[]FetchableTopicResponse", "versions": "0+",
"about": "The response topics.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]FetchablePartitionResponse", "versions": "0+",
"about": "The topic partitions.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partiiton index." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no fetch error." },
{ "name": "HighWatermark", "type": "int64", "versions": "0+",
"about": "The current high water mark." },
{ "name": "LastStableOffset", "type": "int64", "versions": "4+", "default": "-1", "ignorable": true,
"about": "The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)" },
{ "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true,
"about": "The current log start offset." },
{ "name": "Aborted", "type": "[]AbortedTransaction", "versions": "4+", "nullableVersions": "4+", "ignorable": false,
"about": "The aborted transactions.", "fields": [
{ "name": "ProducerId", "type": "int64", "versions": "4+",
"about": "The producer id associated with the aborted transaction." },
{ "name": "FirstOffset", "type": "int64", "versions": "4+",
"about": "The first offset in the aborted transaction." }
]},
{ "name": "Records", "type": "bytes", "versions": "0+", "nullableVersions": "0+",
"about": "The record data." }
]}
]}
]
}

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.
{
"apiKey": 10,
"type": "request",
"name": "FindCoordinatorRequest",
// Version 1 adds KeyType.
// Version 2 is the same as version 1.
"validVersions": "0-2",
"fields": [
{ "name": "Key", "type": "string", "versions": "0+",
"about": "The coordinator key." },
{ "name": "KeyType", "type": "int8", "versions": "1+", "default": "0", "ignorable": false,
"about": "The coordinator key type. (Group, transaction, etc.)" }
]
}

View File

@ -0,0 +1,37 @@
// 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": 10,
"type": "response",
"name": "FindCoordinatorResponse",
// Version 1 adds throttle time and error messages.
// Starting in version 2, on quota violation, brokers send out responses before throttling.
"validVersions": "0-2",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
"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+",
"about": "The error code, or 0 if there was no error." },
{ "name": "ErrorMessage", "type": "string", "versions": "1+", "nullableVersions": "1+", "ignorable": true,
"about": "The error message, or null if there was no error." },
{ "name": "NodeId", "type": "int32", "versions": "0+",
"about": "The node id." },
{ "name": "Host", "type": "string", "versions": "0+",
"about": "The host name." },
{ "name": "Port", "type": "int32", "versions": "0+",
"about": "The port." }
]
}

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.
{
"apiKey": 12,
"type": "request",
"name": "HeartbeatRequest",
// Version 1 and version 2 are the same as version 0.
"validVersions": "0-2",
"fields": [
{ "name": "GroupId", "type": "string", "versions": "0+",
"about": "The group id." },
{ "name": "Generationid", "type": "int32", "versions": "0+",
"about": "The generation of the group." },
{ "name": "MemberId", "type": "string", "versions": "0+",
"about": "The member ID." }
]
}

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.
{
"apiKey": 12,
"type": "response",
"name": "HeartbeatResponse",
// Version 1 adds throttle time.
// Starting in version 2, on quota violation, brokers send out responses before throttling.
"validVersions": "0-2",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
"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+",
"about": "The error code, or 0 if there was no error." }
]
}

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.
{
"apiKey": 22,
"type": "request",
"name": "InitProducerIdRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "TransactionalId", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The transactional id, or null if the producer is not transactional." },
{ "name": "TransactionTimeoutMs", "type": "int32", "versions": "0+",
"about": "The time in ms to wait for before aborting idle transactions sent by this producer." }
]
}

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.
{
"apiKey": 22,
"type": "response",
"name": "InitProducerIdResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling.
"validVersions": "0-1",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true,
"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+",
"about": "The error code, or 0 if there was no error." },
{ "name": "ProducerId", "type": "int64", "versions": "0+",
"about": "The current producer id." },
{ "name": "ProducerEpoch", "type": "int16", "versions": "0+",
"about": "The current epoch associated with the producer id." }
]
}

View File

@ -0,0 +1,44 @@
// 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": 11,
"type": "request",
"name": "JoinGroupRequest",
// Version 1 adds RebalanceTimeoutMs.
// Version 2 and 3 are the same as version 1.
"validVersions": "0-3",
"fields": [
{ "name": "GroupId", "type": "string", "versions": "0+",
"about": "The group identifier." },
{ "name": "SessionTimeoutMs", "type": "int32", "versions": "0+",
"about": "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds." },
// Note: if RebalanceTimeoutMs is not present, SessionTimeoutMs should be
// used instead. The default of -1 here is just intended as a placeholder.
{ "name": "RebalanceTimeoutMs", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true,
"about": "The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group." },
{ "name": "MemberId", "type": "string", "versions": "0+",
"about": "The member id assigned by the group coordinator." },
{ "name": "ProtocolType", "type": "string", "versions": "0+",
"about": "The unique name the for class of protocols implemented by the group we want to join." },
{ "name": "Protocols", "type": "[]JoinGroupRequestProtocol", "versions": "0+",
"about": "The list of protocols that the member supports.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The protocol name." },
{ "name": "Metadata", "type": "bytes", "versions": "0+",
"about": "The protocol metadata." }
]}
]
}

View File

@ -0,0 +1,44 @@
// 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": 11,
"type": "response",
"name": "JoinGroupResponse",
// Version 1 is the same as version 0.
// Version 2 adds throttle time.
// Starting in version 3, on quota violation, brokers send out responses before throttling.
"validVersions": "0-3",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
"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+",
"about": "The error code, or 0 if there was no error." },
{ "name": "GenerationId", "type": "int32", "versions": "0+",
"about": "The generation ID of the group." },
{ "name": "ProtocolName", "type": "string", "versions": "0+",
"about": "The group protocol selected by the coordinator." },
{ "name": "Leader", "type": "string", "versions": "0+",
"about": "The leader of the group." },
{ "name": "MemberId", "type": "string", "versions": "0+",
"about": "The member ID assigned by the group coordinator." },
{ "name": "Members", "type": "[]JoinGroupResponseMember", "versions": "0+", "fields": [
{ "name": "MemberId", "type": "string", "versions": "0+",
"about": "The group member ID." },
{ "name": "Metadata", "type": "bytes", "versions": "0+",
"about": "The group member metadata." }
]}
]
}

View File

@ -0,0 +1,86 @@
// 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": 4,
"type": "request",
"name": "LeaderAndIsrRequest",
// Version 1 adds IsNew.
//
// Version 2 adds broker epoch and reorganizes the partitions by topic.
"validVersions": "0-2",
"fields": [
{ "name": "ControllerId", "type": "int32", "versions": "0+",
"about": "The current controller ID." },
{ "name": "ControllerEpoch", "type": "int32", "versions": "0+",
"about": "The current controller epoch." },
{ "name": "BrokerEpoch", "type": "int64", "versions": "2+", "ignorable": true, "default": "-1",
"about": "The current broker epoch." },
{ "name": "TopicStates", "type": "[]LeaderAndIsrRequestTopicState", "versions": "2+",
"about": "Each topic.", "fields": [
{ "name": "Name", "type": "string", "versions": "2+",
"about": "The topic name." },
{ "name": "PartitionStates", "type": "[]LeaderAndIsrRequestPartitionState", "versions": "0+",
"about": "The state of each partition", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "ControllerEpoch", "type": "int32", "versions": "0+",
"about": "The controller epoch." },
{ "name": "LeaderKey", "type": "int32", "versions": "0+",
"about": "The broker ID of the leader." },
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+",
"about": "The leader epoch." },
{ "name": "IsrReplicas", "type": "[]int32", "versions": "0+",
"about": "The in-sync replica IDs." },
{ "name": "ZkVersion", "type": "int32", "versions": "0+",
"about": "The ZooKeeper version." },
{ "name": "Replicas", "type": "[]int32", "versions": "0+",
"about": "The replica IDs." },
{ "name": "IsNew", "type": "bool", "versions": "1+", "default": "false", "ignorable": true,
"about": "Whether the replica should have existed on the broker or not." }
]}
]},
{ "name": "PartitionStatesV0", "type": "[]LeaderAndIsrRequestPartitionStateV0", "versions": "0-1",
"about": "The state of each partition", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0-1",
"about": "The topic name." },
{ "name": "PartitionIndex", "type": "int32", "versions": "0-1",
"about": "The partition index." },
{ "name": "ControllerEpoch", "type": "int32", "versions": "0-1",
"about": "The controller epoch." },
{ "name": "LeaderKey", "type": "int32", "versions": "0-1",
"about": "The broker ID of the leader." },
{ "name": "LeaderEpoch", "type": "int32", "versions": "0-1",
"about": "The leader epoch." },
{ "name": "IsrReplicas", "type": "[]int32", "versions": "0-1",
"about": "The in-sync replica IDs." },
{ "name": "ZkVersion", "type": "int32", "versions": "0-1",
"about": "The ZooKeeper version." },
{ "name": "Replicas", "type": "[]int32", "versions": "0-1",
"about": "The replica IDs." },
{ "name": "IsNew", "type": "bool", "versions": "1", "default": "false", "ignorable": true,
"about": "Whether the replica should have existed on the broker or not." }
]},
{ "name": "LiveLeaders", "type": "[]LeaderAndIsrLiveLeader", "versions": "0+",
"about": "The current live leaders.", "fields": [
{ "name": "BrokerId", "type": "int32", "versions": "0+",
"about": "The leader's broker ID." },
{ "name": "HostName", "type": "string", "versions": "0+",
"about": "The leader's hostname." },
{ "name": "Port", "type": "int32", "versions": "0+",
"about": "The leader's port." }
]}
]
}

View File

@ -0,0 +1,37 @@
// 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": 4,
"type": "response",
"name": "LeaderAndIsrResponse",
// Version 1 adds KAFKA_STORAGE_ERROR as a valid error code.
//
// Version 2 is the same as version 1.
"validVersions": "0-2",
"fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." },
{ "name": "Partitions", "type": "[]LeaderAndIsrResponsePartition", "versions": "0+",
"about": "Each partition.", "fields": [
{ "name": "TopicName", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The partition error code, or 0 if there was no error." }
]}
]
}

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.
{
"apiKey": 13,
"type": "request",
"name": "LeaveGroupRequest",
// Version 1 and 2 are the same as version 0.
"validVersions": "0-2",
"fields": [
{ "name": "GroupId", "type": "string", "versions": "0+",
"about": "The ID of the group to leave." },
{ "name": "MemberId", "type": "string", "versions": "0+",
"about": "The member ID to remove from the group." }
]
}

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.
{
"apiKey": 13,
"type": "response",
"name": "LeaveGroupResponse",
// Version 1 adds the throttle time.
// Starting in version 2, on quota violation, brokers send out responses before throttling.
"validVersions": "0-2",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
"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+",
"about": "The error code, or 0 if there was no error." }
]
}

View File

@ -0,0 +1,24 @@
// 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": 16,
"type": "request",
"name": "ListGroupsRequest",
// Version 1 and 2 are the same as version 0.
"validVersions": "0-2",
"fields": [
]
}

View File

@ -0,0 +1,36 @@
// 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": 16,
"type": "response",
"name": "ListGroupsResponse",
// Version 1 adds the throttle time.
// Starting in version 2, on quota violation, brokers send out responses before throttling.
"validVersions": "0-2",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
"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+",
"about": "The error code, or 0 if there was no error." },
{ "name": "Groups", "type": "[]ListedGroup", "versions": "0+",
"about": "Each group in the response.", "fields": [
{ "name": "GroupId", "type": "string", "versions": "0+",
"about": "The group ID." },
{ "name": "ProtocolType", "type": "string", "versions": "0+",
"about": "The group protocol type." }
]}
]
}

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.
{
"apiKey": 2,
"type": "request",
"name": "ListOffsetRequest",
// Version 1 removes MaxNumOffsets. From this version forward, only a single
// offset can be returned.
// Version 2 adds the isolation level, which is used for transactional reads.
// Version 3 is the same as version 2.
// Version 4 adds the current leader epoch, which is used for fencing.
// Version 5 is the same as version 5.
"validVersions": "0-5",
"fields": [
{ "name": "ReplicaId", "type": "int32", "versions": "0+",
"about": "The broker ID of the requestor, or -1 if this request is being made by a normal consumer." },
{ "name": "IsolationLevel", "type": "int8", "versions": "2+",
"about": "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records" },
{ "name": "Topics", "type": "[]ListOffsetTopic", "versions": "0+",
"about": "Each topic in the request.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]ListOffsetPartition", "versions": "0+",
"about": "Each partition in the request.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "CurrentLeaderEpoch", "type": "int32", "versions": "4+",
"about": "The current leader epoch." },
{ "name": "Timestamp", "type": "int64", "versions": "0+",
"about": "The current timestamp." },
{ "name": "MaxNumOffsets", "type": "int32", "versions": "0",
"about": "The maximum number of offsets to report." }
]}
]}
]
}

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.
{
"apiKey": 2,
"type": "response",
"name": "ListOffsetResponse",
// Version 1 removes the offsets array in favor of returning a single offset.
// Version 1 also adds the timestamp associated with the returned offset.
// Version 2 adds the throttle time.
// Starting in version 3, on quota violation, brokers send out responses before throttling.
// Version 4 adds the leader epoch, which is used for fencing.
// Version 5 adds a new error code, OFFSET_NOT_AVAILABLE.
"validVersions": "0-5",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
"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": "Topics", "type": "[]ListOffsetTopicResponse", "versions": "0+",
"about": "Each topic in the response.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name" },
{ "name": "Partitions", "type": "[]ListOffsetPartitionResponse", "versions": "0+",
"about": "Each partition in the response.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The partition error code, or 0 if there was no error." },
{ "name": "OldStyleOffsets", "type": "[]int64", "versions": "0", "ignorable": false,
"about": "The result offsets." },
{ "name": "Timestamp", "type": "int64", "versions": "1+" },
{ "name": "Offset", "type": "int64", "versions": "1+",
"about": "The timestamp associated with the returned offset." },
{ "name": "LeaderEpoch", "type": "int32", "versions": "4+" }
]}
]}
]
}

View File

@ -0,0 +1,37 @@
// 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": 3,
"type": "request",
"name": "MetadataRequest",
"validVersions": "0-7",
"fields": [
// In version 0, an empty array indicates "request metadata for all topics." In version 1 and
// higher, an empty array indicates "request metadata for no topics," and a null array is used to
// indiate "request metadata for all topics."
//
// Version 2 and 3 are the same as version 1.
//
// Version 4 adds AllowAutoTopicCreation.
{ "name": "Topics", "type": "[]MetadataRequestTopic", "versions": "0+", "nullableVersions": "1+",
"about": "The topics to fetch metadata for.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." }
]},
{ "name": "AllowAutoTopicCreation", "type": "bool", "versions": "4+", "default": "true", "ignorable": false,
"about": "If this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so." }
]
}

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.
{
"apiKey": 3,
"type": "response",
"name": "MetadataResponse",
// Version 1 adds fields for the rack of each broker, the controller id, and
// whether or not the topic is internal.
//
// Version 2 adds the cluster ID field.
//
// Version 3 adds the throttle time.
//
// Version 4 is the same as version 3.
//
// Version 5 adds a per-partition offline_replicas field. This field specifies
// the list of replicas that are offline.
//
// Starting in version 6, on quota violation, brokers send out responses before throttling.
//
// Version 7 adds the leader epoch to the partition metadata.
"validVersions": "0-7",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "3+",
"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": "Brokers", "type": "[]MetadataResponseBroker", "versions": "0+",
"about": "Each broker in the response.", "fields": [
{ "name": "NodeId", "type": "int32", "versions": "0+", "mapKey": true,
"about": "The broker ID." },
{ "name": "Host", "type": "string", "versions": "0+",
"about": "The broker hostname." },
{ "name": "Port", "type": "int32", "versions": "0+",
"about": "The broker port." },
{ "name": "Rack", "type": "string", "versions": "1+", "nullableVersions": "1+", "ignorable": true,
"about": "The rack of the broker, or null if it has not been assigned to a rack." }
]},
{ "name": "ClusterId", "type": "string", "nullableVersions": "2+", "versions": "2+", "ignorable": true,
"about": "The cluster ID that responding broker belongs to." },
{ "name": "ControllerId", "type": "int32", "versions": "1+", "default": "-1", "ignorable": "true",
"about": "The ID of the controller broker." },
{ "name": "Topics", "type": "[]MetadataResponseTopic", "versions": "0+",
"about": "Each topic in the response.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The topic error, or 0 if there was no error." },
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The topic name." },
{ "name": "IsInternal", "type": "bool", "versions": "1+", "default": "false", "ignorable": true,
"about": "True if the topic is internal." },
{ "name": "Partitions", "type": "[]MetadataResponsePartition", "versions": "0+",
"about": "Each partition in the topic.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The partition error, or 0 if there was no error." },
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "LeaderId", "type": "int32", "versions": "0+",
"about": "The ID of the leader broker." },
{ "name": "LeaderEpoch", "type": "int32", "versions": "7+", "default": "-1", "ignorable": true,
"about": "The leader epoch of this partition." },
{ "name": "ReplicaNodes", "type": "[]int32", "versions": "0+",
"about": "The set of all nodes that host this partition." },
{ "name": "IsrNodes", "type": "[]int32", "versions": "0+",
"about": "The set of nodes that are in sync with the leader for this partition." },
{ "name": "OfflineReplicas", "type": "[]int32", "versions": "5+", "ignorable": true,
"about": "The set of offline replicas of this partition." }
]}
]}
]
}

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.
{
"apiKey": 8,
"type": "request",
"name": "OffsetCommitRequest",
// Version 1 adds timestamp and group membership information, as well as the commit timestamp.
//
// Version 2 adds retention time. It removes the commit timestamp added in version 1.
//
// Version 3 and 4 are the same as version 2.
//
// Version 5 removes the retention time, which is now controlled only by a broker configuration.
//
// Version 6 adds the leader epoch for fencing.
"validVersions": "0-6",
"fields": [
{ "name": "GroupId", "type": "string", "versions": "0+",
"about": "The unique group identifier." },
{ "name": "GenerationId", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true,
"about": "The generation of the group." },
{ "name": "MemberId", "type": "string", "versions": "1+", "ignorable": true,
"about": "The member ID assigned by the group coordinator." },
{ "name": "RetentionTimeMs", "type": "int64", "versions": "2-4", "default": "-1", "ignorable": true,
"about": "The time period in ms to retain the offset." },
{ "name": "Topics", "type": "[]OffsetCommitRequestTopic", "versions": "0+",
"about": "The topics to commit offsets for.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]OffsetCommitRequestPartition", "versions": "0+",
"about": "Each partition to commit offsets for.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "CommittedOffset", "type": "int64", "versions": "0+",
"about": "The message offset to be committed." },
{ "name": "CommittedLeaderEpoch", "type": "int32", "versions": "6+", "default": "-1", "ignorable": true,
"about": "The leader epoch of this partition." },
// CommitTimestamp has been removed from v2 and later.
{ "name": "CommitTimestamp", "type": "int64", "versions": "1", "default": "-1",
"about": "The timestamp of the commit." },
{ "name": "CommittedMetadata", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "Any associated metadata the client wants to keep." }
]}
]}
]
}

View File

@ -0,0 +1,44 @@
// 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": 8,
"type": "response",
"name": "OffsetCommitResponse",
// Versions 1 and 2 are the same as version 0.
//
// Version 3 adds the throttle time to the response.
//
// Starting in version 4, on quota violation, brokers send out responses before throttling.
//
// Versions 5 and 6 are the same as version 4.
"validVersions": "0-6",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true,
"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": "Topics", "type": "[]OffsetCommitResponseTopic", "versions": "0+",
"about": "The responses for each topic.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]OffsetCommitResponsePartition", "versions": "0+",
"about": "The responses for each partition in the topic.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." }
]}
]}
]
}

View File

@ -0,0 +1,37 @@
// 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": 9,
"type": "request",
"name": "OffsetFetchRequest",
// Starting in version 1, the broker supports fetching offsets from the internal __consumer_offsets topic.
//
// Starting in version 2, the request can contain a null topics array to indicate that offsets
// for all topics should be fetched.
//
// Version 3, 4, and 5 are the same as version 2.
"validVersions": "0-5",
"fields": [
{ "name": "GroupId", "type": "string", "versions": "0+",
"about": "The group to fetch offsets for." },
{ "name": "Topics", "type": "[]OffsetFetchRequestTopic", "versions": "0+", "nullableVersions": "2+",
"about": "Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+" },
{ "name": "PartitionIndexes", "type": "[]int32", "versions": "0+",
"about": "The partition indexes we would like to fetch offsets for." }
]}
]
}

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.
{
"apiKey": 9,
"type": "response",
"name": "OffsetFetchResponse",
// Version 1 is the same as version 0.
//
// Version 2 adds a top-level error code.
//
// Version 3 adds the throttle time.
//
// Starting in version 4, on quota violation, brokers send out responses before throttling.
//
// Version 5 adds the leader epoch to the committed offset.
"validVersions": "0-5",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true,
"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": "Topics", "type": "[]OffsetFetchResponseTopic", "versions": "0+",
"about": "The responses per topic.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]OffsetFetchResponsePartition", "versions": "0+",
"about": "The responses per partition", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "CommittedOffset", "type": "int64", "versions": "0+",
"about": "The committed message offset." },
{ "name": "CommittedLeaderEpoch", "type": "int32", "versions": "5+",
"about": "The leader epoch." },
{ "name": "Metadata", "type": "string", "versions": "0+", "nullableVersions": "0+",
"about": "The partition metadata." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." }
]}
]},
{ "name": "ErrorCode", "type": "int16", "versions": "2+", "default": "0", "ignorable": false,
"about": "The top-level error code, or 0 if there was no error." }
]
}

View File

@ -0,0 +1,40 @@
// 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": 23,
"type": "request",
"name": "OffsetForLeaderEpochRequest",
// Version 1 is the same as version 0.
//
// Version 2 adds the current leader epoch to support fencing.
"validVersions": "0-2",
"fields": [
{ "name": "Topics", "type": "[]OffsetForLeaderTopic", "versions": "0+",
"about": "Each topic to get offsets for.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]OffsetForLeaderPartition", "versions": "0+",
"about": "Each partition to get offsets for.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "CurrentLeaderEpoch", "type": "int32", "versions": "2+", "default": "-1", "ignorable": true,
"about": "An epoch used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned." },
{ "name": "LeaderEpoch", "type": "int32", "versions": "0+",
"about": "The epoch to look up an offset for." }
]}
]}
]
}

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.
{
"apiKey": 23,
"type": "response",
"name": "OffsetForLeaderEpochResponse",
// Version 1 added the leader epoch to the response.
// Version 2 added the throttle time.
"validVersions": "0-2",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
"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": "Topics", "type": "[]OffsetForLeaderTopicResult", "versions": "0+",
"about": "Each topic we fetched offsets for.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]OffsetForLeaderPartitionResult", "versions": "0+",
"about": "Each partition in the topic we fetched offsets for.", "fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code 0, or if there was no error." },
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "LeaderEpoch", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true,
"about": "The leader epoch of the partition." },
{ "name": "EndOffset", "type": "int64", "versions": "0+",
"about": "The end offset of the epoch." }
]}
]}
]
}

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.
{
"apiKey": 0,
"type": "request",
"name": "ProduceRequest",
// Version 1 and 2 are the same as version 0.
//
// Version 3 adds the transactional ID, which is used for authorization when attempting to write
// transactional data. Version 3 also adds support for Kafka Message Format v2.
//
// Version 4 is the same as version 3, but the requestor must be prepared to handle a
// KAFKA_STORAGE_ERROR.
//
// Version 5 and 6 are the same as version 3.
//
// Starting in version 7, records can be produced using ZStandard compression. See KIP-110.
"validVersions": "0-7",
"fields": [
{ "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "0+",
"about": "The transactional ID, or null if the producer is not transactional." },
{ "name": "Acks", "type": "int16", "versions": "0+",
"about": "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR." },
{ "name": "TimeoutMs", "type": "int32", "versions": "0+",
"about": "The timeout to await a response in miliseconds." },
{ "name": "Topics", "type": "[]TopicProduceData", "versions": "0+",
"about": "Each topic to produce to.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name." },
{ "name": "Partitions", "type": "[]PartitionProduceData", "versions": "0+",
"about": "Each partition to produce to.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "Records", "type": "bytes", "versions": "0+", "nullableVersions": "0+",
"about": "The record data to be produced." }
]}
]}
]
}

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.
{
"apiKey": 0,
"type": "response",
"name": "ProduceResponse",
// Version 1 added the throttle time.
//
// Version 2 added the log append time.
//
// Version 3 is the same as version 2.
//
// Version 4 added KAFKA_STORAGE_ERROR as a possible error code.
//
// Version 5 added LogStartOffset to filter out spurious
// OutOfOrderSequenceExceptions on the client.
"validVersions": "0-7",
"fields": [
{ "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+",
"about": "Each produce response", "fields": [
{ "name": "Name", "type": "string", "versions": "0+",
"about": "The topic name" },
{ "name": "Partitions", "type": "[]PartitionProduceResponse", "versions": "0+",
"about": "Each partition that we produced to within the topic.", "fields": [
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no error." },
{ "name": "BaseOffset", "type": "int64", "versions": "0+",
"about": "The base offset." },
{ "name": "LogAppendTimeMs", "type": "int64", "versions": "2+", "default": "-1", "ignorable": true,
"about": "The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended." },
{ "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true,
"about": "The log start offset." }
]}
]},
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
"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." }
]
}

View File

@ -0,0 +1,219 @@
Apache Kafka Message Definitions
================================
Introduction
------------
The JSON files in this directory define the Apache Kafka message protocol.
This protocol describes what information clients and servers send to each
other, and how it is serialized. Note that this version of JSON supports
comments. Comments begin with a double forward slash.
When Kafka is compiled, these specification files are translated into Java code
to read and write messages. Any change to these JSON files will trigger a
recompilation of this generated code.
These specification files replace an older system where hand-written
serialization code was used. Over time, we will migrate all messages to using
automatically generated serialization and deserialization code.
Requests and Responses
----------------------
The Kafka protocol features requests and responses. Requests are sent to a
server in order to get a response. Each request is uniquely identified by a
16-bit integer called the "api key". The API key of the response will always
match that of the request.
Each message has a unique 16-bit version number. The schema might be different
for each version of the message. Sometimes, the version is incremented even
though the schema has not changed. This may indicate that the server should
behave differently in some way. The version of a response must always match
the version of the corresponding request.
Each request or response has a top-level field named "validVersions." This
specifies the versions of the protocol that our code understands. For example,
specifying "0-2" indicates that we understand versions 0, 1, and 2. You must
always specify the highest message version which is supported.
The only old message versions that are no longer supported are version 0 of
MetadataRequest and MetadataResponse. In general, since we adopted KIP-97,
dropping support for old message versions is no longer allowed without a KIP.
Therefore, please be careful not to increase the lower end of the version
support interval for any message.
MessageData Objects
-------------------
Using the JSON files in this directory, we generate Java code for MessageData
objects. These objects store request and response data for kafka. MessageData
objects do not contain a version number. Instead, a single MessageData object
represents every possible version of a Message. This makes working with
messages more convenient, because the same code path can be used for every
version of a message.
Fields
------
Each message contains an array of fields. Fields specify the data that should
be sent with the message. In general, fields have a name, a type, and version
information associated with them.
The order that fields appear in a message is important. Fields which come
first in the message definition will be sent first over the network. Changing
the order of the fields in a message is an incompatible change.
In each new message version, we may add or subtract fields. For example, if we
are creating a new version 3 of a message, we can add a new field with the
version spec "3+". This specifies that the field only appears in version 3 and
later. If a field is being removed, we should change its version from "0+" to
"0-2" to indicate that it will not appear in version 3 and later.
Field Types
-----------
There are several primitive field types available.
* "boolean": either true or false. This takes up 1 byte on the wire.
* "int8": an 8-bit integer. This also takes up 1 byte on the wire.
* "int16": a 16-bit integer. This takes up 2 bytes on the wire.
* "int32": a 32-bit integer. This takes up 4 bytes on the wire.
* "int64": a 64-bit integer. This takes up 8 bytes on the wire.
* "string": a string. This must be less than 64kb in size when serialized as UTF-8. This takes up 2 bytes on the wire, plus the length of the string when serialized to UTF-8.
* "bytes": binary data. This takes up 4 bytes on the wire, plus the length of the bytes.
In addition to these primitive field types, there is also an array type. Array
types start with a "[]" and end with the name of the element type. For
example, []Foo declares an array of "Foo" objects. Array fields have their own
array of fields, which specifies what is in the contained objects.
Nullable Fields
---------------
Booleans and ints can never be null. However, fields that are strings, bytes,
or arrays may optionally be "nullable." When a field is "nullable," that
simply means that we are prepared to serialize and deserialize null entries for
that field.
If you want to declare a field as nullable, you set "nullableVersions" for that
field. Nullability is implemented as a version range in order to accomodate a
very common pattern in Kafka where a field that was originally not nullable
becomes nullable in a later version.
If a field is declared as non-nullable, and it is present in the message
version you are using, you should set it to a non-null value before serializing
the message. Otherwise, you will get a runtime error.
Serializing Messages
--------------------
The Message#write method writes out a message to a buffer. The fields that are
written out will depend on the version number that you supply to write(). When
you write out a message using an older version, fields that are too old to be
present in the schema will be omitted.
When working with older message versions, please verify that the older message
schema includes all the data that needs to be sent. For example, it is probably
OK to skip sending a timeout field. However, a field which radically alters the
meaning of the request, such as a "validateOnly" boolean, should not be ignored.
It's often useful to know how much space a message will take up before writing
it out to a buffer. You can find this out by calling the Message#size method.
You can also convert a message to a Struct by calling Message#toStruct. This
allows you to use the functions that serialize Structs to buffers.
Deserializing Messages
----------------------
Message objects may be deserialized using the Message#read method. This method
overwrites all the data currently in the message object with new data.
You can also deserialize a message from a Struct by calling Message#fromStruct.
The Struct will not be modified.
Any fields in the message object that are not present in the version that you
are deserializing will be reset to default values. Unless a custom default has
been set:
* Integer fields default to 0.
* Booleans default to false.
* Strings default to the empty string.
* Bytes fields default to the empty byte array.
* Array fields default to empty.
Null is never used as a default for any field.
Custom Default Values
---------------------
You may set a custom default for fields that are integers, booleans, or strings.
Just add a "default" entry in the JSON object. The custom default overrides the
normal default for the type. So for example, you could make a boolean field
default to true rather than false, and so forth.
Note that the default must be valid for the field type. So the default for an
int16 field must by an integer that fits in 16 bits, and so forth. You may
specify hex or octal values, as long as they are prefixed with 0x or 0. It is
currently not possible to specify a custom default for bytes or array fields.
Custom defaults are useful when an older message version lacked some
information. For example, if an older request lacked a timeout field, you may
want to specify that the server should assume that the timeout for such a
request is 5000 ms (or some other arbitrary value.)
Ignorable Fields
----------------
When we write messages using an older or newer format, not all fields may be
present. The message receiver will fill in the default value for the field
during deserialization. Therefore, if the source field was set to a non-default
value, that information will be lost.
In some cases, this information loss is acceptable. For example, if a timeout
field does not get preserved, this is not a problem. However, in other cases,
the field is really quite important and should not be discarded. One example is
a "verify only" boolean which changes the whole meaning of the request.
By default, we assume that information loss is not acceptable. The message
serialization code will throw an exception if the ignored field is not set to
the default value. If information loss for a field is OK, please set
"ignorable" to true for the field to disable this behavior. When ignorable is
set to true, the field may be silently omitted during serialization.
Hash Sets
---------
One very common pattern in Kafka is to load array elements from a message into
a Map or Set for easier access. The message protocol makes this easier with
the "mapKey" concept.
If some of the elemements of an array are annotated with "mapKey": true, the
entire array will be treated as a linked hash set rather than a list. Elements
in this set will be accessible in O(1) time with an automatically generated
"find" function. The order of elements in the set will still be preserved,
however. New entries that are added to the set always show up as last in the
ordering.
Incompatible Changes
--------------------
It's very important to avoid making incompatible changes to the message
protocol. Here are some examples of incompatible changes:
#### Making changes to a protocol version which has already been released.
Protocol versions that have been released must be regarded as done. If there
were mistakes, they should be corrected in a new version rather than changing
the existing version.
#### Re-ordering existing fields.
It is OK to add new fields before or after existing fields. However, existing
fields should not be re-ordered with respect to each other.
#### Changing the default of an existing field.
You must never change the default of a field which already exists. Otherwise,
new clients and old servers will not agree on the default, and so forth.
#### Changing the type of an existing field.
One exception is that an array of primitives may be changed to an array of
structures containing the same data, as long as the conversion is done
correctly. The Kafka protocol does not do any "boxing" of structures, so an
array of structs that contain a single int32 is the same as an array of int32s.

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.
{
"apiKey": 39,
"type": "request",
"name": "RenewDelegationTokenRequest",
// Version 1 is the same as version 0.
"validVersions": "0-1",
"fields": [
{ "name": "Hmac", "type": "bytes", "versions": "0+",
"about": "The HMAC of the delegation token to be renewed." },
{ "name": "RenewPeriodMs", "type": "int64", "versions": "0+",
"about": "The renewal time period in milliseconds." }
]
}

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