diff --git a/incubator/command-log/src/main/java/io/aklivity/zilla/runtime/command/log/internal/LoggableStream.java b/incubator/command-log/src/main/java/io/aklivity/zilla/runtime/command/log/internal/LoggableStream.java index f857dd303d..c8a8689afb 100644 --- a/incubator/command-log/src/main/java/io/aklivity/zilla/runtime/command/log/internal/LoggableStream.java +++ b/incubator/command-log/src/main/java/io/aklivity/zilla/runtime/command/log/internal/LoggableStream.java @@ -87,6 +87,8 @@ import io.aklivity.zilla.runtime.command.log.internal.types.stream.KafkaFetchDataExFW; import io.aklivity.zilla.runtime.command.log.internal.types.stream.KafkaFetchFlushExFW; import io.aklivity.zilla.runtime.command.log.internal.types.stream.KafkaFlushExFW; +import io.aklivity.zilla.runtime.command.log.internal.types.stream.KafkaGroupBeginExFW; +import io.aklivity.zilla.runtime.command.log.internal.types.stream.KafkaGroupDataExFW; import io.aklivity.zilla.runtime.command.log.internal.types.stream.KafkaMergedBeginExFW; import io.aklivity.zilla.runtime.command.log.internal.types.stream.KafkaMergedDataExFW; import io.aklivity.zilla.runtime.command.log.internal.types.stream.KafkaMergedFlushExFW; @@ -890,6 +892,9 @@ private void onKafkaBeginEx( case KafkaBeginExFW.KIND_DESCRIBE: onKafkaDescribeBeginEx(offset, timestamp, kafkaBeginEx.describe()); break; + case KafkaBeginExFW.KIND_GROUP: + onKafkaGroupBeginEx(offset, timestamp, kafkaBeginEx.group()); + break; case KafkaBeginExFW.KIND_FETCH: onKafkaFetchBeginEx(offset, timestamp, kafkaBeginEx.fetch()); break; @@ -946,6 +951,19 @@ private void onKafkaDescribeBeginEx( configs.forEach(c -> out.printf(verboseFormat, index, offset, timestamp, c.asString())); } + private void onKafkaGroupBeginEx( + int offset, + long timestamp, + KafkaGroupBeginExFW group) + { + String16FW groupId = group.groupId(); + String16FW protocol = group.protocol(); + int timeout = group.timeout(); + + out.printf(verboseFormat, index, offset, timestamp, format("[group] %s %s %d", + groupId.asString(), protocol.asString(), timeout)); + } + private void onKafkaFetchBeginEx( int offset, long timestamp, @@ -1062,6 +1080,9 @@ private void onKafkaDataEx( case KafkaDataExFW.KIND_DESCRIBE: onKafkaDescribeDataEx(offset, timestamp, kafkaDataEx.describe()); break; + case KafkaDataExFW.KIND_GROUP: + onKafkaGroupDataEx(offset, timestamp, kafkaDataEx.group()); + break; case KafkaDataExFW.KIND_FETCH: onKafkaFetchDataEx(offset, timestamp, kafkaDataEx.fetch()); break; @@ -1089,6 +1110,17 @@ private void onKafkaDescribeDataEx( format("%s: %s", c.name().asString(), c.value().asString()))); } + private void onKafkaGroupDataEx( + int offset, + long timestamp, + KafkaGroupDataExFW group) + { + String16FW leader = group.leaderId(); + String16FW member = group.memberId(); + + out.printf(verboseFormat, index, offset, timestamp, format("[group] %s %s", leader.asString(), member.asString())); + } + private void onKafkaFetchDataEx( int offset, long timestamp, diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/KafkaConfiguration.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/KafkaConfiguration.java index d9cf0a1d04..54dffe1320 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/KafkaConfiguration.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/KafkaConfiguration.java @@ -20,9 +20,12 @@ import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodType; +import java.lang.reflect.Method; import java.math.BigInteger; import java.nio.file.Path; import java.security.SecureRandom; +import java.time.Duration; +import java.util.UUID; import java.util.function.Supplier; import org.agrona.LangUtil; @@ -63,6 +66,9 @@ public class KafkaConfiguration extends Configuration public static final IntPropertyDef KAFKA_CACHE_CLIENT_TRAILERS_SIZE_MAX; public static final IntPropertyDef KAFKA_CACHE_SERVER_RECONNECT_DELAY; public static final PropertyDef KAFKA_CLIENT_SASL_SCRAM_NONCE; + public static final PropertyDef KAFKA_CLIENT_GROUP_REBALANCE_TIMEOUT; + public static final PropertyDef KAFKA_CLIENT_ID; + public static final PropertyDef KAFKA_CLIENT_INSTANCE_ID; private static final ConfigurationDef KAFKA_CONFIG; @@ -100,6 +106,11 @@ public class KafkaConfiguration extends Configuration KAFKA_CACHE_CLIENT_TRAILERS_SIZE_MAX = config.property("cache.client.trailers.size.max", 256); KAFKA_CLIENT_SASL_SCRAM_NONCE = config.property(NonceSupplier.class, "client.sasl.scram.nonce", KafkaConfiguration::decodeNonceSupplier, KafkaConfiguration::defaultNonceSupplier); + KAFKA_CLIENT_GROUP_REBALANCE_TIMEOUT = config.property(Duration.class, "client.group.rebalance.timeout", + (c, v) -> Duration.parse(v), "PT4S"); + KAFKA_CLIENT_ID = config.property("client.id", "zilla"); + KAFKA_CLIENT_INSTANCE_ID = config.property(InstanceIdSupplier.class, "client.instance.id", + KafkaConfiguration::decodeInstanceId, KafkaConfiguration::defaultInstanceId); KAFKA_CONFIG = config; } @@ -248,6 +259,16 @@ public int cacheClientTrailersSizeMax() return KAFKA_CACHE_CLIENT_TRAILERS_SIZE_MAX.getAsInt(this); } + public String clientId() + { + return KAFKA_CLIENT_ID.get(this); + } + + public Duration clientGroupRebalanceTimeout() + { + return KAFKA_CLIENT_GROUP_REBALANCE_TIMEOUT.get(this); + } + private static Path cacheDirectory( Configuration config, String cacheDirectory) @@ -267,6 +288,11 @@ public Supplier nonceSupplier() return KAFKA_CLIENT_SASL_SCRAM_NONCE.get(this)::get; } + public Supplier clientInstanceIdSupplier() + { + return KAFKA_CLIENT_INSTANCE_ID.get(this)::get; + } + @FunctionalInterface private interface NonceSupplier { @@ -315,4 +341,55 @@ private static NonceSupplier defaultNonceSupplier( return () -> new BigInteger(130, new SecureRandom()).toString(Character.MAX_RADIX); } + + @FunctionalInterface + public interface InstanceIdSupplier extends Supplier + { + } + + private static InstanceIdSupplier decodeInstanceId( + Configuration config, + String value) + { + try + { + String className = value.substring(0, value.indexOf("$$Lambda")); + Class lambdaClass = Class.forName(className); + + Method targetMethod = null; + for (Method method : lambdaClass.getDeclaredMethods()) + { + if (method.isSynthetic()) + { + targetMethod = method; + break; + } + } + + Method finalTargetMethod = targetMethod; + return () -> + { + try + { + finalTargetMethod.setAccessible(true); + return (String) finalTargetMethod.invoke(null); + } + catch (Exception e) + { + throw new RuntimeException("Failed to invoke the lambda method.", e); + } + }; + } + catch (Throwable ex) + { + LangUtil.rethrowUnchecked(ex); + } + return null; + } + + private static InstanceIdSupplier defaultInstanceId( + Configuration config) + { + return () -> String.format("%s-%s", KAFKA_CLIENT_ID.get(config), UUID.randomUUID()); + } } diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaBindingConfig.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaBindingConfig.java index df6aac76cc..ff9e7c6d0e 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaBindingConfig.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaBindingConfig.java @@ -45,7 +45,18 @@ public KafkaRouteConfig resolve( String topic) { return routes.stream() - .filter(r -> r.authorized(authorization) && r.matches(topic)) + .filter(r -> r.authorized(authorization) && r.matches(topic, null)) + .findFirst() + .orElse(null); + } + + public KafkaRouteConfig resolve( + long authorization, + String topic, + String groupId) + { + return routes.stream() + .filter(r -> r.authorized(authorization) && r.matches(topic, groupId)) .findFirst() .orElse(null); } diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfig.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfig.java index d217d16ea9..0d222e179d 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfig.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfig.java @@ -20,10 +20,13 @@ public final class KafkaConditionConfig extends ConditionConfig { public final String topic; + public final String groupId; public KafkaConditionConfig( - String topic) + String topic, + String groupId) { this.topic = topic; + this.groupId = groupId; } } diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfigAdapter.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfigAdapter.java index 351d45ce06..46e3cdd2e5 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfigAdapter.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfigAdapter.java @@ -27,6 +27,7 @@ public final class KafkaConditionConfigAdapter implements ConditionConfigAdapterSpi, JsonbAdapter { private static final String TOPIC_NAME = "topic"; + private static final String GROUP_ID_NAME = "groupId"; @Override public String type() @@ -47,6 +48,11 @@ public JsonObject adaptToJson( object.add(TOPIC_NAME, kafkaCondition.topic); } + if (kafkaCondition.groupId != null) + { + object.add(GROUP_ID_NAME, kafkaCondition.groupId); + } + return object.build(); } @@ -58,6 +64,10 @@ public ConditionConfig adaptFromJson( ? object.getString(TOPIC_NAME) : null; - return new KafkaConditionConfig(topic); + String groupId = object.containsKey(GROUP_ID_NAME) + ? object.getString(GROUP_ID_NAME) + : null; + + return new KafkaConditionConfig(topic, groupId); } } diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionMatcher.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionMatcher.java index 37bf8d0e85..329e70b898 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionMatcher.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionMatcher.java @@ -21,17 +21,20 @@ public final class KafkaConditionMatcher { private final Matcher topicMatch; + private final Matcher grouoIdMatch; public KafkaConditionMatcher( KafkaConditionConfig condition) { this.topicMatch = condition.topic != null ? asMatcher(condition.topic) : null; + this.grouoIdMatch = condition.groupId != null ? asMatcher(condition.groupId) : null; } public boolean matches( - String topic) + String topic, + String groupId) { - return matchesTopic(topic); + return matchesTopic(topic) && matchesGroupId(groupId); } private boolean matchesTopic( @@ -40,9 +43,18 @@ private boolean matchesTopic( return this.topicMatch == null || this.topicMatch.reset(topic).matches(); } + private boolean matchesGroupId( + String groupId) + { + return this.grouoIdMatch == null || this.grouoIdMatch.reset(groupId).matches(); + } + private static Matcher asMatcher( String wildcard) { - return Pattern.compile(wildcard.replace(".", "\\.").replace("*", ".*")).matcher(""); + return Pattern.compile(wildcard + .replace(".", "\\.") + .replace("*", ".*")) + .matcher(""); } } diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaRouteConfig.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaRouteConfig.java index 8949acfc3c..83805da27a 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaRouteConfig.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaRouteConfig.java @@ -50,8 +50,9 @@ boolean authorized( } boolean matches( - String topic) + String topic, + String groupId) { - return when.isEmpty() || when.stream().anyMatch(m -> m.matches(topic)); + return when.isEmpty() || when.stream().anyMatch(m -> m.matches(topic, groupId)); } } diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheClientFactory.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheClientFactory.java index 8833175c5e..de57588c10 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheClientFactory.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheClientFactory.java @@ -64,6 +64,8 @@ public KafkaCacheClientFactory( final KafkaCacheClientDescribeFactory cacheDescribeFactory = new KafkaCacheClientDescribeFactory( config, context, bindings::get, supplyCacheRoute); + final KafkaCacheGroupFactory cacheGroupFactory = new KafkaCacheGroupFactory(config, context, bindings::get); + final KafkaCacheClientFetchFactory cacheFetchFactory = new KafkaCacheClientFetchFactory( config, context, bindings::get, accountant::supplyDebitor, supplyCache, supplyCacheRoute); @@ -76,6 +78,7 @@ public KafkaCacheClientFactory( final Int2ObjectHashMap factories = new Int2ObjectHashMap<>(); factories.put(KafkaBeginExFW.KIND_META, cacheMetaFactory); factories.put(KafkaBeginExFW.KIND_DESCRIBE, cacheDescribeFactory); + factories.put(KafkaBeginExFW.KIND_GROUP, cacheGroupFactory); factories.put(KafkaBeginExFW.KIND_FETCH, cacheFetchFactory); factories.put(KafkaBeginExFW.KIND_PRODUCE, cacheProduceFactory); factories.put(KafkaBeginExFW.KIND_MERGED, cacheMergedFactory); diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheGroupFactory.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheGroupFactory.java new file mode 100644 index 0000000000..c27fe29960 --- /dev/null +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheGroupFactory.java @@ -0,0 +1,1037 @@ +/* + * Copyright 2021-2023 Aklivity Inc. + * + * Aklivity 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 io.aklivity.zilla.runtime.binding.kafka.internal.stream; + +import java.util.function.Consumer; +import java.util.function.LongFunction; +import java.util.function.LongUnaryOperator; + +import org.agrona.DirectBuffer; +import org.agrona.MutableDirectBuffer; +import org.agrona.concurrent.UnsafeBuffer; + +import io.aklivity.zilla.runtime.binding.kafka.internal.KafkaBinding; +import io.aklivity.zilla.runtime.binding.kafka.internal.KafkaConfiguration; +import io.aklivity.zilla.runtime.binding.kafka.internal.config.KafkaBindingConfig; +import io.aklivity.zilla.runtime.binding.kafka.internal.config.KafkaRouteConfig; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.Flyweight; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.OctetsFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.AbortFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.BeginFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.DataFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.EndFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.ExtensionFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.FlushFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.KafkaBeginExFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.KafkaGroupBeginExFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.ResetFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.WindowFW; +import io.aklivity.zilla.runtime.engine.EngineContext; +import io.aklivity.zilla.runtime.engine.binding.BindingHandler; +import io.aklivity.zilla.runtime.engine.binding.function.MessageConsumer; +import io.aklivity.zilla.runtime.engine.buffer.BufferPool; + +public final class KafkaCacheGroupFactory implements BindingHandler +{ + private static final Consumer EMPTY_EXTENSION = ex -> {}; + + + private final BeginFW beginRO = new BeginFW(); + private final DataFW dataRO = new DataFW(); + private final EndFW endRO = new EndFW(); + private final FlushFW flushRO = new FlushFW(); + private final AbortFW abortRO = new AbortFW(); + private final ResetFW resetRO = new ResetFW(); + private final WindowFW windowRO = new WindowFW(); + private final ExtensionFW extensionRO = new ExtensionFW(); + private final KafkaBeginExFW kafkaBeginExRO = new KafkaBeginExFW(); + + private final BeginFW.Builder beginRW = new BeginFW.Builder(); + private final DataFW.Builder dataRW = new DataFW.Builder(); + private final FlushFW.Builder flushRW = new FlushFW.Builder(); + private final EndFW.Builder endRW = new EndFW.Builder(); + private final AbortFW.Builder abortRW = new AbortFW.Builder(); + private final ResetFW.Builder resetRW = new ResetFW.Builder(); + private final WindowFW.Builder windowRW = new WindowFW.Builder(); + private final KafkaBeginExFW.Builder kafkaBeginExRW = new KafkaBeginExFW.Builder(); + + private final int kafkaTypeId; + private final MutableDirectBuffer writeBuffer; + private final BufferPool bufferPool; + private final BindingHandler streamFactory; + private final LongUnaryOperator supplyInitialId; + private final LongUnaryOperator supplyReplyId; + private final LongFunction supplyBinding; + + public KafkaCacheGroupFactory( + KafkaConfiguration config, + EngineContext context, + LongFunction supplyBinding) + { + this.kafkaTypeId = context.supplyTypeId(KafkaBinding.NAME); + this.writeBuffer = new UnsafeBuffer(new byte[context.writeBuffer().capacity()]); + this.bufferPool = context.bufferPool(); + this.streamFactory = context.streamFactory(); + this.supplyInitialId = context::supplyInitialId; + this.supplyReplyId = context::supplyReplyId; + this.supplyBinding = supplyBinding; + } + + @Override + public MessageConsumer newStream( + int msgTypeId, + DirectBuffer buffer, + int index, + int length, + MessageConsumer sender) + { + final BeginFW begin = beginRO.wrap(buffer, index, index + length); + final long originId = begin.originId(); + final long routedId = begin.routedId(); + final long initialId = begin.streamId(); + final long authorization = begin.authorization(); + final long affinity = begin.affinity(); + + assert (initialId & 0x0000_0000_0000_0001L) != 0L; + + final OctetsFW extension = begin.extension(); + final ExtensionFW beginEx = extension.get(extensionRO::tryWrap); + assert beginEx != null && beginEx.typeId() == kafkaTypeId; + final KafkaBeginExFW kafkaBeginEx = extension.get(kafkaBeginExRO::tryWrap); + assert kafkaBeginEx.kind() == KafkaBeginExFW.KIND_GROUP; + final KafkaGroupBeginExFW kafkaGroupBeginEx = kafkaBeginEx.group(); + final String groupId = kafkaGroupBeginEx.groupId().asString(); + final String protocol = kafkaGroupBeginEx.protocol().asString(); + final int timeout = kafkaGroupBeginEx.timeout(); + + MessageConsumer newStream = null; + + final KafkaBindingConfig binding = supplyBinding.apply(routedId); + final KafkaRouteConfig resolved = binding != null ? binding.resolve(authorization, null, groupId) : null; + + if (resolved != null) + { + final long resolvedId = resolved.id; + + newStream = new KafkaCacheGroupApp( + sender, + originId, + routedId, + initialId, + affinity, + authorization, + resolvedId, + groupId, + protocol, + timeout)::onGroupMessage; + } + + return newStream; + } + + private MessageConsumer newStream( + MessageConsumer sender, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long affinity, + Consumer extension) + { + final BeginFW begin = beginRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .affinity(affinity) + .extension(extension) + .build(); + + final MessageConsumer receiver = + streamFactory.newStream(begin.typeId(), begin.buffer(), begin.offset(), begin.sizeof(), sender); + + receiver.accept(begin.typeId(), begin.buffer(), begin.offset(), begin.sizeof()); + + return receiver; + } + + private void doBegin( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long affinity, + Consumer extension) + { + final BeginFW begin = beginRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .affinity(affinity) + .extension(extension) + .build(); + + receiver.accept(begin.typeId(), begin.buffer(), begin.offset(), begin.sizeof()); + } + + private void doData( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long budgetId, + int flags, + int reserved, + OctetsFW payload, + Flyweight extension) + { + final DataFW frame = dataRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .flags(flags) + .budgetId(budgetId) + .reserved(reserved) + .payload(payload) + .extension(extension.buffer(), extension.offset(), extension.sizeof()) + .build(); + + receiver.accept(frame.typeId(), frame.buffer(), frame.offset(), frame.sizeof()); + } + + + private void doDataNull( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long budgetId, + int reserved, + Flyweight extension) + { + final DataFW data = dataRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .budgetId(budgetId) + .reserved(reserved) + .extension(extension.buffer(), extension.offset(), extension.sizeof()) + .build(); + + receiver.accept(data.typeId(), data.buffer(), data.offset(), data.sizeof()); + } + + private void doFlush( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long budgetId, + int reserved, + Consumer extension) + { + final FlushFW flush = flushRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .budgetId(budgetId) + .reserved(reserved) + .extension(extension) + .build(); + + receiver.accept(flush.typeId(), flush.buffer(), flush.offset(), flush.sizeof()); + } + + private void doEnd( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + Consumer extension) + { + final EndFW end = endRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .extension(extension) + .build(); + + receiver.accept(end.typeId(), end.buffer(), end.offset(), end.sizeof()); + } + + private void doAbort( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + Consumer extension) + { + final AbortFW abort = abortRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .extension(extension) + .build(); + + receiver.accept(abort.typeId(), abort.buffer(), abort.offset(), abort.sizeof()); + } + + private void doWindow( + MessageConsumer sender, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long budgetId, + int padding) + { + final WindowFW window = windowRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .budgetId(budgetId) + .padding(padding) + .build(); + + sender.accept(window.typeId(), window.buffer(), window.offset(), window.sizeof()); + } + + private void doReset( + MessageConsumer sender, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization) + { + final ResetFW reset = resetRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .build(); + + sender.accept(reset.typeId(), reset.buffer(), reset.offset(), reset.sizeof()); + } + + final class KafkaCacheGroupNet + { + private final long originId; + private final long routedId; + private final long authorization; + private final KafkaCacheGroupApp delegate; + + private long initialId; + private long replyId; + private MessageConsumer receiver; + + private int state; + + private long initialSeq; + private long initialAck; + private int initialMax; + private long initialBud; + + private long replySeq; + private long replyAck; + private int replyMax; + private int replyPad; + + private KafkaCacheGroupNet( + KafkaCacheGroupApp delegate, + long originId, + long routedId, + long authorization) + { + this.delegate = delegate; + this.originId = originId; + this.routedId = routedId; + this.receiver = MessageConsumer.NOOP; + this.authorization = authorization; + } + + private void doGroupInitialBegin( + long traceId) + { + if (KafkaState.closed(state)) + { + state = 0; + } + + if (!KafkaState.initialOpening(state)) + { + if (KafkaConfiguration.DEBUG) + { + System.out.format("%s GroupId connect\n", delegate.groupId); + } + + assert state == 0; + + this.initialId = supplyInitialId.applyAsLong(routedId); + this.replyId = supplyReplyId.applyAsLong(initialId); + this.receiver = newStream(this::onGroupMessage, + originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, 0L, + ex -> ex.set((b, o, l) -> kafkaBeginExRW.wrap(b, o, l) + .typeId(kafkaTypeId) + .group(g -> g.groupId(delegate.groupId) + .protocol(delegate.protocol) + .timeout(delegate.timeout)) + .build() + .sizeof())); + state = KafkaState.openingInitial(state); + } + } + + private void doGroupInitialData( + long traceId, + long authorization, + long budgetId, + int reserved, + int flags, + OctetsFW payload, + Flyweight extension) + { + doData(receiver, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, budgetId, flags, reserved, payload, extension); + + initialSeq += reserved; + + assert initialSeq <= initialAck + initialMax; + } + + private void doGroupInitialFlush( + long traceId) + { + doFlush(receiver, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, initialBud, 0, EMPTY_EXTENSION); + } + + private void doGroupInitialEnd( + long traceId) + { + if (!KafkaState.initialClosed(state)) + { + doEnd(receiver, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, EMPTY_EXTENSION); + + state = KafkaState.closedInitial(state); + } + } + + private void doGroupInitialAbort( + long traceId) + { + if (!KafkaState.initialClosed(state)) + { + doAbort(receiver, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, EMPTY_EXTENSION); + + state = KafkaState.closedInitial(state); + } + } + + private void onGroupInitialReset( + ResetFW reset) + { + final long sequence = reset.sequence(); + final long acknowledge = reset.acknowledge(); + final long traceId = reset.traceId(); + + assert acknowledge <= sequence; + assert acknowledge >= delegate.initialAck; + + delegate.initialAck = acknowledge; + state = KafkaState.closedInitial(state); + + assert delegate.initialAck <= delegate.initialSeq; + + delegate.doGroupInitialReset(traceId); + + doGroupReplyReset(traceId); + } + + + private void onGroupInitialWindow( + WindowFW window) + { + final long sequence = window.sequence(); + final long acknowledge = window.acknowledge(); + final int maximum = window.maximum(); + final long authorization = window.authorization(); + final long traceId = window.traceId(); + final long budgetId = window.budgetId(); + final int padding = window.padding(); + final int capabilities = window.capabilities(); + + assert acknowledge <= sequence; + assert acknowledge >= delegate.initialAck; + assert maximum >= delegate.initialMax; + + initialAck = acknowledge; + initialMax = maximum; + initialBud = budgetId; + state = KafkaState.openedInitial(state); + + assert initialAck <= initialSeq; + + delegate.doGroupInitialWindow(authorization, traceId, budgetId, padding); + } + + private void onGroupMessage( + int msgTypeId, + DirectBuffer buffer, + int index, + int length) + { + switch (msgTypeId) + { + case BeginFW.TYPE_ID: + final BeginFW begin = beginRO.wrap(buffer, index, index + length); + onGroupReplyBegin(begin); + break; + case DataFW.TYPE_ID: + final DataFW data = dataRO.wrap(buffer, index, index + length); + onGroupReplyData(data); + break; + case EndFW.TYPE_ID: + final EndFW end = endRO.wrap(buffer, index, index + length); + onGroupReplyEnd(end); + break; + case AbortFW.TYPE_ID: + final AbortFW abort = abortRO.wrap(buffer, index, index + length); + onGroupReplyAbort(abort); + break; + case ResetFW.TYPE_ID: + final ResetFW reset = resetRO.wrap(buffer, index, index + length); + onGroupInitialReset(reset); + break; + case WindowFW.TYPE_ID: + final WindowFW window = windowRO.wrap(buffer, index, index + length); + onGroupInitialWindow(window); + break; + default: + break; + } + } + + private void onGroupReplyBegin( + BeginFW begin) + { + final long traceId = begin.traceId(); + + state = KafkaState.openingReply(state); + + delegate.doGroupReplyBegin(traceId); + } + + private void onGroupReplyData( + DataFW data) + { + final long sequence = data.sequence(); + final long acknowledge = data.acknowledge(); + final long traceId = data.traceId(); + final int flags = data.flags(); + final int reserved = data.reserved(); + final OctetsFW payload = data.payload(); + final OctetsFW extension = data.extension(); + + assert acknowledge <= sequence; + assert sequence >= replySeq; + + replySeq = sequence + reserved; + + assert replyAck <= replySeq; + assert replySeq <= replyAck + replyMax; + + delegate.doGroupReplyData(traceId, flags, reserved, payload, extension); + } + + private void onGroupReplyEnd( + EndFW end) + { + final long sequence = end.sequence(); + final long acknowledge = end.acknowledge(); + final long traceId = end.traceId(); + + assert acknowledge <= sequence; + assert sequence >= replySeq; + + replySeq = sequence; + state = KafkaState.closedReply(state); + + assert replyAck <= replySeq; + + delegate.doGroupReplyEnd(traceId); + } + + private void onGroupReplyAbort( + AbortFW abort) + { + final long sequence = abort.sequence(); + final long acknowledge = abort.acknowledge(); + final long traceId = abort.traceId(); + + assert acknowledge <= sequence; + assert sequence >= replySeq; + + replySeq = sequence; + state = KafkaState.closedReply(state); + + assert replyAck <= replySeq; + + delegate.doGroupReplyAbort(traceId); + } + + private void doGroupReplyReset( + long traceId) + { + if (!KafkaState.replyClosed(state)) + { + doReset(receiver, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization); + + state = KafkaState.closedReply(state); + } + } + + private void doGroupReplyWindow( + long traceId, + long authorization, + long budgetId, + int padding) + { + replyAck = Math.max(delegate.replyAck - replyPad, 0); + replyMax = delegate.replyMax; + + doWindow(receiver, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, budgetId, padding + replyPad); + } + } + + private final class KafkaCacheGroupApp + { + private final KafkaCacheGroupNet group; + private final MessageConsumer sender; + private final String groupId; + private final String protocol; + private final int timeout; + private final long originId; + private final long routedId; + private final long initialId; + private final long replyId; + private final long affinity; + private final long authorization; + + private int state; + + private long replyBudgetId; + + private long initialSeq; + private long initialAck; + private int initialMax; + + private long replySeq; + private long replyAck; + private int replyMax; + private int replyPad; + private long replyBud; + private int replyCap; + + KafkaCacheGroupApp( + MessageConsumer sender, + long originId, + long routedId, + long initialId, + long affinity, + long authorization, + long resolvedId, + String groupId, + String protocol, + int timeout) + { + this.group = new KafkaCacheGroupNet(this, routedId, resolvedId, authorization); + this.sender = sender; + this.originId = originId; + this.routedId = routedId; + this.initialId = initialId; + this.replyId = supplyReplyId.applyAsLong(initialId); + this.affinity = affinity; + this.authorization = authorization; + this.groupId = groupId; + this.protocol = protocol; + this.timeout = timeout; + } + + private void onGroupMessage( + int msgTypeId, + DirectBuffer buffer, + int index, + int length) + { + switch (msgTypeId) + { + case BeginFW.TYPE_ID: + final BeginFW begin = beginRO.wrap(buffer, index, index + length); + onGroupInitialBegin(begin); + break; + case DataFW.TYPE_ID: + final DataFW data = dataRO.wrap(buffer, index, index + length); + onGroupInitialData(data); + break; + case EndFW.TYPE_ID: + final EndFW end = endRO.wrap(buffer, index, index + length); + onGroupInitialEnd(end); + break; + case FlushFW.TYPE_ID: + final FlushFW flush = flushRO.wrap(buffer, index, index + length); + onGroupInitialFlush(flush); + break; + case AbortFW.TYPE_ID: + final AbortFW abort = abortRO.wrap(buffer, index, index + length); + onGroupInitialAbort(abort); + break; + case WindowFW.TYPE_ID: + final WindowFW window = windowRO.wrap(buffer, index, index + length); + onGroupReplyWindow(window); + break; + case ResetFW.TYPE_ID: + final ResetFW reset = resetRO.wrap(buffer, index, index + length); + onGroupReplyReset(reset); + break; + default: + break; + } + } + + private void onGroupInitialBegin( + BeginFW begin) + { + final long sequence = begin.sequence(); + final long acknowledge = begin.acknowledge(); + final long traceId = begin.traceId(); + final long authorization = begin.authorization(); + final long affinity = begin.affinity(); + + assert acknowledge <= sequence; + assert sequence >= initialSeq; + assert acknowledge >= initialAck; + + initialSeq = sequence; + initialAck = acknowledge; + state = KafkaState.openingInitial(state); + + assert initialAck <= initialSeq; + + group.doGroupInitialBegin(traceId); + } + + private void onGroupInitialData( + DataFW data) + { + final long sequence = data.sequence(); + final long acknowledge = data.acknowledge(); + final long traceId = data.traceId(); + final long authorization = data.authorization(); + final long budgetId = data.budgetId(); + final int reserved = data.reserved(); + final int flags = data.flags(); + final OctetsFW payload = data.payload(); + final OctetsFW extension = data.extension(); + + assert acknowledge <= sequence; + assert sequence >= initialSeq; + + initialSeq = sequence; + + assert initialAck <= initialSeq; + + group.doGroupInitialData(traceId, authorization, budgetId, reserved, flags, payload, extension); + } + + private void onGroupInitialEnd( + EndFW end) + { + final long sequence = end.sequence(); + final long acknowledge = end.acknowledge(); + final long traceId = end.traceId(); + + assert acknowledge <= sequence; + assert sequence >= initialSeq; + + initialSeq = sequence; + state = KafkaState.closedInitial(state); + + assert initialAck <= initialSeq; + + group.doGroupInitialEnd(traceId); + } + + private void onGroupInitialFlush( + FlushFW flush) + { + final long sequence = flush.sequence(); + final long acknowledge = flush.acknowledge(); + final long traceId = flush.traceId(); + + assert acknowledge <= sequence; + assert sequence >= initialSeq; + + initialSeq = sequence; + state = KafkaState.closedInitial(state); + + assert initialAck <= initialSeq; + + group.doGroupInitialFlush(traceId); + } + + private void onGroupInitialAbort( + AbortFW abort) + { + final long sequence = abort.sequence(); + final long acknowledge = abort.acknowledge(); + final long traceId = abort.traceId(); + + assert acknowledge <= sequence; + assert sequence >= initialSeq; + + initialSeq = sequence; + state = KafkaState.closedInitial(state); + + assert initialAck <= initialSeq; + + group.doGroupInitialAbort(traceId); + } + + private void doGroupInitialReset( + long traceId) + { + if (KafkaState.initialOpening(state) && !KafkaState.initialClosed(state)) + { + state = KafkaState.closedInitial(state); + + doReset(sender, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization); + } + + state = KafkaState.closedInitial(state); + } + + private void doGroupInitialWindow( + long authorization, + long traceId, + long budgetId, + int padding) + { + initialAck = group.initialAck; + initialMax = group.initialMax; + + doWindow(sender, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, budgetId, padding); + } + + private void doGroupReplyBegin( + long traceId) + { + state = KafkaState.openingReply(state); + + doBegin(sender, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, affinity, EMPTY_EXTENSION); + } + + private void doGroupReplyData( + long traceId, + int flag, + int reserved, + OctetsFW payload, + Flyweight extension) + { + + doData(sender, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, replyBudgetId, flag, reserved, payload, extension); + + replySeq += reserved; + } + + private void doGroupReplyEnd( + long traceId) + { + if (KafkaState.replyOpening(state) && !KafkaState.replyClosed(state)) + { + doEnd(sender, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, EMPTY_EXTENSION); + } + + state = KafkaState.closedReply(state); + } + + private void doGroupReplyAbort( + long traceId) + { + if (KafkaState.replyOpening(state) && !KafkaState.replyClosed(state)) + { + doAbort(sender, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, EMPTY_EXTENSION); + } + + state = KafkaState.closedReply(state); + } + + private void onGroupReplyReset( + ResetFW reset) + { + final long sequence = reset.sequence(); + final long acknowledge = reset.acknowledge(); + final int maximum = reset.maximum(); + final long traceId = reset.traceId(); + + assert acknowledge <= sequence; + assert sequence <= replySeq; + assert acknowledge >= replyAck; + assert maximum >= replyMax; + + replyAck = acknowledge; + replyMax = maximum; + state = KafkaState.closedReply(state); + + assert replyAck <= replySeq; + + cleanup(traceId); + } + + private void onGroupReplyWindow( + WindowFW window) + { + final long sequence = window.sequence(); + final long acknowledge = window.acknowledge(); + final int maximum = window.maximum(); + final long traceId = window.traceId(); + final long budgetId = window.budgetId(); + final int padding = window.padding(); + final int capabilities = window.capabilities(); + + assert acknowledge <= sequence; + assert sequence <= replySeq; + assert acknowledge >= replyAck; + assert maximum >= replyMax; + + replyAck = acknowledge; + replyMax = maximum; + replyBud = budgetId; + replyPad = padding; + replyCap = capabilities; + state = KafkaState.openedReply(state); + + assert replyAck <= replySeq; + + group.doGroupReplyWindow(traceId, acknowledge, budgetId, padding); + } + + private void cleanup( + long traceId) + { + doGroupInitialReset(traceId); + doGroupReplyAbort(traceId); + + group.doGroupInitialAbort(traceId); + group.doGroupReplyReset(traceId); + } + } +} diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheServerFactory.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheServerFactory.java index f9950aabb7..1d6ea15226 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheServerFactory.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaCacheServerFactory.java @@ -67,6 +67,8 @@ public KafkaCacheServerFactory( final KafkaCacheServerDescribeFactory cacheDescribeFactory = new KafkaCacheServerDescribeFactory( config, context, bindings::get, supplyCache, supplyCacheRoute); + final KafkaCacheGroupFactory cacheGroupFactory = new KafkaCacheGroupFactory(config, context, bindings::get); + final KafkaCacheServerFetchFactory cacheFetchFactory = new KafkaCacheServerFetchFactory( config, context, bindings::get, supplyCache, supplyCacheRoute); @@ -76,6 +78,7 @@ public KafkaCacheServerFactory( factories.put(KafkaBeginExFW.KIND_BOOTSTRAP, cacheBootstrapFactory); factories.put(KafkaBeginExFW.KIND_META, cacheMetaFactory); factories.put(KafkaBeginExFW.KIND_DESCRIBE, cacheDescribeFactory); + factories.put(KafkaBeginExFW.KIND_GROUP, cacheGroupFactory); factories.put(KafkaBeginExFW.KIND_FETCH, cacheFetchFactory); factories.put(KafkaBeginExFW.KIND_PRODUCE, cacheProduceFactory); diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientFactory.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientFactory.java index 442e015c50..10adeb7c4a 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientFactory.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientFactory.java @@ -58,6 +58,9 @@ public KafkaClientFactory( final KafkaClientDescribeFactory clientDescribeFactory = new KafkaClientDescribeFactory( config, context, bindings::get, accountant::supplyDebitor); + final KafkaClientGroupFactory clientGroupFactory = new KafkaClientGroupFactory( + config, context, bindings::get, accountant::supplyDebitor); + final KafkaClientFetchFactory clientFetchFactory = new KafkaClientFetchFactory( config, context, bindings::get, accountant::supplyDebitor, supplyClientRoute); @@ -70,6 +73,7 @@ public KafkaClientFactory( final Int2ObjectHashMap factories = new Int2ObjectHashMap<>(); factories.put(KafkaBeginExFW.KIND_META, clientMetaFactory); factories.put(KafkaBeginExFW.KIND_DESCRIBE, clientDescribeFactory); + factories.put(KafkaBeginExFW.KIND_GROUP, clientGroupFactory); factories.put(KafkaBeginExFW.KIND_FETCH, clientFetchFactory); factories.put(KafkaBeginExFW.KIND_PRODUCE, clientProduceFactory); factories.put(KafkaBeginExFW.KIND_MERGED, clientMergedFactory); @@ -85,6 +89,9 @@ public void attach( { KafkaBindingConfig kafkaBinding = new KafkaBindingConfig(binding); bindings.put(binding.id, kafkaBinding); + + KafkaClientGroupFactory clientGroupFactory = (KafkaClientGroupFactory) factories.get(KafkaBeginExFW.KIND_GROUP); + clientGroupFactory.onAttached(binding.id); } @Override @@ -92,6 +99,8 @@ public void detach( long bindingId) { bindings.remove(bindingId); + KafkaClientGroupFactory clientGroupFactory = (KafkaClientGroupFactory) factories.get(KafkaBeginExFW.KIND_GROUP); + clientGroupFactory.onDetached(bindingId); } @Override diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientGroupFactory.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientGroupFactory.java new file mode 100644 index 0000000000..d044e0c257 --- /dev/null +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientGroupFactory.java @@ -0,0 +1,3010 @@ +/* + * Copyright 2021-2023 Aklivity Inc. + * + * Aklivity 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 io.aklivity.zilla.runtime.binding.kafka.internal.stream; + +import static io.aklivity.zilla.runtime.binding.kafka.internal.types.ProxyAddressProtocol.STREAM; +import static io.aklivity.zilla.runtime.engine.buffer.BufferPool.NO_SLOT; +import static io.aklivity.zilla.runtime.engine.concurrent.Signaler.NO_CANCEL_ID; +import static java.lang.System.currentTimeMillis; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.function.LongFunction; +import java.util.function.Supplier; + +import org.agrona.DirectBuffer; +import org.agrona.MutableDirectBuffer; +import org.agrona.collections.Long2ObjectHashMap; +import org.agrona.collections.LongLongConsumer; +import org.agrona.collections.Object2ObjectHashMap; +import org.agrona.concurrent.UnsafeBuffer; + +import io.aklivity.zilla.runtime.binding.kafka.internal.KafkaBinding; +import io.aklivity.zilla.runtime.binding.kafka.internal.KafkaConfiguration; +import io.aklivity.zilla.runtime.binding.kafka.internal.config.KafkaBindingConfig; +import io.aklivity.zilla.runtime.binding.kafka.internal.config.KafkaRouteConfig; +import io.aklivity.zilla.runtime.binding.kafka.internal.config.KafkaSaslConfig; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.Flyweight; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.OctetsFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.String16FW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.RequestHeaderFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.ResponseHeaderFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.config.ResourceRequestFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.config.ResourceResponseFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.AssignmentFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.FindCoordinatorRequestFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.FindCoordinatorResponseFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.HeartbeatRequestFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.HeartbeatResponseFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.JoinGroupRequestFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.JoinGroupResponseFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.LeaveGroupRequestFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.LeaveGroupResponseFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.LeaveMemberFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.MemberMetadataFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.ProtocolMetadataFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.SyncGroupRequestFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.codec.group.SyncGroupResponseFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.AbortFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.BeginFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.DataFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.EndFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.ExtensionFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.FlushFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.KafkaBeginExFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.KafkaDataExFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.KafkaGroupBeginExFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.KafkaResetExFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.ProxyBeginExFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.ResetFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.SignalFW; +import io.aklivity.zilla.runtime.binding.kafka.internal.types.stream.WindowFW; +import io.aklivity.zilla.runtime.engine.EngineContext; +import io.aklivity.zilla.runtime.engine.binding.BindingHandler; +import io.aklivity.zilla.runtime.engine.binding.function.MessageConsumer; +import io.aklivity.zilla.runtime.engine.budget.BudgetDebitor; +import io.aklivity.zilla.runtime.engine.buffer.BufferPool; +import io.aklivity.zilla.runtime.engine.concurrent.Signaler; + +public final class KafkaClientGroupFactory extends KafkaClientSaslHandshaker implements BindingHandler +{ + private static final short ERROR_EXISTS = -1; + private static final short ERROR_NONE = 0; + private static final short ERROR_COORDINATOR_NOT_AVAILABLE = 15; + private static final short ERROR_NOT_COORDINATOR_FOR_CONSUMER = 16; + private static final short ERROR_UNKNOWN_MEMBER = 25; + private static final short ERROR_MEMBER_ID_REQUIRED = 79; + private static final short ERROR_REBALANCE_IN_PROGRESS = 27; + private static final short SIGNAL_NEXT_REQUEST = 1; + private static final short FIND_COORDINATOR_API_KEY = 10; + private static final short FIND_COORDINATOR_API_VERSION = 1; + private static final short JOIN_GROUP_API_KEY = 11; + private static final short JOIN_GROUP_VERSION = 5; + private static final short SYNC_GROUP_API_KEY = 14; + private static final short SYNC_GROUP_VERSION = 3; + private static final short LEAVE_GROUP_API_KEY = 13; + private static final short LEAVE_GROUP_VERSION = 3; + private static final short HEARTBEAT_API_KEY = 12; + private static final short HEARTBEAT_VERSION = 3; + + private static final String UNKNOWN_MEMBER_ID = ""; + private static final String HIGHLANDER_PROTOCOL = "highlander"; + private static final byte GROUP_KEY_TYPE = 0x00; + private static final DirectBuffer EMPTY_BUFFER = new UnsafeBuffer(); + private static final OctetsFW EMPTY_OCTETS = new OctetsFW().wrap(EMPTY_BUFFER, 0, 0); + private static final Consumer EMPTY_EXTENSION = ex -> {}; + + private final BeginFW beginRO = new BeginFW(); + private final DataFW dataRO = new DataFW(); + private final EndFW endRO = new EndFW(); + private final FlushFW flushRO = new FlushFW(); + private final AbortFW abortRO = new AbortFW(); + private final ResetFW resetRO = new ResetFW(); + private final WindowFW windowRO = new WindowFW(); + private final SignalFW signalRO = new SignalFW(); + private final ExtensionFW extensionRO = new ExtensionFW(); + private final KafkaBeginExFW kafkaBeginExRO = new KafkaBeginExFW(); + + private final BeginFW.Builder beginRW = new BeginFW.Builder(); + private final DataFW.Builder dataRW = new DataFW.Builder(); + private final EndFW.Builder endRW = new EndFW.Builder(); + private final AbortFW.Builder abortRW = new AbortFW.Builder(); + private final ResetFW.Builder resetRW = new ResetFW.Builder(); + private final WindowFW.Builder windowRW = new WindowFW.Builder(); + private final KafkaBeginExFW.Builder kafkaBeginExRW = new KafkaBeginExFW.Builder(); + private final KafkaDataExFW.Builder kafkaDataExRW = new KafkaDataExFW.Builder(); + private final KafkaResetExFW.Builder kafkaResetExRW = new KafkaResetExFW.Builder(); + private final ProxyBeginExFW.Builder proxyBeginExRW = new ProxyBeginExFW.Builder(); + + private final RequestHeaderFW.Builder requestHeaderRW = new RequestHeaderFW.Builder(); + private final FindCoordinatorRequestFW.Builder findCoordinatorRequestRW = new FindCoordinatorRequestFW.Builder(); + private final JoinGroupRequestFW.Builder joinGroupRequestRW = new JoinGroupRequestFW.Builder(); + private final ProtocolMetadataFW.Builder protocolMetadataRW = new ProtocolMetadataFW.Builder(); + private final SyncGroupRequestFW.Builder syncGroupRequestRW = new SyncGroupRequestFW.Builder(); + private final AssignmentFW.Builder assignmentRW = new AssignmentFW.Builder(); + private final HeartbeatRequestFW.Builder heartbeatRequestRW = new HeartbeatRequestFW.Builder(); + private final LeaveGroupRequestFW.Builder leaveGroupRequestRW = new LeaveGroupRequestFW.Builder(); + private final LeaveMemberFW.Builder leaveMemberRW = new LeaveMemberFW.Builder(); + private final ResourceRequestFW.Builder resourceRequestRW = new ResourceRequestFW.Builder(); + + private final ResponseHeaderFW responseHeaderRO = new ResponseHeaderFW(); + private final FindCoordinatorResponseFW findCoordinatorResponseRO = new FindCoordinatorResponseFW(); + private final JoinGroupResponseFW joinGroupResponseRO = new JoinGroupResponseFW(); + private final MemberMetadataFW memberMetadataRO = new MemberMetadataFW(); + private final SyncGroupResponseFW syncGroupResponseRO = new SyncGroupResponseFW(); + private final HeartbeatResponseFW heartbeatResponseRO = new HeartbeatResponseFW(); + private final LeaveGroupResponseFW leaveGroupResponseRO = new LeaveGroupResponseFW(); + private final LeaveMemberFW leaveMemberRO = new LeaveMemberFW(); + private final ResourceResponseFW resourceResponseRO = new ResourceResponseFW(); + + private final KafkaGroupClusterClientDecoder decodeClusterSaslHandshakeResponse = this::decodeSaslHandshakeResponse; + private final KafkaGroupClusterClientDecoder decodeClusterSaslHandshake = this::decodeSaslHandshake; + private final KafkaGroupClusterClientDecoder decodeClusterSaslHandshakeMechanisms = this::decodeSaslHandshakeMechanisms; + private final KafkaGroupClusterClientDecoder decodeClusterSaslHandshakeMechanism = this::decodeSaslHandshakeMechanism; + private final KafkaGroupClusterClientDecoder decodeClusterSaslAuthenticateResponse = this::decodeSaslAuthenticateResponse; + private final KafkaGroupClusterClientDecoder decodeClusterSaslAuthenticate = this::decodeSaslAuthenticate; + private final KafkaGroupClusterClientDecoder decodeFindCoordinatorResponse = this::decodeFindCoordinatorResponse; + private final KafkaGroupClusterClientDecoder decodeClusterReject = this::decodeClusterReject; + private final KafkaGroupClusterClientDecoder decodeClusterIgnoreAll = this::decodeIgnoreAll; + private final KafkaGroupCoordinatorClientDecoder decodeCoordinatorSaslHandshakeResponse = + this::decodeSaslHandshakeResponse; + private final KafkaGroupCoordinatorClientDecoder decodeCoordinatorSaslHandshake = + this::decodeSaslHandshake; + private final KafkaGroupCoordinatorClientDecoder decodeCoordinatorSaslHandshakeMechanisms = + this::decodeSaslHandshakeMechanisms; + private final KafkaGroupCoordinatorClientDecoder decodeCoordinatorSaslHandshakeMechanism = + this::decodeSaslHandshakeMechanism; + private final KafkaGroupCoordinatorClientDecoder decodeCoordinatorSaslAuthenticateResponse = + this::decodeSaslAuthenticateResponse; + private final KafkaGroupCoordinatorClientDecoder decodeCoordinatorSaslAuthenticate = + this::decodeSaslAuthenticate; + private final KafkaGroupCoordinatorClientDecoder decodeJoinGroupResponse = + this::decodeJoinGroupResponse; + private final KafkaGroupCoordinatorClientDecoder decodeSyncGroupResponse = + this::decodeSyncGroupResponse; + private final KafkaGroupCoordinatorClientDecoder decodeHeartbeatResponse = + this::decodeHeartbeatResponse; + private final KafkaGroupCoordinatorClientDecoder decodeLeaveGroupResponse = + this::decodeLeaveGroupResponse; + private final KafkaGroupCoordinatorClientDecoder decodeCoordinatorIgnoreAll = this::decodeIgnoreAll; + private final KafkaGroupCoordinatorClientDecoder decodeCoordinatorReject = this::decodeCoordinatorReject; + + private final int kafkaTypeId; + private final int proxyTypeId; + private final MutableDirectBuffer writeBuffer; + private final MutableDirectBuffer extBuffer; + private final BufferPool decodePool; + private final BufferPool encodePool; + private final Signaler signaler; + private final BindingHandler streamFactory; + private final LongFunction supplyBinding; + private final Supplier supplyInstanceId; + private final Long2ObjectHashMap instanceIds; + private final Object2ObjectHashMap groupStreams; + private final String clientId; + private final Duration rebalanceTimeout; + + + public KafkaClientGroupFactory( + KafkaConfiguration config, + EngineContext context, + LongFunction supplyBinding, + LongFunction supplyDebitor) + { + super(config, context); + this.kafkaTypeId = context.supplyTypeId(KafkaBinding.NAME); + this.proxyTypeId = context.supplyTypeId("proxy"); + this.signaler = context.signaler(); + this.streamFactory = context.streamFactory(); + this.writeBuffer = new UnsafeBuffer(new byte[context.writeBuffer().capacity()]); + this.extBuffer = new UnsafeBuffer(new byte[context.writeBuffer().capacity()]); + this.decodePool = context.bufferPool(); + this.encodePool = context.bufferPool(); + this.supplyBinding = supplyBinding; + this.rebalanceTimeout = config.clientGroupRebalanceTimeout(); + this.clientId = config.clientId(); + this.supplyInstanceId = config.clientInstanceIdSupplier(); + this.instanceIds = new Long2ObjectHashMap<>(); + this.groupStreams = new Object2ObjectHashMap<>(); + } + + @Override + public MessageConsumer newStream( + int msgTypeId, + DirectBuffer buffer, + int index, + int length, + MessageConsumer application) + { + final BeginFW begin = beginRO.wrap(buffer, index, index + length); + final long originId = begin.originId(); + final long routedId = begin.routedId(); + final long initialId = begin.streamId(); + final long affinity = begin.affinity(); + final long traceId = begin.traceId(); + final long authorization = begin.authorization(); + final OctetsFW extension = begin.extension(); + final ExtensionFW beginEx = extensionRO.tryWrap(extension.buffer(), extension.offset(), extension.limit()); + final KafkaBeginExFW kafkaBeginEx = beginEx != null && beginEx.typeId() == kafkaTypeId ? + kafkaBeginExRO.tryWrap(extension.buffer(), extension.offset(), extension.limit()) : null; + + assert kafkaBeginEx.kind() == KafkaBeginExFW.KIND_GROUP; + final KafkaGroupBeginExFW kafkaGroupBeginEx = kafkaBeginEx.group(); + + MessageConsumer newStream = null; + + final KafkaBindingConfig binding = supplyBinding.apply(routedId); + final KafkaRouteConfig resolved; + final int timeout = kafkaGroupBeginEx.timeout(); + final String groupId = kafkaGroupBeginEx.groupId().asString(); + final String protocol = kafkaGroupBeginEx.protocol().asString(); + + if (binding != null) + { + resolved = binding.resolve(authorization, null, groupId); + + if (resolved != null) + { + final long resolvedId = resolved.id; + final KafkaSaslConfig sasl = binding.sasl(); + + final GroupMembership groupMembership = instanceIds.get(binding.id); + assert groupMembership != null; + + KafkaGroupStream stream = groupStreams.get(groupId); + if (stream == null || HIGHLANDER_PROTOCOL.equals(protocol)) + { + if (stream != null) + { + stream.streamCleanup(traceId, traceId); + } + + KafkaGroupStream group = new KafkaGroupStream( + application, + originId, + routedId, + initialId, + affinity, + resolvedId, + groupId, + protocol, + timeout, + groupMembership, + sasl); + newStream = group::onApplication; + + groupStreams.put(groupId, group); + } + } + } + + return newStream; + } + + public void onAttached( + long bindingId) + { + instanceIds.put(bindingId, new GroupMembership(supplyInstanceId.get())); + } + + public void onDetached( + long bindingId) + { + instanceIds.remove(bindingId); + } + + private MessageConsumer newStream( + MessageConsumer sender, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long affinity, + Consumer extension) + { + final BeginFW begin = beginRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .affinity(affinity) + .extension(extension) + .build(); + + final MessageConsumer receiver = + streamFactory.newStream(begin.typeId(), begin.buffer(), begin.offset(), begin.sizeof(), sender); + + receiver.accept(begin.typeId(), begin.buffer(), begin.offset(), begin.sizeof()); + + return receiver; + } + + private void doBegin( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long affinity, + Consumer extension) + { + final BeginFW begin = beginRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .affinity(affinity) + .extension(extension) + .build(); + + receiver.accept(begin.typeId(), begin.buffer(), begin.offset(), begin.sizeof()); + } + + private void doData( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long budgetId, + int reserved, + DirectBuffer payload, + int offset, + int length, + Consumer extension) + { + final DataFW data = dataRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .budgetId(budgetId) + .reserved(reserved) + .payload(payload, offset, length) + .extension(extension) + .build(); + + receiver.accept(data.typeId(), data.buffer(), data.offset(), data.sizeof()); + } + + private void doDataNull( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long budgetId, + int reserved, + Consumer extension) + { + final DataFW data = dataRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .budgetId(budgetId) + .reserved(reserved) + .extension(extension) + .build(); + + receiver.accept(data.typeId(), data.buffer(), data.offset(), data.sizeof()); + } + + private void doEnd( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + Consumer extension) + { + final EndFW end = endRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .extension(extension) + .build(); + + receiver.accept(end.typeId(), end.buffer(), end.offset(), end.sizeof()); + } + + private void doAbort( + MessageConsumer receiver, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + Consumer extension) + { + final AbortFW abort = abortRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .extension(extension) + .build(); + + receiver.accept(abort.typeId(), abort.buffer(), abort.offset(), abort.sizeof()); + } + + private void doWindow( + MessageConsumer sender, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + long budgetId, + int padding) + { + final WindowFW window = windowRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .budgetId(budgetId) + .padding(padding) + .build(); + + sender.accept(window.typeId(), window.buffer(), window.offset(), window.sizeof()); + } + + private void doReset( + MessageConsumer sender, + long originId, + long routedId, + long streamId, + long sequence, + long acknowledge, + int maximum, + long traceId, + long authorization, + Flyweight extension) + { + final ResetFW reset = resetRW.wrap(writeBuffer, 0, writeBuffer.capacity()) + .originId(originId) + .routedId(routedId) + .streamId(streamId) + .sequence(sequence) + .acknowledge(acknowledge) + .maximum(maximum) + .traceId(traceId) + .authorization(authorization) + .extension(extension.buffer(), extension.offset(), extension.sizeof()) + .build(); + + sender.accept(reset.typeId(), reset.buffer(), reset.offset(), reset.sizeof()); + } + + @FunctionalInterface + private interface KafkaGroupClusterClientDecoder + { + int decode( + ClusterClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + MutableDirectBuffer buffer, + int offset, + int progress, + int limit); + } + + @FunctionalInterface + private interface KafkaGroupCoordinatorClientDecoder + { + int decode( + CoordinatorClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + MutableDirectBuffer buffer, + int offset, + int progress, + int limit); + } + + private int decodeFindCoordinatorResponse( + ClusterClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + DirectBuffer buffer, + int offset, + int progress, + int limit) + { + final int length = limit - progress; + + decode: + if (length != 0) + { + final ResponseHeaderFW responseHeader = responseHeaderRO.tryWrap(buffer, progress, limit); + if (responseHeader == null) + { + client.decoder = decodeClusterIgnoreAll; + break decode; + } + + final int responseSize = responseHeader.length(); + + if (length >= responseHeader.sizeof() + responseSize) + { + progress = responseHeader.limit(); + + final FindCoordinatorResponseFW findCoordinatorResponse = + findCoordinatorResponseRO.tryWrap(buffer, progress, limit); + + if (findCoordinatorResponse == null) + { + client.decoder = decodeClusterIgnoreAll; + break decode; + } + else if (findCoordinatorResponse.errorCode() == ERROR_COORDINATOR_NOT_AVAILABLE) + { + client.onCoordinatorNotAvailable(traceId, authorization); + } + else if (findCoordinatorResponse.errorCode() == ERROR_NONE) + { + client.onFindCoordinator(traceId, authorization, + findCoordinatorResponse.host(), findCoordinatorResponse.port()); + } + else + { + client.decoder = decodeClusterIgnoreAll; + } + + progress = findCoordinatorResponse.limit(); + } + } + + if (client.decoder == decodeClusterIgnoreAll) + { + client.onError(traceId); + } + + return progress; + } + + + private int decodeClusterReject( + ClusterClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + DirectBuffer buffer, + int offset, + int progress, + int limit) + { + client.doNetworkReset(traceId); + client.decoder = decodeClusterIgnoreAll; + return limit; + } + + private int decodeCoordinatorReject( + CoordinatorClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + DirectBuffer buffer, + int offset, + int progress, + int limit) + { + client.doNetworkReset(traceId); + client.decoder = decodeCoordinatorIgnoreAll; + return limit; + } + + private int decodeIgnoreAll( + KafkaSaslClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + DirectBuffer buffer, + int offset, + int progress, + int limit) + { + return limit; + } + + private int decodeJoinGroupResponse( + CoordinatorClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + DirectBuffer buffer, + int offset, + int progress, + int limit) + { + final int length = limit - progress; + + decode: + if (length != 0) + { + final ResponseHeaderFW responseHeader = responseHeaderRO.tryWrap(buffer, progress, limit); + if (responseHeader == null) + { + client.decoder = decodeJoinGroupResponse; + progress = limit; + break decode; + } + + final int responseSize = responseHeader.length(); + + if (length >= responseHeader.sizeof() + responseSize) + { + progress = responseHeader.limit(); + + final JoinGroupResponseFW joinGroupResponse = + joinGroupResponseRO.tryWrap(buffer, progress, limit); + + final short errorCode = joinGroupResponse != null ? joinGroupResponse.errorCode() : ERROR_EXISTS; + + if (joinGroupResponse == null) + { + client.decoder = decodeJoinGroupResponse; + progress = limit; + break decode; + } + else if (errorCode == ERROR_NOT_COORDINATOR_FOR_CONSUMER) + { + client.onNotCoordinatorError(traceId, authorization); + progress = joinGroupResponse.limit(); + } + else if (errorCode == ERROR_UNKNOWN_MEMBER) + { + client.onJoinGroupMemberIdError(traceId, authorization, UNKNOWN_MEMBER_ID); + progress = joinGroupResponse.limit(); + } + else if (errorCode == ERROR_MEMBER_ID_REQUIRED) + { + client.onJoinGroupMemberIdError(traceId, authorization, + joinGroupResponse.memberId().asString()); + progress = joinGroupResponse.limit(); + } + else if (errorCode == ERROR_NONE) + { + progress = joinGroupResponse.limit(); + client.members.clear(); + + client.generationId = joinGroupResponse.generatedId(); + + metadata: + for (int i = 0; i < joinGroupResponse.memberCount(); i++) + { + final MemberMetadataFW memberMetadata = memberMetadataRO.tryWrap(buffer, progress, limit); + if (memberMetadata != null) + { + client.members.add(memberMetadata.memberId().asString()); + progress = memberMetadata.limit(); + } + else + { + break metadata; + } + } + + client.onJoinGroupResponse(traceId, authorization, joinGroupResponse.leader().asString(), + joinGroupResponse.memberId().asString(), errorCode); + } + else + { + client.decoder = decodeCoordinatorIgnoreAll; + break decode; + } + + } + } + + if (client.decoder == decodeCoordinatorIgnoreAll) + { + client.onError(traceId); + } + + return progress; + } + + private int decodeSyncGroupResponse( + CoordinatorClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + DirectBuffer buffer, + int offset, + int progress, + int limit) + { + final int length = limit - progress; + + decode: + if (length != 0) + { + final ResponseHeaderFW responseHeader = responseHeaderRO.tryWrap(buffer, progress, limit); + if (responseHeader == null) + { + client.decoder = decodeCoordinatorIgnoreAll; + break decode; + } + + final int responseSize = responseHeader.length(); + + if (length >= responseHeader.sizeof() + responseSize) + { + progress = responseHeader.limit(); + + final SyncGroupResponseFW syncGroupResponse = + syncGroupResponseRO.tryWrap(buffer, progress, limit); + + final short errorCode = syncGroupResponse != null ? syncGroupResponse.errorCode() : ERROR_EXISTS; + + if (syncGroupResponse == null) + { + client.decoder = decodeCoordinatorIgnoreAll; + break decode; + } + else if (errorCode == ERROR_REBALANCE_IN_PROGRESS) + { + client.onSynGroupRebalance(traceId, authorization); + } + else if (errorCode == ERROR_NONE) + { + client.onSyncGroupResponse(traceId, authorization, syncGroupResponse.assignment()); + } + else + { + client.decoder = decodeCoordinatorIgnoreAll; + break decode; + } + + progress = syncGroupResponse.limit(); + } + } + + if (client.decoder == decodeCoordinatorIgnoreAll) + { + client.onError(traceId); + } + + return progress; + } + + private int decodeHeartbeatResponse( + CoordinatorClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + DirectBuffer buffer, + int offset, + int progress, + int limit) + { + final int length = limit - progress; + + decode: + if (length != 0) + { + final ResponseHeaderFW responseHeader = responseHeaderRO.tryWrap(buffer, progress, limit); + if (responseHeader == null) + { + client.decoder = decodeCoordinatorIgnoreAll; + break decode; + } + + final int responseSize = responseHeader.length(); + + if (length >= responseHeader.sizeof() + responseSize) + { + progress = responseHeader.limit(); + + final HeartbeatResponseFW heartbeatResponse = + heartbeatResponseRO.tryWrap(buffer, progress, limit); + + if (heartbeatResponse == null) + { + client.decoder = decodeCoordinatorIgnoreAll; + break decode; + } + else if (heartbeatResponse.errorCode() == ERROR_REBALANCE_IN_PROGRESS) + { + client.onRebalanceError(traceId, authorization); + } + else if (heartbeatResponse.errorCode() == ERROR_NONE) + { + client.onHeartbeatResponse(traceId, authorization); + } + else + { + client.decoder = decodeCoordinatorIgnoreAll; + break decode; + } + + progress = heartbeatResponse.limit(); + } + } + + if (client.decoder == decodeCoordinatorIgnoreAll) + { + client.onError(traceId); + } + + return progress; + } + + private int decodeLeaveGroupResponse( + CoordinatorClient client, + long traceId, + long authorization, + long budgetId, + int reserved, + DirectBuffer buffer, + int offset, + int progress, + int limit) + { + final int length = limit - progress; + + decode: + if (length != 0) + { + final ResponseHeaderFW responseHeader = responseHeaderRO.tryWrap(buffer, progress, limit); + if (responseHeader == null) + { + client.decoder = decodeCoordinatorIgnoreAll; + break decode; + } + + final int responseSize = responseHeader.length(); + + if (length >= responseHeader.sizeof() + responseSize) + { + progress = responseHeader.limit(); + + final LeaveGroupResponseFW leaveGroupResponse = + leaveGroupResponseRO.tryWrap(buffer, progress, limit); + + if (leaveGroupResponse == null) + { + client.decoder = decodeCoordinatorIgnoreAll; + break decode; + } + else + { + progress = leaveGroupResponse.limit(); + + members: + for (int i = 0; i < leaveGroupResponse.memberCount(); i++) + { + final LeaveMemberFW member = leaveMemberRO.tryWrap(buffer, progress, limit); + if (member != null) + { + progress = member.limit(); + } + else + { + break members; + } + } + + client.onLeaveGroupResponse(traceId, authorization); + } + } + } + + if (client.decoder == decodeCoordinatorIgnoreAll) + { + client.onError(traceId); + } + + return progress; + } + + private final class KafkaGroupStream + { + private final MessageConsumer application; + private final ClusterClient clusterClient; + private final CoordinatorClient coordinatorClient; + private final GroupMembership groupMembership; + private final String groupId; + private final String protocol; + private final int timeout; + private final long originId; + private final long routedId; + private final long initialId; + private final long replyId; + private final long affinity; + private final long resolvedId; + private final KafkaSaslConfig sasl; + + private int state; + + private long initialSeq; + private long initialAck; + private int initialMax; + + private long replySeq; + private long replyAck; + private int replyMax; + private int replyPad; + + private long replyBudgetId; + + KafkaGroupStream( + MessageConsumer application, + long originId, + long routedId, + long initialId, + long affinity, + long resolvedId, + String groupId, + String protocol, + int timeout, + GroupMembership groupMembership, + KafkaSaslConfig sasl) + { + this.application = application; + this.originId = originId; + this.routedId = routedId; + this.initialId = initialId; + this.replyId = supplyReplyId.applyAsLong(initialId); + this.affinity = affinity; + this.groupId = groupId; + this.protocol = protocol; + this.timeout = timeout; + this.resolvedId = resolvedId; + this.groupMembership = groupMembership; + this.sasl = sasl; + this.clusterClient = new ClusterClient(routedId, resolvedId, sasl, this); + this.coordinatorClient = new CoordinatorClient(routedId, resolvedId, sasl, this); + } + + private void onApplication( + int msgTypeId, + DirectBuffer buffer, + int index, + int length) + { + switch (msgTypeId) + { + case BeginFW.TYPE_ID: + final BeginFW begin = beginRO.wrap(buffer, index, index + length); + onApplicationBegin(begin); + break; + case DataFW.TYPE_ID: + final DataFW data = dataRO.wrap(buffer, index, index + length); + onApplicationData(data); + break; + case FlushFW.TYPE_ID: + final FlushFW flush = flushRO.wrap(buffer, index, index + length); + onApplicationFlush(flush); + break; + case EndFW.TYPE_ID: + final EndFW end = endRO.wrap(buffer, index, index + length); + onApplicationEnd(end); + break; + case AbortFW.TYPE_ID: + final AbortFW abort = abortRO.wrap(buffer, index, index + length); + onApplicationAbort(abort); + break; + case WindowFW.TYPE_ID: + final WindowFW window = windowRO.wrap(buffer, index, index + length); + onApplicationWindow(window); + break; + case ResetFW.TYPE_ID: + final ResetFW reset = resetRO.wrap(buffer, index, index + length); + onApplicationReset(reset); + break; + default: + break; + } + } + + private void onApplicationBegin( + BeginFW begin) + { + final long traceId = begin.traceId(); + final long authorization = begin.authorization(); + + state = KafkaState.openingInitial(state); + + clusterClient.doNetworkBeginIfNecessary(traceId, authorization, affinity); + doApplicationWindow(traceId, 0L, 0, 0, 0); + } + + private void onApplicationData( + DataFW data) + { + final long traceId = data.traceId(); + final long budgetId = data.budgetId(); + + coordinatorClient.doSyncRequest(traceId, budgetId, data.payload()); + } + + private void onApplicationEnd( + EndFW end) + { + final long traceId = end.traceId(); + final long authorization = end.authorization(); + + state = KafkaState.closingInitial(state); + coordinatorClient.doLeaveGroupRequest(traceId); + } + + private void onApplicationFlush( + FlushFW flush) + { + final long traceId = flush.traceId(); + + coordinatorClient.doHeartbeat(traceId); + } + + private void onApplicationAbort( + AbortFW abort) + { + final long traceId = abort.traceId(); + final long authorization = abort.authorization(); + + state = KafkaState.closedInitial(state); + + clusterClient.doNetworkAbort(traceId); + coordinatorClient.doNetworkAbort(traceId); + + cleanupApplication(traceId, EMPTY_OCTETS); + } + + private void onApplicationWindow( + WindowFW window) + { + final long sequence = window.sequence(); + final long acknowledge = window.acknowledge(); + final int maximum = window.maximum(); + final long budgetId = window.budgetId(); + final int padding = window.padding(); + + assert acknowledge <= sequence; + assert sequence <= replySeq; + assert acknowledge >= replyAck; + assert maximum >= replyMax; + + this.replyAck = acknowledge; + this.replyMax = maximum; + this.replyPad = padding; + this.replyBudgetId = budgetId; + + assert replyAck <= replySeq; + } + + private void onApplicationReset( + ResetFW reset) + { + final long traceId = reset.traceId(); + + state = KafkaState.closedInitial(state); + + clusterClient.doNetworkReset(traceId); + } + + private boolean isApplicationReplyOpen() + { + return KafkaState.replyOpening(state); + } + + private void doApplicationBeginIfNecessary( + long traceId, + long authorization) + { + if (!KafkaState.replyOpening(state)) + { + doApplicationBegin(traceId, authorization); + } + } + + private void doApplicationBegin( + long traceId, + long authorization) + { + state = KafkaState.openingReply(state); + + doBegin(application, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, affinity, EMPTY_EXTENSION); + } + + private void doApplicationData( + long traceId, + long authorization, + OctetsFW payload, + Consumer extension) + { + final int reserved = replyPad; + + if (payload.sizeof() > 0) + { + doData(application, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, replyBudgetId, reserved, + payload.value(), payload.offset(), payload.sizeof(), extension); + } + else + { + doDataNull(application, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, replyBudgetId, reserved, extension); + } + + replySeq += reserved; + + assert replyAck <= replySeq; + } + + private void doApplicationEnd( + long traceId) + { + if (!KafkaState.replyClosed(state)) + { + state = KafkaState.closedReply(state); + doEnd(application, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, 0, EMPTY_EXTENSION); + } + } + + private void doApplicationAbort( + long traceId) + { + if (!KafkaState.replyClosed(state)) + { + state = KafkaState.closedReply(state); + doAbort(application, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, 0, EMPTY_EXTENSION); + } + } + + private void doApplicationWindow( + long traceId, + long budgetId, + int minInitialNoAck, + int minInitialPad, + int minInitialMax) + { + final long newInitialAck = Math.max(initialSeq - minInitialNoAck, initialAck); + + if (newInitialAck > initialAck || minInitialMax > initialMax || !KafkaState.initialOpened(state)) + { + initialAck = newInitialAck; + assert initialAck <= initialSeq; + + initialMax = minInitialMax; + + state = KafkaState.openedInitial(state); + + doWindow(application, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, clusterClient.authorization, budgetId, minInitialPad); + } + } + + private void doApplicationReset( + long traceId, + Flyweight extension) + { + state = KafkaState.closedInitial(state); + + doReset(application, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, clusterClient.authorization, extension); + } + + private void doApplicationAbortIfNecessary( + long traceId) + { + if (KafkaState.replyOpening(state) && !KafkaState.replyClosed(state)) + { + doApplicationAbort(traceId); + } + } + + private void doApplicationResetIfNecessary( + long traceId, + Flyweight extension) + { + if (KafkaState.initialOpening(state) && !KafkaState.initialClosed(state)) + { + doApplicationReset(traceId, extension); + } + } + + private void onNotCoordinatorError( + long traceId, + long authority) + { + clusterClient.doNetworkBeginIfNecessary(traceId, authority, affinity); + } + + private void cleanupApplication( + long traceId, + int error) + { + final KafkaResetExFW kafkaResetEx = kafkaResetExRW.wrap(extBuffer, 0, extBuffer.capacity()) + .typeId(kafkaTypeId) + .error(error) + .build(); + + cleanupApplication(traceId, kafkaResetEx); + } + + private void cleanupApplication( + long traceId, + Flyweight extension) + { + doApplicationResetIfNecessary(traceId, extension); + doApplicationAbortIfNecessary(traceId); + + groupStreams.remove(groupId); + } + + private void streamCleanup( + long traceId, + long authorizationId) + { + cleanupApplication(traceId, EMPTY_OCTETS); + clusterClient.cleanupNetwork(traceId, authorizationId); + coordinatorClient.cleanupNetwork(traceId, authorizationId); + } + } + + private final class ClusterClient extends KafkaSaslClient + { + private final LongLongConsumer encodeSaslHandshakeRequest = this::doEncodeSaslHandshakeRequest; + private final LongLongConsumer encodeSaslAuthenticateRequest = this::doEncodeSaslAuthenticateRequest; + private final LongLongConsumer encodeFindCoordinatorRequest = this::doEncodeFindCoordinatorRequest; + private final KafkaGroupStream delegate; + + private MessageConsumer network; + + private int state; + private long authorization; + + private long initialSeq; + private long initialAck; + private int initialMax; + private int initialPad; + private long initialBudgetId; + + private long replySeq; + private long replyAck; + private int replyMax; + + private int encodeSlot = NO_SLOT; + private int encodeSlotOffset; + private long encodeSlotTraceId; + + private int decodeSlot = NO_SLOT; + private int decodeSlotOffset; + private int decodeSlotReserved; + + private int nextResponseId; + + private KafkaGroupClusterClientDecoder decoder; + private LongLongConsumer encoder; + + ClusterClient( + long originId, + long routedId, + KafkaSaslConfig sasl, + KafkaGroupStream delegate) + { + super(sasl, originId, routedId); + + this.encoder = sasl != null ? encodeSaslHandshakeRequest : encodeFindCoordinatorRequest; + this.delegate = delegate; + this.decoder = decodeClusterReject; + } + + private void onNetwork( + int msgTypeId, + DirectBuffer buffer, + int index, + int length) + { + switch (msgTypeId) + { + case BeginFW.TYPE_ID: + final BeginFW begin = beginRO.wrap(buffer, index, index + length); + onNetworkBegin(begin); + break; + case DataFW.TYPE_ID: + final DataFW data = dataRO.wrap(buffer, index, index + length); + onNetworkData(data); + break; + case EndFW.TYPE_ID: + final EndFW end = endRO.wrap(buffer, index, index + length); + onNetworkEnd(end); + break; + case AbortFW.TYPE_ID: + final AbortFW abort = abortRO.wrap(buffer, index, index + length); + onNetworkAbort(abort); + break; + case ResetFW.TYPE_ID: + final ResetFW reset = resetRO.wrap(buffer, index, index + length); + onNetworkReset(reset); + break; + case WindowFW.TYPE_ID: + final WindowFW window = windowRO.wrap(buffer, index, index + length); + onNetworkWindow(window); + break; + case SignalFW.TYPE_ID: + final SignalFW signal = signalRO.wrap(buffer, index, index + length); + onNetworkSignal(signal); + break; + default: + break; + } + } + + private void onNetworkBegin( + BeginFW begin) + { + final long traceId = begin.traceId(); + + authorization = begin.authorization(); + state = KafkaState.openingReply(state); + + doNetworkWindow(traceId, 0L, 0, 0, decodePool.slotCapacity()); + } + + private void onNetworkData( + DataFW data) + { + final long sequence = data.sequence(); + final long acknowledge = data.acknowledge(); + final long traceId = data.traceId(); + final long budgetId = data.budgetId(); + + assert acknowledge <= sequence; + assert sequence >= replySeq; + + replySeq = sequence + data.reserved(); + authorization = data.authorization(); + + assert replyAck <= replySeq; + + if (replySeq > replyAck + replyMax) + { + onError(traceId); + } + else + { + if (decodeSlot == NO_SLOT) + { + decodeSlot = decodePool.acquire(initialId); + } + + if (decodeSlot == NO_SLOT) + { + onError(traceId); + } + else + { + final OctetsFW payload = data.payload(); + int reserved = data.reserved(); + int offset = payload.offset(); + int limit = payload.limit(); + + final MutableDirectBuffer buffer = decodePool.buffer(decodeSlot); + buffer.putBytes(decodeSlotOffset, payload.buffer(), offset, limit - offset); + decodeSlotOffset += limit - offset; + decodeSlotReserved += reserved; + + offset = 0; + limit = decodeSlotOffset; + reserved = decodeSlotReserved; + + decodeNetwork(traceId, authorization, budgetId, reserved, buffer, offset, limit); + } + } + } + + private void onNetworkEnd( + EndFW end) + { + final long traceId = end.traceId(); + + state = KafkaState.closedReply(state); + + cleanupDecodeSlotIfNecessary(); + + if (!delegate.isApplicationReplyOpen()) + { + onError(traceId); + } + else if (decodeSlot == NO_SLOT) + { + delegate.doApplicationEnd(traceId); + } + } + + private void onNetworkAbort( + AbortFW abort) + { + final long traceId = abort.traceId(); + + state = KafkaState.closedReply(state); + + onError(traceId); + } + + private void onNetworkReset( + ResetFW reset) + { + final long traceId = reset.traceId(); + + state = KafkaState.closedInitial(state); + + onError(traceId); + } + + private void onNetworkWindow( + WindowFW window) + { + final long sequence = window.sequence(); + final long acknowledge = window.acknowledge(); + final int maximum = window.maximum(); + final long traceId = window.traceId(); + final long budgetId = window.budgetId(); + final int padding = window.padding(); + + assert acknowledge <= sequence; + assert sequence <= initialSeq; + assert acknowledge >= initialAck; + assert maximum + acknowledge >= initialMax + initialAck; + + this.initialAck = acknowledge; + this.initialMax = maximum; + this.initialPad = padding; + this.initialBudgetId = budgetId; + + assert initialAck <= initialSeq; + + this.authorization = window.authorization(); + + state = KafkaState.openedInitial(state); + + if (encodeSlot != NO_SLOT) + { + final MutableDirectBuffer buffer = encodePool.buffer(encodeSlot); + final int limit = encodeSlotOffset; + + encodeNetwork(encodeSlotTraceId, authorization, budgetId, buffer, 0, limit); + } + + doEncodeRequestIfNecessary(traceId, budgetId); + } + + private void onNetworkSignal( + SignalFW signal) + { + final long traceId = signal.traceId(); + final int signalId = signal.signalId(); + + if (signalId == SIGNAL_NEXT_REQUEST) + { + doEncodeRequestIfNecessary(traceId, initialBudgetId); + } + } + + private void doNetworkBeginIfNecessary( + long traceId, + long authorization, + long affinity) + { + if (KafkaState.closed(state)) + { + replyAck = 0; + replySeq = 0; + state = 0; + } + + if (!KafkaState.initialOpening(state)) + { + doNetworkBegin(traceId, authorization, affinity); + } + } + + private void doNetworkBegin( + long traceId, + long authorization, + long affinity) + { + assert state == 0; + + this.initialId = supplyInitialId.applyAsLong(routedId); + this.replyId = supplyReplyId.applyAsLong(initialId); + + state = KafkaState.openingInitial(state); + + network = newStream(this::onNetwork, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, affinity, EMPTY_EXTENSION); + } + + @Override + protected void doNetworkData( + long traceId, + long budgetId, + DirectBuffer buffer, + int offset, + int limit) + { + if (encodeSlot != NO_SLOT) + { + final MutableDirectBuffer encodeBuffer = encodePool.buffer(encodeSlot); + encodeBuffer.putBytes(encodeSlotOffset, buffer, offset, limit - offset); + encodeSlotOffset += limit - offset; + encodeSlotTraceId = traceId; + + buffer = encodeBuffer; + offset = 0; + limit = encodeSlotOffset; + } + + encodeNetwork(traceId, authorization, budgetId, buffer, offset, limit); + } + + private void doNetworkEnd( + long traceId, + long authorization) + { + if (!KafkaState.initialClosed(state)) + { + state = KafkaState.closedInitial(state); + + doEnd(network, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, EMPTY_EXTENSION); + } + + cleanupEncodeSlotIfNecessary(); + } + + private void doNetworkAbort( + long traceId) + { + if (!KafkaState.initialClosed(state)) + { + doAbort(network, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, EMPTY_EXTENSION); + state = KafkaState.closedInitial(state); + } + + cleanupEncodeSlotIfNecessary(); + } + + private void doNetworkReset( + long traceId) + { + if (!KafkaState.replyClosed(state)) + { + doReset(network, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, EMPTY_OCTETS); + state = KafkaState.closedReply(state); + } + + cleanupDecodeSlotIfNecessary(); + } + + private void doNetworkWindow( + long traceId, + long budgetId, + int minReplyNoAck, + int minReplyPad, + int minReplyMax) + { + final long newReplyAck = Math.max(replySeq - minReplyNoAck, replyAck); + + if (newReplyAck > replyAck || minReplyMax > replyMax || !KafkaState.replyOpened(state)) + { + replyAck = newReplyAck; + assert replyAck <= replySeq; + + replyMax = minReplyMax; + + state = KafkaState.openedReply(state); + + doWindow(network, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, budgetId, minReplyPad); + } + } + + private void doEncodeRequestIfNecessary( + long traceId, + long budgetId) + { + if (nextRequestId == nextResponseId) + { + encoder.accept(traceId, budgetId); + } + } + + private void doEncodeFindCoordinatorRequest( + long traceId, + long budgetId) + { + final MutableDirectBuffer encodeBuffer = writeBuffer; + final int encodeOffset = DataFW.FIELD_OFFSET_PAYLOAD; + final int encodeLimit = encodeBuffer.capacity(); + + int encodeProgress = encodeOffset; + + final RequestHeaderFW requestHeader = requestHeaderRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .length(0) + .apiKey(FIND_COORDINATOR_API_KEY) + .apiVersion(FIND_COORDINATOR_API_VERSION) + .correlationId(0) + .clientId(clientId) + .build(); + + encodeProgress = requestHeader.limit(); + + final FindCoordinatorRequestFW findCoordinatorRequest = + findCoordinatorRequestRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .key(delegate.groupId) + .keyType(GROUP_KEY_TYPE) + .build(); + + encodeProgress = findCoordinatorRequest.limit(); + + final int requestId = nextRequestId++; + final int requestSize = encodeProgress - encodeOffset - RequestHeaderFW.FIELD_OFFSET_API_KEY; + + requestHeaderRW.wrap(encodeBuffer, requestHeader.offset(), requestHeader.limit()) + .length(requestSize) + .apiKey(requestHeader.apiKey()) + .apiVersion(requestHeader.apiVersion()) + .correlationId(requestId) + .clientId(requestHeader.clientId().asString()) + .build(); + + doNetworkData(traceId, budgetId, encodeBuffer, encodeOffset, encodeProgress); + + decoder = decodeFindCoordinatorResponse; + } + + private void encodeNetwork( + long traceId, + long authorization, + long budgetId, + DirectBuffer buffer, + int offset, + int limit) + { + final int maxLength = limit - offset; + final int initialWin = initialMax - (int)(initialSeq - initialAck); + final int length = Math.max(Math.min(initialWin - initialPad, maxLength), 0); + + if (length > 0) + { + final int reserved = length + initialPad; + + doData(network, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, budgetId, reserved, buffer, offset, length, EMPTY_EXTENSION); + + initialSeq += reserved; + + assert initialAck <= initialSeq; + } + + final int remaining = maxLength - length; + if (remaining > 0) + { + if (encodeSlot == NO_SLOT) + { + encodeSlot = encodePool.acquire(initialId); + } + + if (encodeSlot == NO_SLOT) + { + onError(traceId); + } + else + { + final MutableDirectBuffer encodeBuffer = encodePool.buffer(encodeSlot); + encodeBuffer.putBytes(0, buffer, offset + length, remaining); + encodeSlotOffset = remaining; + } + } + else + { + cleanupEncodeSlotIfNecessary(); + } + } + + private void decodeNetwork( + long traceId, + long authorization, + long budgetId, + int reserved, + MutableDirectBuffer buffer, + int offset, + int limit) + { + KafkaGroupClusterClientDecoder previous = null; + int progress = offset; + while (progress <= limit && previous != decoder) + { + previous = decoder; + progress = decoder.decode(this, traceId, authorization, budgetId, reserved, buffer, offset, progress, limit); + } + + if (progress < limit) + { + if (decodeSlot == NO_SLOT) + { + decodeSlot = decodePool.acquire(initialId); + } + + if (decodeSlot == NO_SLOT) + { + onError(traceId); + } + else + { + final MutableDirectBuffer decodeBuffer = decodePool.buffer(decodeSlot); + decodeBuffer.putBytes(0, buffer, progress, limit - progress); + decodeSlotOffset = limit - progress; + decodeSlotReserved = (limit - progress) * reserved / (limit - offset); + } + + doNetworkWindow(traceId, budgetId, decodeSlotOffset, 0, replyMax); + } + else + { + cleanupDecodeSlotIfNecessary(); + + if (reserved > 0) + { + doNetworkWindow(traceId, budgetId, 0, 0, replyMax); + } + } + } + + @Override + protected void doDecodeSaslHandshakeResponse( + long traceId) + { + decoder = decodeClusterSaslHandshakeResponse; + } + + @Override + protected void doDecodeSaslHandshake( + long traceId) + { + decoder = decodeClusterSaslHandshake; + } + + @Override + protected void doDecodeSaslHandshakeMechanisms( + long traceId) + { + decoder = decodeClusterSaslHandshakeMechanisms; + } + + @Override + protected void doDecodeSaslHandshakeMechansim( + long traceId) + { + decoder = decodeClusterSaslHandshakeMechanism; + } + + @Override + protected void doDecodeSaslAuthenticateResponse( + long traceId) + { + decoder = decodeClusterSaslAuthenticateResponse; + } + + @Override + protected void doDecodeSaslAuthenticate( + long traceId) + { + decoder = decodeClusterSaslAuthenticate; + } + + @Override + protected void onDecodeSaslHandshakeResponse( + long traceId, + long authorization, + int errorCode) + { + switch (errorCode) + { + case ERROR_NONE: + encoder = encodeSaslAuthenticateRequest; + decoder = decodeClusterSaslAuthenticateResponse; + break; + default: + delegate.cleanupApplication(traceId, errorCode); + doNetworkEnd(traceId, authorization); + break; + } + } + + @Override + protected void onDecodeSaslAuthenticateResponse( + long traceId, + long authorization, + int errorCode) + { + switch (errorCode) + { + case ERROR_NONE: + encoder = encodeFindCoordinatorRequest; + decoder = decodeFindCoordinatorResponse; + break; + default: + delegate.cleanupApplication(traceId, errorCode); + doNetworkEnd(traceId, authorization); + break; + } + } + + @Override + protected void onDecodeSaslResponse( + long traceId) + { + nextResponseId++; + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void onCoordinatorNotAvailable( + long traceId, + long authorization) + { + nextResponseId++; + + encoder = encodeFindCoordinatorRequest; + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void onFindCoordinator( + long traceId, + long authorization, + String16FW host, + int port) + { + nextResponseId++; + + delegate.coordinatorClient.doNetworkBeginIfNecessary(traceId, authorization, 0, host, port); + + cleanupNetwork(traceId, authorization); + } + + private void cleanupNetwork( + long traceId, + long authorization) + { + replySeq = 0; + replyAck = 0; + + doNetworkEnd(traceId, authorization); + doNetworkReset(traceId); + } + + private void onError( + long traceId) + { + doNetworkAbort(traceId); + doNetworkReset(traceId); + + delegate.cleanupApplication(traceId, EMPTY_OCTETS); + } + + private void cleanupDecodeSlotIfNecessary() + { + if (decodeSlot != NO_SLOT) + { + decodePool.release(decodeSlot); + decodeSlot = NO_SLOT; + decodeSlotOffset = 0; + decodeSlotReserved = 0; + } + } + + private void cleanupEncodeSlotIfNecessary() + { + if (encodeSlot != NO_SLOT) + { + encodePool.release(encodeSlot); + encodeSlot = NO_SLOT; + encodeSlotOffset = 0; + encodeSlotTraceId = 0; + } + } + } + + private final class CoordinatorClient extends KafkaSaslClient + { + private final LongLongConsumer encodeSaslHandshakeRequest = this::doEncodeSaslHandshakeRequest; + private final LongLongConsumer encodeSaslAuthenticateRequest = this::doEncodeSaslAuthenticateRequest; + private final LongLongConsumer encodeJoinGroupRequest = this::doEncodeJoinGroupRequest; + private final LongLongConsumer encodeSyncGroupRequest = this::doEncodeSyncGroupRequest; + private final LongLongConsumer encodeHeartbeatRequest = this::doEncodeHeartbeatRequest; + private final LongLongConsumer encodeLeaveGroupRequest = this::doEncodeLeaveGroupRequest; + private final List members; + private final KafkaGroupStream delegate; + + private MessageConsumer network; + + private int state; + private long authorization; + + private long initialSeq; + private long initialAck; + private int initialMax; + private int initialPad; + private long initialBudgetId; + + private long replySeq; + private long replyAck; + private int replyMax; + + private int encodeSlot = NO_SLOT; + private int encodeSlotOffset; + private long encodeSlotTraceId; + + private int decodeSlot = NO_SLOT; + private int decodeSlotOffset; + private int decodeSlotReserved; + + private int nextResponseId; + private long heartbeatRequestId = NO_CANCEL_ID; + + private String leader; + + private int generationId; + private KafkaGroupCoordinatorClientDecoder decoder; + private LongLongConsumer encoder; + private OctetsFW assignment = EMPTY_OCTETS; + + CoordinatorClient( + long originId, + long routedId, + KafkaSaslConfig sasl, + KafkaGroupStream delegate) + { + super(sasl, originId, routedId); + + this.encoder = sasl != null ? encodeSaslHandshakeRequest : encodeJoinGroupRequest; + this.delegate = delegate; + this.decoder = decodeCoordinatorReject; + this.members = new ArrayList<>(); + } + + private void onNetwork( + int msgTypeId, + DirectBuffer buffer, + int index, + int length) + { + switch (msgTypeId) + { + case BeginFW.TYPE_ID: + final BeginFW begin = beginRO.wrap(buffer, index, index + length); + onNetworkBegin(begin); + break; + case DataFW.TYPE_ID: + final DataFW data = dataRO.wrap(buffer, index, index + length); + onNetworkData(data); + break; + case EndFW.TYPE_ID: + final EndFW end = endRO.wrap(buffer, index, index + length); + onNetworkEnd(end); + break; + case AbortFW.TYPE_ID: + final AbortFW abort = abortRO.wrap(buffer, index, index + length); + onNetworkAbort(abort); + break; + case ResetFW.TYPE_ID: + final ResetFW reset = resetRO.wrap(buffer, index, index + length); + onNetworkReset(reset); + break; + case WindowFW.TYPE_ID: + final WindowFW window = windowRO.wrap(buffer, index, index + length); + onNetworkWindow(window); + break; + case SignalFW.TYPE_ID: + final SignalFW signal = signalRO.wrap(buffer, index, index + length); + onNetworkSignal(signal); + break; + default: + break; + } + } + + private void onNetworkBegin( + BeginFW begin) + { + final long traceId = begin.traceId(); + + authorization = begin.authorization(); + state = KafkaState.openingReply(state); + + doNetworkWindow(traceId, 0L, 0, 0, decodePool.slotCapacity()); + } + + private void onNetworkData( + DataFW data) + { + final long sequence = data.sequence(); + final long acknowledge = data.acknowledge(); + final long traceId = data.traceId(); + final long budgetId = data.budgetId(); + + assert acknowledge <= sequence; + assert sequence >= replySeq; + + replySeq = sequence + data.reserved(); + authorization = data.authorization(); + + assert replyAck <= replySeq; + + if (replySeq > replyAck + replyMax) + { + onError(traceId); + } + else + { + if (decodeSlot == NO_SLOT) + { + decodeSlot = decodePool.acquire(initialId); + } + + if (decodeSlot == NO_SLOT) + { + onError(traceId); + } + else + { + final OctetsFW payload = data.payload(); + int reserved = data.reserved(); + int offset = payload.offset(); + int limit = payload.limit(); + + final MutableDirectBuffer buffer = decodePool.buffer(decodeSlot); + buffer.putBytes(decodeSlotOffset, payload.buffer(), offset, limit - offset); + decodeSlotOffset += limit - offset; + decodeSlotReserved += reserved; + + offset = 0; + limit = decodeSlotOffset; + reserved = decodeSlotReserved; + + decodeNetwork(traceId, authorization, budgetId, reserved, buffer, offset, limit); + } + } + } + + private void onNetworkEnd( + EndFW end) + { + final long traceId = end.traceId(); + + state = KafkaState.closedReply(state); + + cleanupDecodeSlotIfNecessary(); + + if (!delegate.isApplicationReplyOpen()) + { + onError(traceId); + } + } + + private void onNetworkAbort( + AbortFW abort) + { + final long traceId = abort.traceId(); + + state = KafkaState.closedReply(state); + + onError(traceId); + } + + private void onNetworkReset( + ResetFW reset) + { + final long traceId = reset.traceId(); + + state = KafkaState.closedInitial(state); + + onError(traceId); + } + + private void onNetworkWindow( + WindowFW window) + { + final long sequence = window.sequence(); + final long acknowledge = window.acknowledge(); + final int maximum = window.maximum(); + final long traceId = window.traceId(); + final long budgetId = window.budgetId(); + final int padding = window.padding(); + + assert acknowledge <= sequence; + assert sequence <= initialSeq; + assert acknowledge >= initialAck; + assert maximum + acknowledge >= initialMax + initialAck; + + this.initialAck = acknowledge; + this.initialMax = maximum; + this.initialPad = padding; + this.initialBudgetId = budgetId; + + assert initialAck <= initialSeq; + + this.authorization = window.authorization(); + + state = KafkaState.openedInitial(state); + + if (encodeSlot != NO_SLOT) + { + final MutableDirectBuffer buffer = encodePool.buffer(encodeSlot); + final int limit = encodeSlotOffset; + + encodeNetwork(encodeSlotTraceId, authorization, budgetId, buffer, 0, limit); + } + + doEncodeRequestIfNecessary(traceId, budgetId); + } + + private void onNetworkSignal( + SignalFW signal) + { + final long traceId = signal.traceId(); + final int signalId = signal.signalId(); + + if (signalId == SIGNAL_NEXT_REQUEST) + { + doEncodeRequestIfNecessary(traceId, initialBudgetId); + } + } + + private void doNetworkBeginIfNecessary( + long traceId, + long authorization, + long affinity, + String16FW host, + int port) + { + if (KafkaState.closed(state)) + { + replyAck = 0; + replySeq = 0; + state = 0; + } + + if (!KafkaState.initialOpening(state)) + { + doNetworkBegin(traceId, authorization, affinity, host, port); + } + } + + private void doNetworkBegin( + long traceId, + long authorization, + long affinity, + String16FW host, + int port) + { + this.initialId = supplyInitialId.applyAsLong(routedId); + this.replyId = supplyReplyId.applyAsLong(initialId); + + state = KafkaState.openingInitial(state); + + Consumer extension = e -> e.set((b, o, l) -> proxyBeginExRW.wrap(b, o, l) + .typeId(proxyTypeId) + .address(a -> a.inet(i -> i.protocol(p -> p.set(STREAM)) + .source("0.0.0.0") + .destination(host) + .sourcePort(0) + .destinationPort(port))) + .build() + .sizeof()); + + network = newStream(this::onNetwork, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, affinity, extension); + } + + @Override + protected void doNetworkData( + long traceId, + long budgetId, + DirectBuffer buffer, + int offset, + int limit) + { + if (encodeSlot != NO_SLOT) + { + final MutableDirectBuffer encodeBuffer = encodePool.buffer(encodeSlot); + encodeBuffer.putBytes(encodeSlotOffset, buffer, offset, limit - offset); + encodeSlotOffset += limit - offset; + encodeSlotTraceId = traceId; + + buffer = encodeBuffer; + offset = 0; + limit = encodeSlotOffset; + } + + encodeNetwork(traceId, authorization, budgetId, buffer, offset, limit); + } + + private void doNetworkEnd( + long traceId, + long authorization) + { + if (!KafkaState.initialClosed(state)) + { + state = KafkaState.closedInitial(state); + + doEnd(network, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, EMPTY_EXTENSION); + } + + cleanupEncodeSlotIfNecessary(); + + } + + private void doNetworkAbort( + long traceId) + { + if (KafkaState.initialOpened(state) && + !KafkaState.initialClosed(state)) + { + doAbort(network, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, EMPTY_EXTENSION); + state = KafkaState.closedInitial(state); + } + + cleanupEncodeSlotIfNecessary(); + } + + private void doNetworkReset( + long traceId) + { + if (!KafkaState.replyClosed(state)) + { + doReset(network, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, EMPTY_OCTETS); + state = KafkaState.closedReply(state); + } + + cleanupDecodeSlotIfNecessary(); + } + + private void doNetworkWindow( + long traceId, + long budgetId, + int minReplyNoAck, + int minReplyPad, + int minReplyMax) + { + final long newReplyAck = Math.max(replySeq - minReplyNoAck, replyAck); + + if (newReplyAck > replyAck || minReplyMax > replyMax || !KafkaState.replyOpened(state)) + { + replyAck = newReplyAck; + assert replyAck <= replySeq; + + replyMax = minReplyMax; + + state = KafkaState.openedReply(state); + + doWindow(network, originId, routedId, replyId, replySeq, replyAck, replyMax, + traceId, authorization, budgetId, minReplyPad); + } + } + + private void doEncodeRequestIfNecessary( + long traceId, + long budgetId) + { + if (nextRequestId == nextResponseId) + { + encoder.accept(traceId, budgetId); + } + } + + private void doEncodeJoinGroupRequest( + long traceId, + long budgetId) + { + final MutableDirectBuffer encodeBuffer = writeBuffer; + final int encodeOffset = DataFW.FIELD_OFFSET_PAYLOAD; + final int encodeLimit = encodeBuffer.capacity(); + + int encodeProgress = encodeOffset; + + final RequestHeaderFW requestHeader = requestHeaderRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .length(0) + .apiKey(JOIN_GROUP_API_KEY) + .apiVersion(JOIN_GROUP_VERSION) + .correlationId(0) + .clientId(clientId) + .build(); + + encodeProgress = requestHeader.limit(); + + final String memberId = delegate.groupMembership.memberIds.getOrDefault(delegate.groupId, UNKNOWN_MEMBER_ID); + + final JoinGroupRequestFW joinGroupRequest = + joinGroupRequestRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .groupId(delegate.groupId) + .sessionTimeoutMillis(delegate.timeout) + .rebalanceTimeoutMillis((int) rebalanceTimeout.toMillis()) + .memberId(memberId) + .groupInstanceId(delegate.groupMembership.instanceId) + .protocolType("consumer") + .protocolCount(1) + .build(); + + encodeProgress = joinGroupRequest.limit(); + + final ProtocolMetadataFW protocolMetadata = + protocolMetadataRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .name(delegate.protocol) + .metadata(EMPTY_OCTETS) + .build(); + + encodeProgress = protocolMetadata.limit(); + + final int requestId = nextRequestId++; + final int requestSize = encodeProgress - encodeOffset - RequestHeaderFW.FIELD_OFFSET_API_KEY; + + requestHeaderRW.wrap(encodeBuffer, requestHeader.offset(), requestHeader.limit()) + .length(requestSize) + .apiKey(requestHeader.apiKey()) + .apiVersion(requestHeader.apiVersion()) + .correlationId(requestId) + .clientId(requestHeader.clientId().asString()) + .build(); + + doNetworkData(traceId, budgetId, encodeBuffer, encodeOffset, encodeProgress); + + decoder = decodeJoinGroupResponse; + + delegate.doApplicationBeginIfNecessary(traceId, authorization); + } + + private void doEncodeSyncGroupRequest( + long traceId, + long budgetId) + { + final MutableDirectBuffer encodeBuffer = writeBuffer; + final int encodeOffset = DataFW.FIELD_OFFSET_PAYLOAD; + final int encodeLimit = encodeBuffer.capacity(); + + int encodeProgress = encodeOffset; + + final RequestHeaderFW requestHeader = requestHeaderRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .length(0) + .apiKey(SYNC_GROUP_API_KEY) + .apiVersion(SYNC_GROUP_VERSION) + .correlationId(0) + .clientId(clientId) + .build(); + + encodeProgress = requestHeader.limit(); + + final String memberId = delegate.groupMembership.memberIds.get(delegate.groupId); + + final SyncGroupRequestFW syncGroupRequest = + syncGroupRequestRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .groupId(delegate.groupId) + .generatedId(generationId) + .memberId(memberId) + .groupInstanceId(delegate.groupMembership.instanceId) + .assignmentCount(members.size()) + .build(); + + encodeProgress = syncGroupRequest.limit(); + + for (int i = 0; i < members.size(); i++) + { + final AssignmentFW groupAssignment = + assignmentRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .memberId(members.get(i)) + .value(assignment) + .build(); + + encodeProgress = groupAssignment.limit(); + } + + final int requestId = nextRequestId++; + final int requestSize = encodeProgress - encodeOffset - RequestHeaderFW.FIELD_OFFSET_API_KEY; + + requestHeaderRW.wrap(encodeBuffer, requestHeader.offset(), requestHeader.limit()) + .length(requestSize) + .apiKey(requestHeader.apiKey()) + .apiVersion(requestHeader.apiVersion()) + .correlationId(requestId) + .clientId(requestHeader.clientId().asString()) + .build(); + + doNetworkData(traceId, budgetId, encodeBuffer, encodeOffset, encodeProgress); + + decoder = decodeSyncGroupResponse; + } + + private void doEncodeHeartbeatRequest( + long traceId, + long budgetId) + { + final MutableDirectBuffer encodeBuffer = writeBuffer; + final int encodeOffset = DataFW.FIELD_OFFSET_PAYLOAD; + final int encodeLimit = encodeBuffer.capacity(); + + int encodeProgress = encodeOffset; + + final RequestHeaderFW requestHeader = requestHeaderRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .length(0) + .apiKey(HEARTBEAT_API_KEY) + .apiVersion(HEARTBEAT_VERSION) + .correlationId(0) + .clientId(clientId) + .build(); + + encodeProgress = requestHeader.limit(); + + final String memberId = delegate.groupMembership.memberIds.get(delegate.groupId); + + final HeartbeatRequestFW heartbeatRequest = + heartbeatRequestRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .groupId(delegate.groupId) + .generatedId(generationId) + .memberId(memberId) + .groupInstanceId(delegate.groupMembership.instanceId) + .build(); + + encodeProgress = heartbeatRequest.limit(); + + final int requestId = nextRequestId++; + final int requestSize = encodeProgress - encodeOffset - RequestHeaderFW.FIELD_OFFSET_API_KEY; + + requestHeaderRW.wrap(encodeBuffer, requestHeader.offset(), requestHeader.limit()) + .length(requestSize) + .apiKey(requestHeader.apiKey()) + .apiVersion(requestHeader.apiVersion()) + .correlationId(requestId) + .clientId(requestHeader.clientId().asString()) + .build(); + + doNetworkData(traceId, budgetId, encodeBuffer, encodeOffset, encodeProgress); + + decoder = decodeHeartbeatResponse; + } + + private void doEncodeLeaveGroupRequest( + long traceId, + long budgetId) + { + final MutableDirectBuffer encodeBuffer = writeBuffer; + final int encodeOffset = DataFW.FIELD_OFFSET_PAYLOAD; + final int encodeLimit = encodeBuffer.capacity(); + + int encodeProgress = encodeOffset; + + final RequestHeaderFW requestHeader = requestHeaderRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .length(0) + .apiKey(LEAVE_GROUP_API_KEY) + .apiVersion(LEAVE_GROUP_VERSION) + .correlationId(0) + .clientId(clientId) + .build(); + + encodeProgress = requestHeader.limit(); + + final LeaveGroupRequestFW leaveGroupRequest = + leaveGroupRequestRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .groupId(delegate.groupId) + .memberCount(1) + .build(); + + encodeProgress = leaveGroupRequest.limit(); + + final String memberId = delegate.groupMembership.memberIds.get(delegate.groupId); + + final LeaveMemberFW leaveMember = leaveMemberRW.wrap(encodeBuffer, encodeProgress, encodeLimit) + .memberId(memberId) + .groupInstanceId(delegate.groupMembership.instanceId) + .build(); + + encodeProgress = leaveMember.limit(); + + final int requestId = nextRequestId++; + final int requestSize = encodeProgress - encodeOffset - RequestHeaderFW.FIELD_OFFSET_API_KEY; + + requestHeaderRW.wrap(encodeBuffer, requestHeader.offset(), requestHeader.limit()) + .length(requestSize) + .apiKey(requestHeader.apiKey()) + .apiVersion(requestHeader.apiVersion()) + .correlationId(requestId) + .clientId(requestHeader.clientId().asString()) + .build(); + + doNetworkData(traceId, budgetId, encodeBuffer, encodeOffset, encodeProgress); + + decoder = decodeLeaveGroupResponse; + } + + private void doSyncRequest( + long traceId, + long budgetId, + OctetsFW assignment) + { + this.assignment = assignment; + doEncodeSyncGroupRequest(traceId, budgetId); + } + + private void doHeartbeat( + long traceId) + { + if (heartbeatRequestId != NO_CANCEL_ID) + { + signaler.cancel(heartbeatRequestId); + heartbeatRequestId = NO_CANCEL_ID; + } + + encoder = encodeHeartbeatRequest; + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void doLeaveGroupRequest( + long traceId) + { + if (heartbeatRequestId != NO_CANCEL_ID) + { + signaler.cancel(heartbeatRequestId); + heartbeatRequestId = NO_CANCEL_ID; + } + + encoder = encodeLeaveGroupRequest; + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void encodeNetwork( + long traceId, + long authorization, + long budgetId, + DirectBuffer buffer, + int offset, + int limit) + { + final int maxLength = limit - offset; + final int initialWin = initialMax - (int)(initialSeq - initialAck); + final int length = Math.max(Math.min(initialWin - initialPad, maxLength), 0); + + if (length > 0) + { + final int reserved = length + initialPad; + + doData(network, originId, routedId, initialId, initialSeq, initialAck, initialMax, + traceId, authorization, budgetId, reserved, buffer, offset, length, EMPTY_EXTENSION); + + initialSeq += reserved; + + assert initialAck <= initialSeq; + } + + final int remaining = maxLength - length; + if (remaining > 0) + { + if (encodeSlot == NO_SLOT) + { + encodeSlot = encodePool.acquire(initialId); + } + + if (encodeSlot == NO_SLOT) + { + onError(traceId); + } + else + { + final MutableDirectBuffer encodeBuffer = encodePool.buffer(encodeSlot); + encodeBuffer.putBytes(0, buffer, offset + length, remaining); + encodeSlotOffset = remaining; + } + } + else + { + cleanupEncodeSlotIfNecessary(); + } + } + + private void decodeNetwork( + long traceId, + long authorization, + long budgetId, + int reserved, + MutableDirectBuffer buffer, + int offset, + int limit) + { + KafkaGroupCoordinatorClientDecoder previous = null; + int progress = offset; + while (progress <= limit && previous != decoder) + { + previous = decoder; + progress = decoder.decode(this, traceId, authorization, budgetId, reserved, buffer, offset, progress, limit); + } + + if (progress < limit) + { + if (decodeSlot == NO_SLOT) + { + decodeSlot = decodePool.acquire(initialId); + } + + if (decodeSlot == NO_SLOT) + { + onError(traceId); + } + else + { + final MutableDirectBuffer decodeBuffer = decodePool.buffer(decodeSlot); + decodeBuffer.putBytes(0, buffer, progress, limit - progress); + decodeSlotOffset = limit - progress; + decodeSlotReserved = (limit - progress) * reserved / (limit - offset); + } + + doNetworkWindow(traceId, budgetId, decodeSlotOffset, 0, replyMax); + } + else + { + cleanupDecodeSlotIfNecessary(); + + if (reserved > 0) + { + doNetworkWindow(traceId, budgetId, 0, 0, replyMax); + } + } + } + + @Override + protected void doDecodeSaslHandshakeResponse( + long traceId) + { + decoder = decodeCoordinatorSaslHandshakeResponse; + } + + @Override + protected void doDecodeSaslHandshake( + long traceId) + { + decoder = decodeCoordinatorSaslHandshake; + } + + @Override + protected void doDecodeSaslHandshakeMechanisms( + long traceId) + { + decoder = decodeCoordinatorSaslHandshakeMechanisms; + } + + @Override + protected void doDecodeSaslHandshakeMechansim( + long traceId) + { + decoder = decodeCoordinatorSaslHandshakeMechanism; + } + + @Override + protected void doDecodeSaslAuthenticateResponse( + long traceId) + { + decoder = decodeCoordinatorSaslAuthenticateResponse; + } + + @Override + protected void doDecodeSaslAuthenticate( + long traceId) + { + decoder = decodeCoordinatorSaslAuthenticate; + } + + @Override + protected void onDecodeSaslHandshakeResponse( + long traceId, + long authorization, + int errorCode) + { + switch (errorCode) + { + case ERROR_NONE: + encoder = encodeSaslAuthenticateRequest; + decoder = decodeCoordinatorSaslAuthenticateResponse; + break; + default: + delegate.cleanupApplication(traceId, errorCode); + doNetworkEnd(traceId, authorization); + break; + } + } + + @Override + protected void onDecodeSaslAuthenticateResponse( + long traceId, + long authorization, + int errorCode) + { + switch (errorCode) + { + case ERROR_NONE: + encoder = encodeJoinGroupRequest; + decoder = decodeJoinGroupResponse; + break; + default: + delegate.cleanupApplication(traceId, errorCode); + doNetworkEnd(traceId, authorization); + break; + } + } + + @Override + protected void onDecodeSaslResponse( + long traceId) + { + nextResponseId++; + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void onNotCoordinatorError( + long traceId, + long authorization) + { + nextResponseId++; + + cleanupNetwork(traceId, authorization); + + delegate.onNotCoordinatorError(traceId, authorization); + } + + private void onJoinGroupUnknownMemberError( + long traceId, + long authorization) + { + nextResponseId++; + + delegate.groupMembership.memberIds.put(delegate.groupId, UNKNOWN_MEMBER_ID); + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void onJoinGroupMemberIdError( + long traceId, + long authorization, + String memberId) + { + nextResponseId++; + + delegate.groupMembership.memberIds.put(delegate.groupId, memberId); + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void onJoinGroupResponse( + long traceId, + long authorization, + String leader, + String memberId, + int error) + { + nextResponseId++; + + this.leader = leader; + + delegate.groupMembership.memberIds.put(delegate.groupId, memberId); + + encoder = encodeSyncGroupRequest; + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void onSynGroupRebalance( + long traceId, + long authorization) + { + nextResponseId++; + + encoder = encodeJoinGroupRequest; + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void onSyncGroupResponse( + long traceId, + long authorization, + OctetsFW assignment) + { + nextResponseId++; + + final String memberId = delegate.groupMembership.memberIds.get(delegate.groupId); + + delegate.doApplicationData(traceId, authorization, assignment, + ex -> ex.set((b, o, l) -> kafkaDataExRW.wrap(b, o, l) + .typeId(kafkaTypeId) + .group(g -> g.leaderId(leader).memberId(memberId)) + .build() + .sizeof())); + + if (heartbeatRequestId != NO_CANCEL_ID) + { + encoder = encodeHeartbeatRequest; + + heartbeatRequestId = signaler.signalAt(currentTimeMillis() + delegate.timeout / 2, + originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + } + + private void onHeartbeatResponse( + long traceId, + long authorization) + { + nextResponseId++; + + if (heartbeatRequestId != NO_CANCEL_ID) + { + signaler.cancel(heartbeatRequestId); + heartbeatRequestId = NO_CANCEL_ID; + } + + encoder = encodeHeartbeatRequest; + + heartbeatRequestId = signaler.signalAt(currentTimeMillis() + delegate.timeout / 2, + originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void onLeaveGroupResponse( + long traceId, + long authorization) + { + doNetworkEnd(traceId, authorization); + doNetworkReset(traceId); + + delegate.doApplicationEnd(traceId); + delegate.doApplicationResetIfNecessary(traceId, EMPTY_OCTETS); + } + + private void onRebalanceError( + long traceId, + long authorization) + { + nextResponseId++; + + encoder = encodeJoinGroupRequest; + signaler.signalNow(originId, routedId, initialId, SIGNAL_NEXT_REQUEST, 0); + } + + private void cleanupNetwork( + long traceId, + long authorization) + { + doNetworkEnd(traceId, authorization); + doNetworkReset(traceId); + } + + private void onError( + long traceId) + { + doNetworkAbort(traceId); + doNetworkReset(traceId); + + delegate.cleanupApplication(traceId, EMPTY_OCTETS); + } + + private void cleanupDecodeSlotIfNecessary() + { + if (decodeSlot != NO_SLOT) + { + decodePool.release(decodeSlot); + decodeSlot = NO_SLOT; + decodeSlotOffset = 0; + decodeSlotReserved = 0; + } + } + + private void cleanupEncodeSlotIfNecessary() + { + if (encodeSlot != NO_SLOT) + { + encodePool.release(encodeSlot); + encodeSlot = NO_SLOT; + encodeSlotOffset = 0; + encodeSlotTraceId = 0; + } + } + } + + private final class GroupMembership + { + public final String instanceId; + public final Map memberIds; + + GroupMembership( + String instanceId) + { + this.instanceId = instanceId; + this.memberIds = new Object2ObjectHashMap<>(); + } + } +} diff --git a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientSaslHandshaker.java b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientSaslHandshaker.java index 00ed048dbe..13337e1505 100644 --- a/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientSaslHandshaker.java +++ b/runtime/binding-kafka/src/main/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/KafkaClientSaslHandshaker.java @@ -113,8 +113,8 @@ public abstract class KafkaSaslClient protected final KafkaSaslConfig sasl; protected final long originId; protected final long routedId; - protected final long initialId; - protected final long replyId; + protected long initialId; + protected long replyId; protected int nextRequestId; diff --git a/runtime/binding-kafka/src/main/zilla/protocol.idl b/runtime/binding-kafka/src/main/zilla/protocol.idl index d3267cb430..02cbe07d4b 100644 --- a/runtime/binding-kafka/src/main/zilla/protocol.idl +++ b/runtime/binding-kafka/src/main/zilla/protocol.idl @@ -336,7 +336,145 @@ scope protocol struct ProduceResponseTrailer { - int32 thottleTimeMillis; + int32 throttleTimeMillis; + } + } + + scope group + { + struct FindCoordinatorRequest + { + string16 key; + int8 keyType; + } + + struct FindCoordinatorResponse + { + int32 correlationId; + int32 throttleTimeMillis; + int16 errorCode; + string16 errorMessage = null; + int32 nodeId; + string16 host; + int32 port; + } + + struct JoinGroupRequest + { + string16 groupId; + int32 sessionTimeoutMillis; + int32 rebalanceTimeoutMillis; + string16 memberId; + string16 groupInstanceId = null; + string16 protocolType; + int32 protocolCount; + } + + struct ProtocolMetadata + { + string16 name; + uint32 metadataLength; + octets[metadataLength] metadata; + } + + struct RangeProtocol + { + int32 version; + string16 topic; + int32 partitionCount; + } + + struct JoinGroupResponse + { + int32 correlationId; + int32 throttleTimeMillis; + int16 errorCode; + int32 generatedId; + string16 protocolName; + string16 leader; + string16 memberId; + int32 memberCount; + } + + struct MemberMetadata + { + string16 memberId; + string16 groupInstanceId = null; + uint32 length; + octets[length] metadata; + } + + struct SyncGroupRequest + { + string16 groupId; + int32 generatedId; + string16 memberId; + string16 groupInstanceId = null; + int32 assignmentCount; + } + + struct Assignment + { + string16 memberId; + uint32 length; + octets[length] value; + } + + struct TopicPartition + { + int32 version; + string16 topic; + int32 partitionCount; + } + + struct Partition + { + int32 partitionId; + int32 offsetId; + } + + struct SyncGroupResponse + { + int32 correlationId; + int32 throttleTimeMillis; + int16 errorCode; + uint32 assignmentLength; + octets[assignmentLength] assignment; + } + + struct HeartbeatRequest + { + string16 groupId; + int32 generatedId; + string16 memberId; + string16 groupInstanceId = null; + } + + struct HeartbeatResponse + { + int32 correlationId; + int32 throttleTimeMillis; + int16 errorCode; + } + + struct LeaveGroupRequest + { + string16 groupId; + int32 memberCount; + } + + struct LeaveGroupResponse + { + int32 correlationId; + int32 throttleTimeMillis; + int16 errorCode; + int32 memberCount; + } + + struct LeaveMember + { + string16 memberId; + string16 groupInstanceId = null; } } diff --git a/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfigAdapterTest.java b/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfigAdapterTest.java index 05c8731c2f..d28a654c15 100644 --- a/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfigAdapterTest.java +++ b/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/config/KafkaConditionConfigAdapterTest.java @@ -44,23 +44,25 @@ public void shouldReadCondition() { String text = "{" + - "\"topic\": \"test\"" + + "\"topic\": \"test\"," + + "\"groupId\": \"test\"" + "}"; KafkaConditionConfig condition = jsonb.fromJson(text, KafkaConditionConfig.class); assertThat(condition, not(nullValue())); assertThat(condition.topic, equalTo("test")); + assertThat(condition.groupId, equalTo("test")); } @Test public void shouldWriteCondition() { - KafkaConditionConfig condition = new KafkaConditionConfig("test"); + KafkaConditionConfig condition = new KafkaConditionConfig("test", "test"); String text = jsonb.toJson(condition); assertThat(text, not(nullValue())); - assertThat(text, equalTo("{\"topic\":\"test\"}")); + assertThat(text, equalTo("{\"topic\":\"test\",\"groupId\":\"test\"}")); } } diff --git a/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/CacheGroupIT.java b/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/CacheGroupIT.java new file mode 100644 index 0000000000..e7c19b50aa --- /dev/null +++ b/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/CacheGroupIT.java @@ -0,0 +1,68 @@ +/* + * Copyright 2021-2023 Aklivity Inc. + * + * Aklivity 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 io.aklivity.zilla.runtime.binding.kafka.internal.stream; + +import static io.aklivity.zilla.runtime.binding.kafka.internal.KafkaConfiguration.KAFKA_CACHE_SERVER_BOOTSTRAP; +import static io.aklivity.zilla.runtime.binding.kafka.internal.KafkaConfiguration.KAFKA_CACHE_SERVER_RECONNECT_DELAY; +import static io.aklivity.zilla.runtime.engine.EngineConfiguration.ENGINE_BUFFER_SLOT_CAPACITY; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.junit.rules.RuleChain.outerRule; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.DisableOnDebug; +import org.junit.rules.TestRule; +import org.junit.rules.Timeout; +import org.kaazing.k3po.junit.annotation.ScriptProperty; +import org.kaazing.k3po.junit.annotation.Specification; +import org.kaazing.k3po.junit.rules.K3poRule; + +import io.aklivity.zilla.runtime.engine.test.EngineRule; +import io.aklivity.zilla.runtime.engine.test.annotation.Configuration; + +public class CacheGroupIT +{ + private final K3poRule k3po = new K3poRule() + .addScriptRoot("app", "io/aklivity/zilla/specs/binding/kafka/streams/application/group"); + + private final TestRule timeout = new DisableOnDebug(new Timeout(10, SECONDS)); + + private final EngineRule engine = new EngineRule() + .directory("target/zilla-itests") + .commandBufferCapacity(1024) + .responseBufferCapacity(1024) + .counterValuesBufferCapacity(8192) + .configure(ENGINE_BUFFER_SLOT_CAPACITY, 8192) + .configure(KAFKA_CACHE_SERVER_BOOTSTRAP, false) + .configure(KAFKA_CACHE_SERVER_RECONNECT_DELAY, 0) + .configurationRoot("io/aklivity/zilla/specs/binding/kafka/config") + .external("app1") + .clean(); + + @Rule + public final TestRule chain = outerRule(engine).around(k3po).around(timeout); + + @Test + @Configuration("cache.yaml") + @Specification({ + "${app}/rebalance.protocol.highlander/client", + "${app}/rebalance.protocol.highlander/server"}) + @ScriptProperty("serverAddress \"zilla://streams/app1\"") + public void shouldRebalanceLeader() throws Exception + { + k3po.finish(); + } +} diff --git a/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/ClientGroupIT.java b/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/ClientGroupIT.java new file mode 100644 index 0000000000..a39c8b872a --- /dev/null +++ b/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/ClientGroupIT.java @@ -0,0 +1,133 @@ +/* + * Copyright 2021-2023 Aklivity Inc. + * + * Aklivity 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 io.aklivity.zilla.runtime.binding.kafka.internal.stream; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.junit.rules.RuleChain.outerRule; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.DisableOnDebug; +import org.junit.rules.TestRule; +import org.junit.rules.Timeout; +import org.kaazing.k3po.junit.annotation.Specification; +import org.kaazing.k3po.junit.rules.K3poRule; + +import io.aklivity.zilla.runtime.engine.test.EngineRule; +import io.aklivity.zilla.runtime.engine.test.annotation.Configuration; + +public class ClientGroupIT +{ + private final K3poRule k3po = new K3poRule() + .addScriptRoot("net", "io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3") + .addScriptRoot("app", "io/aklivity/zilla/specs/binding/kafka/streams/application/group"); + + private final TestRule timeout = new DisableOnDebug(new Timeout(15, SECONDS)); + + private final EngineRule engine = new EngineRule() + .directory("target/zilla-itests") + .commandBufferCapacity(1024) + .responseBufferCapacity(1024) + .counterValuesBufferCapacity(8192) + .configurationRoot("io/aklivity/zilla/specs/binding/kafka/config") + .external("net0") + .clean(); + + @Rule + public final TestRule chain = outerRule(engine).around(k3po).around(timeout); + + + @Test + @Configuration("client.yaml") + @Specification({ + "${app}/client.sent.write.abort.before.coordinator.response/client", + "${net}/client.sent.write.abort.before.coordinator.response/server"}) + public void shouldHandleClientSentWriteAbortBeforeCoordinatorResponse() throws Exception + { + k3po.finish(); + } + + @Test + @Configuration("client.yaml") + @Specification({ + "${app}/rebalance.protocol.highlander/client", + "${net}/rebalance.protocol.highlander/server"}) + + public void shouldLeaveGroupOnGroupRebalanceError() throws Exception + { + k3po.finish(); + } + + @Test + @Configuration("client.yaml") + @Specification({ + "${app}/leader/client", + "${net}/coordinator.not.available/server"}) + public void shouldHandleCoordinatorNotAvailableError() throws Exception + { + k3po.finish(); + } + + @Test + @Configuration("client.yaml") + @Specification({ + "${app}/leader/client", + "${net}/coordinator.reject.invalid.consumer/server"}) + public void shouldHRejectInvalidConsumer() throws Exception + { + k3po.finish(); + } + + @Test + @Configuration("client.yaml") + @Specification({ + "${app}/leader/client", + "${net}/rebalance.protocol.highlander.unknown.member.id/server"}) + public void shouldRebalanceProtocolHighlanderUnknownMemberId() throws Exception + { + k3po.finish(); + } + + @Test + @Configuration("client.yaml") + @Specification({ + "${app}/rebalance.protocol.highlander.migrate.leader/client", + "${net}/rebalance.protocol.highlander.migrate.leader/server"}) + public void shouldRebalanceProtocolHighlanderMigrateLeader() throws Exception + { + k3po.finish(); + } + + @Test + @Configuration("client.yaml") + @Specification({ + "${app}/rebalance.protocol.unknown/client", + "${net}/rebalance.protocol.unknown/server"}) + public void shouldRejectSecondStreamOnUnknownProtocol() throws Exception + { + k3po.finish(); + } + + @Test + @Configuration("client.yaml") + @Specification({ + "${app}/leader/client", + "${net}/rebalance.sync.group/server"}) + public void shouldHandleRebalanceSyncGroup() throws Exception + { + k3po.finish(); + } +} diff --git a/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/ClientGroupSaslIT.java b/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/ClientGroupSaslIT.java new file mode 100644 index 0000000000..789359fb95 --- /dev/null +++ b/runtime/binding-kafka/src/test/java/io/aklivity/zilla/runtime/binding/kafka/internal/stream/ClientGroupSaslIT.java @@ -0,0 +1,63 @@ +/* + * Copyright 2021-2023 Aklivity Inc. + * + * Aklivity 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 io.aklivity.zilla.runtime.binding.kafka.internal.stream; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.junit.rules.RuleChain.outerRule; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.DisableOnDebug; +import org.junit.rules.TestRule; +import org.junit.rules.Timeout; +import org.kaazing.k3po.junit.annotation.Specification; +import org.kaazing.k3po.junit.rules.K3poRule; + +import io.aklivity.zilla.runtime.engine.test.EngineRule; +import io.aklivity.zilla.runtime.engine.test.annotation.Configuration; + +public class ClientGroupSaslIT +{ + private final K3poRule k3po = new K3poRule() + .addScriptRoot("net", + "io/aklivity/zilla/specs/binding/kafka/streams/network/group.sasl.f1.j5.s3.l3.h3.handshake.v1") + .addScriptRoot("app", "io/aklivity/zilla/specs/binding/kafka/streams/application/group"); + + private final TestRule timeout = new DisableOnDebug(new Timeout(15, SECONDS)); + + private final EngineRule engine = new EngineRule() + .directory("target/zilla-itests") + .commandBufferCapacity(1024) + .responseBufferCapacity(1024) + .counterValuesBufferCapacity(8192) + .configurationRoot("io/aklivity/zilla/specs/binding/kafka/config") + .external("net0") + .clean(); + + @Rule + public final TestRule chain = outerRule(engine).around(k3po).around(timeout); + + + @Test + @Configuration("client.options.sasl.plain.yaml") + @Specification({ + "${app}/leader/client", + "${net}/leader/server"}) + public void shouldBecomeLeader() throws Exception + { + k3po.finish(); + } +} diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/schema/kafka.schema.patch.json b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/schema/kafka.schema.patch.json index 2b20c34eef..a27ea5f0a3 100644 --- a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/schema/kafka.schema.patch.json +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/schema/kafka.schema.patch.json @@ -11,7 +11,7 @@ { "if": { - "properties": + "properties": { "type": { @@ -218,6 +218,11 @@ { "title": "Topic", "type": "string" + }, + "groupId": + { + "title": "groupId", + "type": "string" } }, "additionalProperties": false diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/client.sent.write.abort.before.coordinator.response/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/client.sent.write.abort.before.coordinator.response/client.rpt new file mode 100644 index 0000000000..ca416d10ee --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/client.sent.write.abort.before.coordinator.response/client.rpt @@ -0,0 +1,32 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +connect "zilla://streams/app0" + option zilla:window 8192 + option zilla:transmission "half-duplex" + +write zilla:begin.ext ${kafka:beginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + +write abort diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/client.sent.write.abort.before.coordinator.response/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/client.sent.write.abort.before.coordinator.response/server.rpt new file mode 100644 index 0000000000..6d3853b32e --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/client.sent.write.abort.before.coordinator.response/server.rpt @@ -0,0 +1,36 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property serverAddress "zilla://streams/app0" + +accept ${serverAddress} + option zilla:window 8192 + option zilla:transmission "half-duplex" + +accepted + +read zilla:begin.ext ${kafka:matchBeginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + +read aborted diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/leader/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/leader/client.rpt new file mode 100644 index 0000000000..ea472ace1c --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/leader/client.rpt @@ -0,0 +1,41 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +connect "zilla://streams/app0" + option zilla:window 8192 + option zilla:transmission "half-duplex" + +write zilla:begin.ext ${kafka:beginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + + +read zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +read zilla:data.null + diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/leader/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/leader/server.rpt new file mode 100644 index 0000000000..178b40bc4d --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/leader/server.rpt @@ -0,0 +1,43 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property serverAddress "zilla://streams/app0" + +accept ${serverAddress} + option zilla:window 8192 + option zilla:transmission "half-duplex" + +accepted + +read zilla:begin.ext ${kafka:matchBeginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + +write zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +write flush diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander.migrate.leader/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander.migrate.leader/client.rpt new file mode 100644 index 0000000000..567e4f7725 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander.migrate.leader/client.rpt @@ -0,0 +1,69 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +connect "zilla://streams/app0" + option zilla:window 8192 + option zilla:transmission "half-duplex" + +write zilla:begin.ext ${kafka:beginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + + +read zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +read zilla:data.null + +read notify ROUTED_BROKER_SERVER + +write aborted + +connect await ROUTED_BROKER_SERVER "zilla://streams/app0" + option zilla:window 8192 + option zilla:transmission "half-duplex" + +write zilla:begin.ext ${kafka:beginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + + +read zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +read zilla:data.null diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander.migrate.leader/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander.migrate.leader/server.rpt new file mode 100644 index 0000000000..ff4baeea4a --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander.migrate.leader/server.rpt @@ -0,0 +1,68 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property serverAddress "zilla://streams/app0" + +accept ${serverAddress} + option zilla:window 8192 + option zilla:transmission "half-duplex" + +accepted + +read zilla:begin.ext ${kafka:matchBeginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + +write zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +write flush + +read abort + +accepted + +read zilla:begin.ext ${kafka:matchBeginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + +write zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +write flush + diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander/client.rpt new file mode 100644 index 0000000000..b7765fa2a4 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander/client.rpt @@ -0,0 +1,54 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +connect "zilla://streams/app0" + option zilla:window 8192 + option zilla:transmission "half-duplex" + +write zilla:begin.ext ${kafka:beginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + + +read zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +read zilla:data.null + +write advise zilla:flush + +read zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +read zilla:data.null + +write close +read closed diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander/server.rpt new file mode 100644 index 0000000000..4f0f18bb9c --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.highlander/server.rpt @@ -0,0 +1,57 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property serverAddress "zilla://streams/app0" + +accept ${serverAddress} + option zilla:window 8192 + option zilla:transmission "half-duplex" + +accepted + +read zilla:begin.ext ${kafka:matchBeginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("highlander") + .timeout(45000) + .build() + .build()} + +connected + +write zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +write flush + +read advised zilla:flush + +write zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +write flush + +read closed +write close diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.unknown/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.unknown/client.rpt new file mode 100644 index 0000000000..b8293ceeca --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.unknown/client.rpt @@ -0,0 +1,57 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +connect "zilla://streams/app0" + option zilla:window 8192 + option zilla:transmission "half-duplex" + +write zilla:begin.ext ${kafka:beginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("unknown") + .timeout(45000) + .build() + .build()} + +connected + + +read zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +read zilla:data.null + +read notify ROUTED_BROKER_SERVER + +connect await ROUTED_BROKER_SERVER "zilla://streams/app0" + option zilla:window 8192 + option zilla:transmission "half-duplex" + +write zilla:begin.ext ${kafka:beginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("unknown") + .timeout(45000) + .build() + .build()} + +connect aborted diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.unknown/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.unknown/server.rpt new file mode 100644 index 0000000000..a72a8ba9f8 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/application/group/rebalance.protocol.unknown/server.rpt @@ -0,0 +1,45 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property serverAddress "zilla://streams/app0" + +accept ${serverAddress} + option zilla:window 8192 + option zilla:transmission "half-duplex" + +accepted + +read zilla:begin.ext ${kafka:matchBeginEx() + .typeId(zilla:id("kafka")) + .group() + .groupId("test") + .protocol("unknown") + .timeout(45000) + .build() + .build()} + +connected + +write zilla:data.ext ${kafka:dataEx() + .typeId(zilla:id("kafka")) + .group() + .leaderId("memberId-1") + .memberId("memberId-1") + .build() + .build()} +write flush + +rejected diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/client.sent.write.abort.before.coordinator.response/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/client.sent.write.abort.before.coordinator.response/client.rpt new file mode 100644 index 0000000000..1012892d86 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/client.sent.write.abort.before.coordinator.response/client.rpt @@ -0,0 +1,41 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property networkConnectWindow 8192 + +property newRequestId ${kafka:newRequestId()} +property fetchWaitMax 500 +property fetchBytesMax 65535 +property partitionBytesMax 8192 + +connect "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # "session" coordinator key + [0x00] # coordinator group type + + +write abort +read aborted diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/client.sent.write.abort.before.coordinator.response/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/client.sent.write.abort.before.coordinator.response/server.rpt new file mode 100644 index 0000000000..1db15e2632 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/client.sent.write.abort.before.coordinator.response/server.rpt @@ -0,0 +1,35 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +accept "zilla://streams/net0" + option zilla:window 8192 + option zilla:transmission "duplex" + option zilla:byteorder "network" + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # "session" coordinator key + [0x00] # coordinator group type + +read aborted +write abort diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.not.available/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.not.available/client.rpt new file mode 100644 index 0000000000..34b66f8209 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.not.available/client.rpt @@ -0,0 +1,150 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property networkConnectWindow 8192 +property instanceId ${kafka:randomBytes(42)} + +property newRequestId ${kafka:newRequestId()} +property fetchWaitMax 500 +property fetchBytesMax 65535 +property partitionBytesMax 8192 + +connect "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +read 54 # size + (int:newRequestId) + 0 # throttle time + 15s # no coordinator available + 32s "The coordinator is not available" # error no coordinator available + -1 # coordinator node + 0s # host + -1 # port + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4s "none" # error message none + 0 # coordinator node + 9s "localhost" # host + 9092 # port + +write close +read abort + +read notify ROUTED_BROKER_SERVER + +connect await ROUTED_BROKER_SERVER + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 105 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 34 # size + (int:newRequestId) + 0 # throttle time + 79s # member id required + -1 # generated id + 0s # protocol name + 0s # leader id + 10s "memberId-1" # consumer member group id + 0 # members + +write 115 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 112 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment + diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.not.available/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.not.available/server.rpt new file mode 100644 index 0000000000..960d26d2d4 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.not.available/server.rpt @@ -0,0 +1,140 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property instanceId ${kafka:randomBytes(42)} + +accept "zilla://streams/net0" + option zilla:window 8192 + option zilla:transmission "duplex" + option zilla:byteorder "network" + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 54 # size + ${newRequestId} + 0 # throttle time + 15s # no coordinator available + 32s "The coordinator is not available" # error no coordinator available + -1 # coordinator node + 0s # host + -1 # port + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4s "none" # error message none + 0 # coordinator node + 9s "localhost" # host + 9092 # port + +read closed +write aborted + +accepted + +connected + +read 105 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + + +write 34 # size + ${newRequestId} + 0 # throttle time + 79s # member id required + -1 # generated id + 0s # protocol name + 0s # leader id + 10s "memberId-1" # consumer member group id + 0 # members + +read 115 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +write 112 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.reject.invalid.consumer/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.reject.invalid.consumer/client.rpt new file mode 100644 index 0000000000..bb8ccf51e6 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.reject.invalid.consumer/client.rpt @@ -0,0 +1,176 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property networkConnectWindow 8192 +property instanceId ${kafka:randomBytes(42)} + +property newRequestId ${kafka:newRequestId()} +property fetchWaitMax 500 +property fetchBytesMax 65535 +property partitionBytesMax 8192 + +connect "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4s "none" # error message none + 0 # coordinator node + 9s "localhost" # host + 9092 # port + +write close +read abort + +read notify ROUTED_BROKER_SERVER_FIRST + +connect await ROUTED_BROKER_SERVER_FIRST + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 105 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 24 # size + (int:newRequestId) + 0 # throttle time + 16s # not a coordinator for a consumer + -1 # generated id + 0s # protocol name + 0s # leader id + 0s # not a coordinator for a consumer + 0 # members + +write close +read abort + +read notify ROUTED_BROKER_SERVER_SECOND + +connect await ROUTED_BROKER_SERVER_SECOND + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4s "none" # error message none + 0 # coordinator node + 9s "localhost" # host + 9092 # port + +write close +read abort + +read notify ROUTED_BROKER_SERVER_THIRD + +connect await ROUTED_BROKER_SERVER_THIRD + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 105 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 112 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment + diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.reject.invalid.consumer/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.reject.invalid.consumer/server.rpt new file mode 100644 index 0000000000..6f7f230d1a --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/coordinator.reject.invalid.consumer/server.rpt @@ -0,0 +1,153 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property instanceId ${kafka:randomBytes(42)} + +accept "zilla://streams/net0" + option zilla:window 8192 + option zilla:transmission "duplex" + option zilla:byteorder "network" + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4s "none" # error message none + 0 # coordinator node + 9s "localhost" # host + 9092 # port + +read closed +write aborted + +accepted + +connected + +read 105 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +write 24 # size + ${newRequestId} + 0 # throttle time + 16s # not a coordinator for a consumer + -1 # generated id + 0s # protocol name + 0s # leader id + 0s # not a coordinator for a consumer + 0 # members + +read closed +write aborted + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4s "none" # error message none + 0 # coordinator node + 9s "localhost" # host + 9092 # port + +read closed +write aborted + +accepted + +connected + +read 105 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +write 112 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.migrate.leader/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.migrate.leader/client.rpt new file mode 100644 index 0000000000..71cb8605e4 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.migrate.leader/client.rpt @@ -0,0 +1,195 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property networkConnectWindow 8192 +property instanceId ${kafka:randomBytes(42)} + +property newRequestId ${kafka:newRequestId()} +property fetchWaitMax 500 +property fetchBytesMax 65535 +property partitionBytesMax 8192 + +connect "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "session" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +write close +read abort + +read notify ROUTED_BROKER_SERVER + +connect await ROUTED_BROKER_SERVER + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 105 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 112 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment + +write close +read abort + +read notify ROUTED_BROKER_SERVER_SECOND + +connect await ROUTED_BROKER_SERVER_SECOND "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "session" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +write close +read abort + +read notify ROUTED_BROKER_SERVER_THIRD + +connect await ROUTED_BROKER_SERVER_THIRD "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 115 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 112 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.migrate.leader/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.migrate.leader/server.rpt new file mode 100644 index 0000000000..efaee4ce49 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.migrate.leader/server.rpt @@ -0,0 +1,175 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property instanceId ${kafka:randomBytes(42)} + +accept "zilla://streams/net0" + option zilla:window 8192 + option zilla:transmission "duplex" + option zilla:byteorder "network" + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +read closed +write aborted + +accepted + +connected + +read 105 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +write 112 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment + +read closed +write aborted + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +read closed +write aborted + +accepted + +connected + +read 115 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +write 112 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.unknown.member.id/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.unknown.member.id/client.rpt new file mode 100644 index 0000000000..b956d5e1f1 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.unknown.member.id/client.rpt @@ -0,0 +1,132 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property networkConnectWindow 8192 +property instanceId ${kafka:randomBytes(42)} + +property newRequestId ${kafka:newRequestId()} +property fetchWaitMax 500 +property fetchBytesMax 65535 +property partitionBytesMax 8192 + +connect "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "session" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +write close +read abort + +read notify ROUTED_BROKER_SERVER + +connect await ROUTED_BROKER_SERVER + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 105 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 24 # size + (int:newRequestId) + 0 # throttle time + 25s # unknown member id + -1 # generated id + 0s # protocol name + 0s # leader id + 0s # consumer member group id + 0 # members + +write 105 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 112 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.unknown.member.id/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.unknown.member.id/server.rpt new file mode 100644 index 0000000000..06ea422a7a --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander.unknown.member.id/server.rpt @@ -0,0 +1,123 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property instanceId ${kafka:randomBytes(42)} + +accept "zilla://streams/net0" + option zilla:window 8192 + option zilla:transmission "duplex" + option zilla:byteorder "network" + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +read closed +write aborted + +accepted + +connected + +read 105 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + + +write 24 # size + ${newRequestId} + 0 # throttle time + 25s # member id required + -1 # generated id + 0s # protocol name + 0s # leader id + 0s # consumer member group id + 0 # members + +read 105 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +write 112 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander/client.rpt new file mode 100644 index 0000000000..5165d45fd2 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander/client.rpt @@ -0,0 +1,221 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property networkConnectWindow 8192 +property instanceId ${kafka:randomBytes(42)} + +property newRequestId ${kafka:newRequestId()} +property fetchWaitMax 500 +property fetchBytesMax 65535 +property partitionBytesMax 8192 + +connect "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "session" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +write close +read abort + +read notify ROUTED_BROKER_SERVER + +connect await ROUTED_BROKER_SERVER + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 105 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 34 # size + (int:newRequestId) + 0 # throttle time + 79s # member id required + -1 # generated id + 0s # protocol name + 0s # leader id + 10s "memberId-1" # consumer member group id + 0 # members + +write 115 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 112 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment + +write 81 # size + 12s # heartbeat + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + +read 10 # size + (int:newRequestId) + 0 # throttle time + 27s # REBALANCE_IN_PROGRESS + +write 115 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + + +read 170 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 2 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + 10s "memberId-2" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 117 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 2 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + 10s "memberId-2" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment + +write 81 # size + 13s # leave group + 3s # v3 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # consumer group + 1 # assignments + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + +read 70 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 1 # assignments + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + +write close +read abort diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander/server.rpt new file mode 100644 index 0000000000..077b3cffc5 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.highlander/server.rpt @@ -0,0 +1,212 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property instanceId ${kafka:randomBytes(42)} + +accept "zilla://streams/net0" + option zilla:window 8192 + option zilla:transmission "duplex" + option zilla:byteorder "network" + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +read closed +write aborted + +accepted + +connected + +read 105 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + + +write 34 # size + ${newRequestId} + 0 # throttle time + 79s # member id required + -1 # generated id + 0s # protocol name + 0s # leader id + 10s "memberId-1" # consumer member group id + 0 # members + +read 115 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +write 112 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment + +read 81 # size + 12s # heartbeat + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + +write 10 # size + ${newRequestId} + 0 # throttle time + 27s # REBALANCE_IN_PROGRESS + +read 115 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + + +write 170 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 2 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + 10s "memberId-2" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 117 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 2 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + 10s "memberId-2" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment + +read 81 # size + 13s # leave group + 3s # v3 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # consumer group + 1 # assignments + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + +write 70 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 1 # assignments + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + +read closed +write aborted diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.unknown/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.unknown/client.rpt new file mode 100644 index 0000000000..03968c7817 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.unknown/client.rpt @@ -0,0 +1,108 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property networkConnectWindow 8192 +property instanceId ${kafka:randomBytes(42)} + +property newRequestId ${kafka:newRequestId()} +property fetchWaitMax 500 +property fetchBytesMax 65535 +property partitionBytesMax 8192 + +connect "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "session" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 #port + +write close +read abort + +read notify ROUTED_BROKER_SERVER + +connect await ROUTED_BROKER_SERVER + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 102 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 7s "unknown" # protocol name + 0 # metadata + +read 109 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 7s "unknown" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment + diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.unknown/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.unknown/server.rpt new file mode 100644 index 0000000000..aecd6f1d65 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.protocol.unknown/server.rpt @@ -0,0 +1,97 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property instanceId ${kafka:randomBytes(42)} + +accept "zilla://streams/net0" + option zilla:window 8192 + option zilla:transmission "duplex" + option zilla:byteorder "network" + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +read closed +write aborted + +accepted + +connected + +read 102 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 7s "unknown" # protocol name + 0 # metadata + +write 109 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 7s "unknown" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.sync.group/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.sync.group/client.rpt new file mode 100644 index 0000000000..8d198b5a1f --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.sync.group/client.rpt @@ -0,0 +1,154 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property networkConnectWindow 8192 +property instanceId ${kafka:randomBytes(42)} + +property newRequestId ${kafka:newRequestId()} +property fetchWaitMax 500 +property fetchBytesMax 65535 +property partitionBytesMax 8192 + +connect "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "session" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +write close +read abort + +read notify ROUTED_BROKER_SERVER + +connect await ROUTED_BROKER_SERVER + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 105 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 112 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 27s # rebalance + 0 # assignment + +write 115 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 112 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 4 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 4 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.sync.group/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.sync.group/server.rpt new file mode 100644 index 0000000000..7543f130d7 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3/rebalance.sync.group/server.rpt @@ -0,0 +1,144 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property instanceId ${kafka:randomBytes(42)} + +accept "zilla://streams/net0" + option zilla:window 8192 + option zilla:transmission "duplex" + option zilla:byteorder "network" + +accepted + +connected + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4s "none" # error message none + 1 # coordinator node + 9s "localhost" # host + 9092 # port + +read closed +write aborted + +accepted + +connected + +read 105 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +write 112 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 27s # rebalance + 0 # assignment + +read 115 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 10s "memberId-1" # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +write 112 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 4 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 4 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.sasl.f1.j5.s3.l3.h3.handshake.v1/leader/client.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.sasl.f1.j5.s3.l3.h3.handshake.v1/leader/client.rpt new file mode 100644 index 0000000000..a12b220d76 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.sasl.f1.j5.s3.l3.h3.handshake.v1/leader/client.rpt @@ -0,0 +1,165 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property networkConnectWindow 8192 +property instanceId ${kafka:randomBytes(42)} + +property newRequestId ${kafka:newRequestId()} +property fetchWaitMax 500 +property fetchBytesMax 65535 +property partitionBytesMax 8192 + +connect "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 17 # size + 17s # sasl.handshake + 1s # v1 + ${newRequestId} + -1s # no client id + 5s "PLAIN" # mechanism + +read 17 # size + ${newRequestId} + 0s # no error + 1 # mechanisms + 5s "PLAIN" # PLAIN + +write 32 # size + 36s # sasl.authenticate + 1s # v1 + ${newRequestId} + -1s # no client id + 18 + [0x00] "username" # authentication bytes + [0x00] "password" + +read 20 # size + ${newRequestId} + 0s # no error + -1 + -1s # authentication bytes + 0L # session lifetime + +write 22 # size + 10s # find coordinator + 1s # v1 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # "session" coordinator key + [0x00] # coordinator group type + +read 35 # size + (int:newRequestId) + 0 #throttle time + 0s #no error + 4s "none" #error message none + 1 #coordinator node + 9s "localhost" #host + 9092 #port + +write close +read abort + +read notify ROUTED_BROKER_SERVER + +connect await ROUTED_BROKER_SERVER + "zilla://streams/net0" + option zilla:window ${networkConnectWindow} + option zilla:transmission "duplex" + option zilla:byteorder "network" + +connected + +write 17 # size + 17s # sasl.handshake + 1s # v1 + ${newRequestId} + -1s # no client id + 5s "PLAIN" # mechanism + +read 17 # size + ${newRequestId} + 0s # no error + 1 # mechanisms + 5s "PLAIN" # PLAIN + +write 32 # size + 36s # sasl.authenticate + 1s # v1 + ${newRequestId} + -1s # no client id + 18 + [0x00] "username" # authentication bytes + [0x00] "password" + +read 20 # size + ${newRequestId} + 0s # no error + -1 + -1s # authentication bytes + 0L # session lifetime + +write 105 # size + 11s # join group + 5s # v5 + ${newRequestId} + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s ${instanceId} # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + +read 112 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 0 # metadata + +write 101 # size + 14s # sync group + 3s # v3 + ${newRequestId} + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +read 14 # size + (int:newRequestId) + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.sasl.f1.j5.s3.l3.h3.handshake.v1/leader/server.rpt b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.sasl.f1.j5.s3.l3.h3.handshake.v1/leader/server.rpt new file mode 100644 index 0000000000..7d5eea0df9 --- /dev/null +++ b/specs/binding-kafka.spec/src/main/scripts/io/aklivity/zilla/specs/binding/kafka/streams/network/group.sasl.f1.j5.s3.l3.h3.handshake.v1/leader/server.rpt @@ -0,0 +1,157 @@ +# +# Copyright 2021-2023 Aklivity Inc. +# +# Aklivity 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. +# + +property instanceId ${kafka:randomBytes(42)} + +accept "zilla://streams/net0" + option zilla:window 8192 + option zilla:transmission "duplex" + option zilla:byteorder "network" + +accepted + +connected + +read 17 # size + 17s # sasl.handshake + 1s # v1 + (int:requestId) + -1s # no client id + 5s "PLAIN" # mechanism + +write 17 # size + ${requestId} + 0s # no error + 1 # mechanisms + 5s "PLAIN" # PLAIN + +read 32 # size + 36s # sasl.authenticate + 1s # v1 + (int:requestId) + -1s # no client id + 18 + [0x00] "username" # authentication bytes + [0x00] "password" + +write 20 # size + ${requestId} + 0s # no error + -1 + -1s # authentication bytes + 0L # session lifetime + +read 22 # size + 10s # find coordinator + 1s # v1 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # "test" coordinator key + [0x00] # coordinator group type + +write 35 # size + ${newRequestId} + 0 #throttle time + 0s #no error + 4s "none" #error message none + 1 #coordinator node + 9s "localhost" #host + 9092 #port + +read closed +write aborted + +accepted + +connected + +read 17 # size + 17s # sasl.handshake + 1s # v1 + (int:requestId) + -1s # no client id + 5s "PLAIN" # mechanism + +write 17 # size + ${requestId} + 0s # no error + 1 # mechanisms + 5s "PLAIN" # PLAIN + +read 32 # size + 36s # sasl.authenticate + 1s # v1 + (int:requestId) + -1s # no client id + 18 + [0x00] "username" # authentication bytes + [0x00] "password" + +write 20 # size + ${requestId} + 0s # no error + -1 + -1s # authentication bytes + 0L # session lifetime + +read 105 # size + 11s # join group + 5s # v5 + (int:newRequestId) + 5s "zilla" # client id + 4s "test" # consumer group + 45000 # session timeout + 4000 # rebalance timeout + 0s # consumer group member + 42s [0..42] # group instance id + 8s "consumer" # protocol type + 1 # group protocol + 10s "highlander" # protocol name + 0 # metadata + + + +write 112 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 3 # generated id + 10s "highlander" # protocol name + 10s "memberId-1" # leader id + 10s "memberId-1" # consumer member group id + 1 # members + 10s "memberId-1" # consumer member group id + 42s ${instanceId} # group instance id + 0 # metadata + +read 101 # size + 14s # sync group + 3s # v3 + (int:newRequestId) + 5s "zilla" # no client id + 4s "test" # consumer group + 3 # generation id + 10s "memberId-1" # consumer member group id + 42s [0..42] # group instance id + 1 # assignments + 10s "memberId-1" # consumer member group id + 0 # metadata + +write 14 # size + ${newRequestId} + 0 # throttle time + 0s # no error + 0 # assignment diff --git a/specs/binding-kafka.spec/src/test/java/io/aklivity/zilla/specs/binding/kafka/streams/application/GroupIT.java b/specs/binding-kafka.spec/src/test/java/io/aklivity/zilla/specs/binding/kafka/streams/application/GroupIT.java new file mode 100644 index 0000000000..dbf8186465 --- /dev/null +++ b/specs/binding-kafka.spec/src/test/java/io/aklivity/zilla/specs/binding/kafka/streams/application/GroupIT.java @@ -0,0 +1,84 @@ +/* + * Copyright 2021-2023 Aklivity Inc. + * + * Aklivity 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 io.aklivity.zilla.specs.binding.kafka.streams.application; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.junit.rules.RuleChain.outerRule; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.DisableOnDebug; +import org.junit.rules.TestRule; +import org.junit.rules.Timeout; +import org.kaazing.k3po.junit.annotation.Specification; +import org.kaazing.k3po.junit.rules.K3poRule; + + +public class GroupIT +{ + private final K3poRule k3po = new K3poRule() + .addScriptRoot("app", "io/aklivity/zilla/specs/binding/kafka/streams/application/group"); + + private final TestRule timeout = new DisableOnDebug(new Timeout(5, SECONDS)); + + @Rule + public final TestRule chain = outerRule(k3po).around(timeout); + + @Test + @Specification({ + "${app}/rebalance.protocol.highlander/client", + "${app}/rebalance.protocol.highlander/server"}) + public void shouldLeaveGroupOnGroupRebalanceError() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${app}/client.sent.write.abort.before.coordinator.response/client", + "${app}/client.sent.write.abort.before.coordinator.response/server"}) + public void shouldHandleClientSentWriteAbortBeforeCoordinatorResponse() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${app}/leader/client", + "${app}/leader/server"}) + public void shouldBecameLeader() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${app}/rebalance.protocol.highlander.migrate.leader/client", + "${app}/rebalance.protocol.highlander.migrate.leader/server"}) + public void shouldRebalanceProtocolHighlanderMigrateLeader() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${app}/rebalance.protocol.unknown/client", + "${app}/rebalance.protocol.unknown/server"}) + public void shouldRejectSecondStreamOnUnknownProtocol() throws Exception + { + k3po.finish(); + } +} diff --git a/specs/binding-kafka.spec/src/test/java/io/aklivity/zilla/specs/binding/kafka/streams/network/GroupIT.java b/specs/binding-kafka.spec/src/test/java/io/aklivity/zilla/specs/binding/kafka/streams/network/GroupIT.java new file mode 100644 index 0000000000..f9bd895508 --- /dev/null +++ b/specs/binding-kafka.spec/src/test/java/io/aklivity/zilla/specs/binding/kafka/streams/network/GroupIT.java @@ -0,0 +1,110 @@ +/* + * Copyright 2021-2023 Aklivity Inc. + * + * Aklivity 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 io.aklivity.zilla.specs.binding.kafka.streams.network; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.junit.rules.RuleChain.outerRule; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.DisableOnDebug; +import org.junit.rules.TestRule; +import org.junit.rules.Timeout; +import org.kaazing.k3po.junit.annotation.Specification; +import org.kaazing.k3po.junit.rules.K3poRule; + +public class GroupIT +{ + private final K3poRule k3po = new K3poRule() + .addScriptRoot("net", "io/aklivity/zilla/specs/binding/kafka/streams/network/group.f1.j5.s3.l3.h3"); + + private final TestRule timeout = new DisableOnDebug(new Timeout(5, SECONDS)); + + @Rule + public final TestRule chain = outerRule(k3po).around(timeout); + + @Test + @Specification({ + "${net}/rebalance.protocol.highlander/client", + "${net}/rebalance.protocol.highlander/server"}) + public void shouldLeaveGroupOnGroupRebalanceError() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${net}/client.sent.write.abort.before.coordinator.response/client", + "${net}/client.sent.write.abort.before.coordinator.response/server"}) + public void shouldHandleClientSentWriteAbortBeforeCoordinatorResponse() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${net}/coordinator.not.available/client", + "${net}/coordinator.not.available/server"}) + public void shouldHandleCoordinatorNotAvailableError() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${net}/coordinator.reject.invalid.consumer/client", + "${net}/coordinator.reject.invalid.consumer/server"}) + public void shouldHRejectInvalidConsumer() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${net}/rebalance.protocol.highlander.unknown.member.id/client", + "${net}/rebalance.protocol.highlander.unknown.member.id/server"}) + public void shouldRebalanceProtocolHighlanderUnknownMemberId() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${net}/rebalance.protocol.highlander.migrate.leader/client", + "${net}/rebalance.protocol.highlander.migrate.leader/server"}) + public void shouldRebalanceProtocolHighlanderMigrateLeader() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${net}/rebalance.protocol.unknown/client", + "${net}/rebalance.protocol.unknown/server"}) + public void shouldRejectSecondStreamOnUnknownProtocol() throws Exception + { + k3po.finish(); + } + + @Test + @Specification({ + "${net}/rebalance.sync.group//client", + "${net}/rebalance.sync.group/server"}) + public void shouldHandleRebalanceSyncGroup() throws Exception + { + k3po.finish(); + } +} diff --git a/specs/binding-kafka.spec/src/test/java/io/aklivity/zilla/specs/binding/kafka/streams/network/GroupSaslIT.java b/specs/binding-kafka.spec/src/test/java/io/aklivity/zilla/specs/binding/kafka/streams/network/GroupSaslIT.java new file mode 100644 index 0000000000..dc2c16378c --- /dev/null +++ b/specs/binding-kafka.spec/src/test/java/io/aklivity/zilla/specs/binding/kafka/streams/network/GroupSaslIT.java @@ -0,0 +1,49 @@ +/* + * Copyright 2021-2023 Aklivity Inc. + * + * Aklivity 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 io.aklivity.zilla.specs.binding.kafka.streams.network; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.junit.rules.RuleChain.outerRule; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.DisableOnDebug; +import org.junit.rules.TestRule; +import org.junit.rules.Timeout; +import org.kaazing.k3po.junit.annotation.Specification; +import org.kaazing.k3po.junit.rules.K3poRule; + + +public class GroupSaslIT +{ + private final K3poRule k3po = new K3poRule() + .addScriptRoot("net", + "io/aklivity/zilla/specs/binding/kafka/streams/network/group.sasl.f1.j5.s3.l3.h3.handshake.v1"); + + private final TestRule timeout = new DisableOnDebug(new Timeout(5, SECONDS)); + + @Rule + public final TestRule chain = outerRule(k3po).around(timeout); + + @Test + @Specification({ + "${net}/leader/client", + "${net}/leader/server"}) + public void shouldBecameLeader() throws Exception + { + k3po.finish(); + } +}