From a96d0a4af8258c3a08ae1f2cbdb5d4169e94f425 Mon Sep 17 00:00:00 2001 From: Jia Zhai Date: Wed, 25 Dec 2019 23:22:57 +0800 Subject: [PATCH] [Issue #32] Consumer group cluster (#52) fix Issue #32 Make Consumer Group work distributed. Add unit test for multi-brokers. Pass tests for 2 brokers with `bin/kop kafka-broker` --- conf/kop.conf | 21 +- conf/kop_standalone.conf | 1 + pom.xml | 7 - .../streamnative/kop/KafkaCommandDecoder.java | 25 + .../kop/KafkaProtocolHandler.java | 168 +- .../streamnative/kop/KafkaRequestHandler.java | 280 +- .../io/streamnative/kop/KafkaService.java | 128 +- .../kop/KafkaServiceConfiguration.java | 6 + .../streamnative/kop/KafkaTopicManager.java | 12 + .../streamnative/kop/MessageFetchContext.java | 2 +- .../coordinator/group/GroupCoordinator.java | 39 +- .../group/GroupMetadataManager.java | 316 ++- .../kop/coordinator/group/OffsetConfig.java | 8 +- .../kop/utils/TopicNameUtils.java | 1 + .../pulsar/client/ReaderBuilderImpl2.java | 170 ++ .../apache/pulsar/client/package-info.java | 19 + .../io/streamnative/kop/KafkaApisTest.java | 3 +- .../kop/KafkaRequestTypeTest.java | 2 +- .../io/streamnative/kop/LogOffsetTest.java | 3 +- .../kop/MockKafkaServiceBaseTest.java | 30 +- .../io/streamnative/kop/SaslPlainTest.java | 10 +- .../DistributedGroupCoordinatorTest.java | 401 +++ .../group/GroupCoordinatorTest.java | 50 +- .../group/GroupMetadataManagerTest.java | 2317 ++++++++--------- 24 files changed, 2424 insertions(+), 1595 deletions(-) create mode 100644 src/main/java/org/apache/pulsar/client/ReaderBuilderImpl2.java create mode 100644 src/main/java/org/apache/pulsar/client/package-info.java create mode 100644 src/test/java/io/streamnative/kop/coordinator/group/DistributedGroupCoordinatorTest.java diff --git a/conf/kop.conf b/conf/kop.conf index 6866cc67cf533..68cb2c47a60f0 100755 --- a/conf/kop.conf +++ b/conf/kop.conf @@ -15,10 +15,24 @@ ### --- Kafka broker settings --- ### -kafkaServicePort=9092 - enableGroupCoordinator=true +messagingProtocols=kafka + +listeners=PLAINTEXT://127.0.0.1:9092 + +offsetsTopicNumPartitions=8 + +advertisedAddress=127.0.0.1 + +### --- Changed for KoP --- ### + +# Enable the deletion of inactive topics +brokerDeleteInactiveTopicsEnabled=false + +# Name of the cluster to which this broker belongs to +clusterName=kafka-cluster + ### --- General broker settings --- ### # Zookeeper quorum connection string @@ -54,9 +68,6 @@ numHttpServerThreads= # Flag to control features that are meant to be used when running in standalone mode isRunningStandalone= -# Name of the cluster to which this broker belongs to -clusterName=kafka-cluster - # Enable cluster's failure-domain which can distribute brokers into logical region failureDomainsEnabled=false diff --git a/conf/kop_standalone.conf b/conf/kop_standalone.conf index 36cc2ef666495..5978b2be3fbe2 100644 --- a/conf/kop_standalone.conf +++ b/conf/kop_standalone.conf @@ -21,6 +21,7 @@ messagingProtocols=kafka listeners=PLAINTEXT://127.0.0.1:9092 +offsetsTopicNumPartitions=8 ### --- Changed for KoP --- ### diff --git a/pom.xml b/pom.xml index 4f67756122916..8f739844127ca 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,6 @@ 2.10.0 1.18.4 2.22.0 - 4.1.32.Final 2.5.0-9728d5429 1.7.25 3.1.8 @@ -123,12 +122,6 @@ ${pulsar.version} - - io.netty - netty-all - ${netty.version} - - com.google.guava guava diff --git a/src/main/java/io/streamnative/kop/KafkaCommandDecoder.java b/src/main/java/io/streamnative/kop/KafkaCommandDecoder.java index 200f4c908fb6c..d055ad58aa0ac 100644 --- a/src/main/java/io/streamnative/kop/KafkaCommandDecoder.java +++ b/src/main/java/io/streamnative/kop/KafkaCommandDecoder.java @@ -38,6 +38,7 @@ import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.requests.ResponseHeader; + /** * A decoder that decodes kafka requests and responses. */ @@ -65,6 +66,30 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { } } + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + super.channelInactive(ctx); + log.info("Closed connection from {}", remoteAddress); + close(); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + log.error("[{}] Got exception: {}", remoteAddress, cause.getMessage(), cause); + close(); + } + + protected void close() { + ctx.close(); + } + + @Override + public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception { + if (log.isDebugEnabled()) { + log.debug("Channel writability has changed to: {}", ctx.channel().isWritable()); + } + } + // turn input ByteBuf msg, which send from client side, into KafkaHeaderAndRequest protected KafkaHeaderAndRequest byteBufToRequest(ByteBuf msg) { return byteBufToRequest(msg, null); diff --git a/src/main/java/io/streamnative/kop/KafkaProtocolHandler.java b/src/main/java/io/streamnative/kop/KafkaProtocolHandler.java index ec86a3ab09882..e555908b36f48 100644 --- a/src/main/java/io/streamnative/kop/KafkaProtocolHandler.java +++ b/src/main/java/io/streamnative/kop/KafkaProtocolHandler.java @@ -14,8 +14,11 @@ package io.streamnative.kop; import static com.google.common.base.Preconditions.checkState; +import static io.streamnative.kop.utils.TopicNameUtils.getKafkaTopicNameFromPulsarTopicname; +import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import com.google.common.collect.Sets; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; @@ -26,8 +29,10 @@ import io.streamnative.kop.utils.timer.SystemTimer; import java.net.InetSocketAddress; import java.nio.ByteBuffer; +import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; @@ -37,19 +42,25 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.ServiceConfigurationUtils; +import org.apache.pulsar.broker.namespace.NamespaceBundleOwnershipListener; import org.apache.pulsar.broker.protocol.ProtocolHandler; import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.client.ReaderBuilderImpl2; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.Reader; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.NamespaceBundle; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.util.FutureUtil; /** * Kafka Protocol Handler load and run by Pulsar Service. @@ -64,6 +75,88 @@ public class KafkaProtocolHandler implements ProtocolHandler { public static final String TLS_HANDLER = "tls"; public static final String LISTENER_PATTEN = "^(PLAINTEXT?|SSL)://[-a-zA-Z0-9+&@#/%?=~_|!:,.;]*[-0-9+]"; + /** + * Listener for the changing of topic that stores offsets of consumer group. + */ + public static class OffsetTopicListener implements NamespaceBundleOwnershipListener { + + final BrokerService service; + final NamespaceName kafkaMetaNs; + final GroupCoordinator groupCoordinator; + public OffsetTopicListener(BrokerService service, + KafkaServiceConfiguration kafkaConfig, + GroupCoordinator groupCoordinator) { + this.service = service; + this.kafkaMetaNs = NamespaceName + .get(kafkaConfig.getKafkaMetadataTenant(), kafkaConfig.getKafkaMetadataNamespace()); + this.groupCoordinator = groupCoordinator; + } + + @Override + public void onLoad(NamespaceBundle bundle) { + // 1. get new partitions owned by this pulsar service. + // 2. load partitions by GroupCoordinator.handleGroupImmigration. + service.pulsar().getNamespaceService().getOwnedTopicListForNamespaceBundle(bundle) + .whenComplete((topics, ex) -> { + if (ex == null) { + for (String topic : topics) { + TopicName name = TopicName.get(topic); + // already filtered namespace, check the local name without partition + if (Topic.GROUP_METADATA_TOPIC_NAME.equals(getKafkaTopicNameFromPulsarTopicname(name))) { + checkState(name.isPartitioned(), + "OffsetTopic should be partitioned in onLoad, but get " + name); + if (log.isDebugEnabled()) { + log.debug("New offset partition load: {}, broker: {}", + name, service.pulsar().getBrokerServiceUrl()); + } + groupCoordinator.handleGroupImmigration(name.getPartitionIndex()); + } + } + } else { + log.error("Failed to get owned topic list for " + + "OffsetTopicListener when triggering on-loading bundle {}.", + bundle, ex); + } + }); + } + + @Override + public void unLoad(NamespaceBundle bundle) { + // 1. get partitions owned by this pulsar service. + // 2. remove partitions by groupCoordinator.handleGroupEmigration. + service.pulsar().getNamespaceService().getOwnedTopicListForNamespaceBundle(bundle) + .whenComplete((topics, ex) -> { + if (ex == null) { + for (String topic : topics) { + TopicName name = TopicName.get(topic); + + // already filtered namespace, check the local name without partition + if (Topic.GROUP_METADATA_TOPIC_NAME.equals(getKafkaTopicNameFromPulsarTopicname(name))) { + checkState(name.isPartitioned(), + "OffsetTopic should be partitioned in unLoad, but get " + name); + if (log.isDebugEnabled()) { + log.debug("Offset partition unload: {}, broker: {}", + name, service.pulsar().getBrokerServiceUrl()); + } + groupCoordinator.handleGroupEmigration(name.getPartitionIndex()); + } + } + } else { + log.error("Failed to get owned topic list for " + + "OffsetTopicListener when triggering un-loading bundle {}.", + bundle, ex); + } + }); + } + + // verify that this bundle is served by this broker, + // and namespace is related to kafka metadata namespace + @Override + public boolean test(NamespaceBundle namespaceBundle) { + return namespaceBundle.getNamespaceObject().equals(kafkaMetaNs); + } + + } /** * Kafka Listener Type. */ @@ -128,6 +221,11 @@ public void start(BrokerService service) { try { initGroupCoordinator(brokerService); startGroupCoordinator(); + // and listener for Offset topics load/unload + brokerService.pulsar() + .getNamespaceService() + .addNamespaceBundleOwnershipListener( + new OffsetTopicListener(brokerService, kafkaConfig, groupCoordinator)); } catch (Exception e) { log.error("initGroupCoordinator failed with", e); } @@ -199,6 +297,10 @@ public void initGroupCoordinator(BrokerService service) throws Exception { ); OffsetConfig offsetConfig = OffsetConfig.builder() + .offsetsTopicName(kafkaConfig.getKafkaMetadataTenant() + "/" + + kafkaConfig.getKafkaMetadataNamespace() + + "/" + Topic.GROUP_METADATA_TOPIC_NAME) + .offsetsTopicNumPartitions(kafkaConfig.getOffsetsTopicNumPartitions()) .offsetsTopicCompressionType(CompressionType.valueOf(kafkaConfig.getOffsetsTopicCompressionCodec())) .maxMetadataSize(kafkaConfig.getOffsetMetadataMaxSize()) .offsetsRetentionCheckIntervalMs(kafkaConfig.getOffsetsRetentionCheckIntervalMs()) @@ -206,20 +308,19 @@ public void initGroupCoordinator(BrokerService service) throws Exception { .build(); createKafkaMetadataNamespaceIfNeeded(service); - String offsetsTopic = createKafkaOffsetsTopic(service); - - TopicName offsetsTopicName = TopicName.get(offsetsTopic); - String offsetsTopicPtn0 = offsetsTopicName.getPartition(0).toString(); - - Producer groupCoordinatorTopicProducer = service.pulsar().getClient().newProducer(Schema.BYTEBUFFER) - .topic(offsetsTopicPtn0) - // TODO: make it configurable - .maxPendingMessages(100000) - .create(); - Reader groupCoordinatorTopicReader = service.pulsar().getClient().newReader(Schema.BYTEBUFFER) - .topic(offsetsTopicPtn0) - .startMessageId(MessageId.earliest) - .create(); + // topicName in pulsar format: tenant/ns/topic + createKafkaOffsetsTopic(service); + + ProducerBuilder groupCoordinatorTopicProducer = service.pulsar().getClient() + .newProducer(Schema.BYTEBUFFER) + .maxPendingMessages(100000); + + // TODO: replace this back to service.pulsar().getClient().newReader after merge pulsar PR: + // https://github.com/apache/pulsar/pull/5923 + ReaderBuilder groupCoordinatorTopicReader = + new ReaderBuilderImpl2<>((PulsarClientImpl) (service.pulsar().getClient()), Schema.BYTEBUFFER); + groupCoordinatorTopicReader.startMessageId(MessageId.earliest); + this.groupCoordinator = GroupCoordinator.of( groupCoordinatorTopicProducer, groupCoordinatorTopicReader, @@ -230,10 +331,10 @@ public void initGroupCoordinator(BrokerService service) throws Exception { .build(), Time.SYSTEM ); + + loadOffsetTopics(groupCoordinator); } - // TODO: make group coordinator running in a distributed mode - // https://github.com/streamnative/kop/issues/32 public void startGroupCoordinator() throws Exception { if (this.groupCoordinator != null) { this.groupCoordinator.startup(false); @@ -289,14 +390,41 @@ private String createKafkaOffsetsTopic(BrokerService service) throws PulsarServe offsetsTopic); service.pulsar().getAdminClient().topics().createPartitionedTopic( offsetsTopic, - KafkaServiceConfiguration.DefaultOffsetsTopicNumPartitions + kafkaConfig.getOffsetsTopicNumPartitions() ); - log.info("Successfully created group metadata topic {}.", offsetsTopic); + for (int i = 0; i < kafkaConfig.getOffsetsTopicNumPartitions(); i++) { + service.pulsar().getAdminClient().topics() + .createNonPartitionedTopic(offsetsTopic + PARTITIONED_TOPIC_SUFFIX + i); + } + log.info("Successfully created group metadata topic {} with {} partitions.", + offsetsTopic, kafkaConfig.getOffsetsTopicNumPartitions()); } return offsetsTopic; } + private void loadOffsetTopics(GroupCoordinator groupCoordinator) throws Exception { + String offsetsTopic = kafkaConfig.getKafkaMetadataTenant() + "/" + kafkaConfig.getKafkaMetadataNamespace() + + "/" + Topic.GROUP_METADATA_TOPIC_NAME; + int numPartitions = kafkaConfig.getOffsetsTopicNumPartitions(); + List> lists = Lists.newArrayListWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; i++) { + String partition = offsetsTopic + PARTITIONED_TOPIC_SUFFIX + i; + String broker = brokerService.pulsar().getAdminClient().lookups() + .lookupTopic(partition); + + if (log.isDebugEnabled()) { + log.debug("found broker {} for offset topic partition {}. current broker: {}", + broker, partition, brokerService.pulsar().getBrokerServiceUrl()); + } + + if (broker.equalsIgnoreCase(brokerService.pulsar().getBrokerServiceUrl())) { + lists.add(groupCoordinator.handleGroupImmigration(i)); + } + } + FutureUtil.waitForAll(lists).get(); + } + public static int getListenerPort(String listener) { checkState(listener.matches(LISTENER_PATTEN), "listener not match patten"); diff --git a/src/main/java/io/streamnative/kop/KafkaRequestHandler.java b/src/main/java/io/streamnative/kop/KafkaRequestHandler.java index a258878945218..20769c75c524a 100644 --- a/src/main/java/io/streamnative/kop/KafkaRequestHandler.java +++ b/src/main/java/io/streamnative/kop/KafkaRequestHandler.java @@ -62,6 +62,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.NotImplementedException; +import org.apache.commons.lang3.tuple.Pair; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.Errors; @@ -105,8 +106,6 @@ import org.apache.kafka.common.requests.SyncGroupRequest; import org.apache.kafka.common.requests.SyncGroupResponse; import org.apache.kafka.common.utils.Utils; -import org.apache.pulsar.broker.PulsarServerException; -import org.apache.pulsar.broker.PulsarServerException.NotFoundException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfigurationUtils; import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; @@ -115,20 +114,20 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authentication.AuthenticationState; import org.apache.pulsar.broker.loadbalance.LoadManager; -import org.apache.pulsar.broker.lookup.LookupResult; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.AuthorizationException; +import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.api.AuthData; -import org.apache.pulsar.common.lookup.data.LookupData; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.util.Murmur3_32Hash; import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; +import org.apache.pulsar.zookeeper.ZooKeeperCache; /** * This class contains all the request handling methods. @@ -395,18 +394,18 @@ protected CompletableFuture handleTopicMetadataRequest(Kafka // whether completed all the topics requests. int finishedTopics = topicsCompleted.incrementAndGet(); - if (log.isDebugEnabled()) { - log.debug("[{}] Request {}: Completed findBroker for topic {}, " + if (log.isTraceEnabled()) { + log.trace("[{}] Request {}: Completed findBroker for topic {}, " + "partitions found/all: {}/{}. \n dump All Metadata:", ctx.channel(), metadataHar.getHeader(), topic, finishedTopics, topicsNumber); allTopicMetadata.stream() .forEach(data -> { - log.debug("topicMetadata: {}", data.toString()); + log.trace("topicMetadata: {}", data.toString()); data.partitionMetadata() .forEach(partitionData -> - log.debug(" partitionMetadata: {}", data.toString())); + log.trace(" partitionMetadata: {}", data.toString())); }); } if (finishedTopics == topicsNumber) { @@ -501,7 +500,6 @@ protected CompletableFuture handleProduceRequest(KafkaHeader return resultFuture; } - // A simple implementation, returns this broker node. protected CompletableFuture handleFindCoordinatorRequest(KafkaHeaderAndRequest findCoordinator) { checkArgument(findCoordinator.getRequest() instanceof FindCoordinatorRequest); @@ -509,28 +507,31 @@ protected CompletableFuture handleProduceRequest(KafkaHeader CompletableFuture resultFuture = new CompletableFuture<>(); if (request.coordinatorType() == FindCoordinatorRequest.CoordinatorType.GROUP) { - AbstractResponse response; - try { - Node node = newSelfNode(); - if (log.isDebugEnabled()) { - log.debug("[{}] Request {}: Return current broker node as Coordinator: {}.", - ctx.channel(), findCoordinator.getHeader(), node); - } + int partition = groupCoordinator.partitionFor(request.coordinatorKey()); + String pulsarTopicName = groupCoordinator.getTopicPartitionName(partition); + + findBroker(pulsarService, TopicName.get(pulsarTopicName)) + .thenApply(partitionMetadata -> partitionMetadata.leader()) + .whenComplete((node, t) -> { + if (t != null){ + log.error("[{}] Request {}: Error while find coordinator.", + ctx.channel(), findCoordinator.getHeader(), t); + return; + } - response = new FindCoordinatorResponse( - Errors.NONE, - node); - } catch (Exception e) { - log.error("[{}] Request {}: Error while find coordinator.", - ctx.channel(), findCoordinator.getHeader(), e); - response = new FindCoordinatorResponse( - Errors.COORDINATOR_NOT_AVAILABLE, - Node.noNode()); - } + if (log.isDebugEnabled()) { + log.debug("[{}] Found node {} as coordinator for key {} partition {}.", + ctx.channel(), node, request.coordinatorKey(), partition); + } - resultFuture.complete(ResponseAndRequest.of(response, findCoordinator)); + AbstractResponse response = new FindCoordinatorResponse( + Errors.NONE, + node); + resultFuture.complete(ResponseAndRequest.of(response, findCoordinator)); + }); } else { - throw new NotImplementedException("FindCoordinatorRequest not support TRANSACTION type"); + throw new NotImplementedException("FindCoordinatorRequest not support TRANSACTION type " + + request.coordinatorType()); } return resultFuture; @@ -720,14 +721,18 @@ protected CompletableFuture handleListOffsetRequest(KafkaHea // For non exist topics handleOffsetCommitRequest return UNKNOWN_TOPIC_OR_PARTITION private Map nonExistingTopicErrors(OffsetCommitRequest request) { - return request.offsetData().entrySet().stream() - .filter(entry -> - // filter not exist topics - !topicManager.topicExists(pulsarTopicName(entry.getKey(), namespace).toString())) - .collect(Collectors.toMap( - e -> e.getKey(), - e -> Errors.UNKNOWN_TOPIC_OR_PARTITION - )); + // TODO: in Kafka Metadata cache, all topics in the cluster is included, we should support it? + // we could get all the topic info by listTopic? + // https://github.com/streamnative/kop/issues/51 + return Maps.newHashMap(); +// return request.offsetData().entrySet().stream() +// .filter(entry -> +// // filter not exist topics +// !topicManager.topicExists(pulsarTopicName(entry.getKey(), namespace).toString())) +// .collect(Collectors.toMap( +// e -> e.getKey(), +// e -> Errors.UNKNOWN_TOPIC_OR_PARTITION +// )); } protected CompletableFuture handleOffsetCommitRequest(KafkaHeaderAndRequest offsetCommit) { @@ -1031,49 +1036,81 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { ctx.close(); } - private CompletableFuture> getProtocolDataToAdvertise(Optional lookupResult, - TopicName topic) { + private CompletableFuture> + getProtocolDataToAdvertise(Pair pulsarAddress, + TopicName topic) { if (log.isDebugEnabled()) { - log.debug("Handle getProtocolDataToAdvertise for {}", topic); + log.debug("Found broker for topic {} logicalAddress: {} physicalAddress: {}", + topic, pulsarAddress.getLeft(), pulsarAddress.getRight()); } - if (!lookupResult.isPresent()) { - log.error("Can't find broker for topic {}", topic); - CompletableFuture> future = new CompletableFuture<>(); - future.completeExceptionally(new NotFoundException("Can't find broker for topic " + topic)); - return future; - } + checkState(pulsarAddress.getLeft().equals(pulsarAddress.getRight())); + InetSocketAddress brokerAddress = pulsarAddress.getLeft(); - LookupData lookupData = lookupResult.get().getLookupData(); - String candidateBroker = lookupData.getBrokerUrl(); - URI uri; + CompletableFuture> returnFuture = new CompletableFuture<>(); - try { - uri = new URI(candidateBroker); - } catch (Exception e) { - log.error("Failed to get URI from {} for topic {}", candidateBroker, topic); - CompletableFuture> future = new CompletableFuture<>(); - future.completeExceptionally(new PulsarServerException(e)); - return future; - } + // advertised data is write in /loadbalance/brokers/advertisedAddress:webServicePort + // here we get the broker url, need to find related webServiceUrl. + ZooKeeperCache zkCache = pulsarService.getLocalZkCache(); + zkCache.getChildrenAsync(LoadManager.LOADBALANCE_BROKERS_ROOT, zkCache) + .whenComplete((set, throwable) -> { + if (throwable != null) { + log.error("Error in getChildrenAsync(zk://loadbalance) for {}", brokerAddress, throwable); + returnFuture.complete(Optional.empty()); + return; + } - // advertised data is write in /loadbalance/brokers/broker_host:webServicePort - int port = kafkaConfig.getWebServicePort().isPresent() ? kafkaConfig.getWebServicePort().get() - : kafkaConfig.getWebServicePortTls().get(); - String path = String.format("%s/%s:%s", LoadManager.LOADBALANCE_BROKERS_ROOT, uri.getHost(), - port); + String hostAndPort = brokerAddress.getHostName() + ":" + brokerAddress.getPort(); + List matchBrokers = Lists.newArrayList(); + // match host part of url + for (String activeBroker : set) { + if (activeBroker.startsWith(brokerAddress.getHostName() + ":")) { + matchBrokers.add(activeBroker); + } + } - return pulsarService.getLocalZkCache() - .getDataAsync(path, pulsarService.getLoadManager().get().getLoadReportDeserializer()) - .thenApply(reportData -> { - if (reportData.isPresent()) { - ServiceLookupData data = reportData.get(); - return data.getProtocol(KafkaProtocolHandler.PROTOCOL_NAME); - } else { - log.error("No node for broker data: {}", path); - return Optional.empty(); + if (matchBrokers.isEmpty()) { + log.error("No node for broker {} under zk://loadbalance", brokerAddress); + returnFuture.complete(Optional.empty()); + return; } + + AtomicInteger atomicInteger = new AtomicInteger(matchBrokers.size()); + matchBrokers.stream().forEach(matchBroker -> { + String path = String.format("%s/%s", LoadManager.LOADBALANCE_BROKERS_ROOT, + matchBroker); + zkCache.getDataAsync(path, pulsarService.getLoadManager().get().getLoadReportDeserializer()) + .whenComplete((serviceLookupData, th) -> { + int wait = atomicInteger.decrementAndGet(); + if (th != null) { + log.error("Error in getDataAsync({}) for {}", path, brokerAddress, th); + returnFuture.complete(Optional.empty()); + return; + } + + if (log.isDebugEnabled()) { + log.debug("Handle getProtocolDataToAdvertise for {}, pulsarUrl: {}, kafka: {}", + topic, serviceLookupData.get().getPulsarServiceUrl(), + serviceLookupData.get().getProtocol(KafkaProtocolHandler.PROTOCOL_NAME)); + } + + ServiceLookupData data = serviceLookupData.get(); + if (data.getPulsarServiceUrl().contains(hostAndPort) + || data.getPulsarServiceUrlTls().contains(hostAndPort) + || data.getWebServiceUrl().contains(hostAndPort) + || data.getWebServiceUrlTls().contains(hostAndPort)) { + returnFuture.complete(data.getProtocol(KafkaProtocolHandler.PROTOCOL_NAME)); + return; + } + + if (wait == 0) { + log.error("Error to search {} in all child of zk://loadbalance", brokerAddress); + returnFuture.complete(Optional.empty()); + } + }); + }); }); + return returnFuture; } private CompletableFuture findBroker(PulsarService pulsarService, TopicName topic) { @@ -1081,62 +1118,71 @@ private CompletableFuture findBroker(PulsarService pulsarServ log.debug("Handle Lookup for {}", topic); } - return pulsarService.getNamespaceService() - .getBrokerServiceUrlAsync(topic, true) - .thenCompose(data -> getProtocolDataToAdvertise(data, topic)) - .thenApply(stringOptional -> { - if (!stringOptional.isPresent()) { - log.error("Not get advertise data for Kafka topic:{} ", topic); - return null; - } + try { + PulsarClientImpl pulsarClient = (PulsarClientImpl) pulsarService.getClient(); + return pulsarClient.getLookup() + .getBroker(topic) + .thenCompose(pair -> getProtocolDataToAdvertise(pair, topic)) + .thenApply(stringOptional -> { + if (!stringOptional.isPresent()) { + log.error("Not get advertise data for Kafka topic:{} ", topic); + return null; + } - try { - String listeners = stringOptional.get(); - String brokerUrl = getBrokerUrl(listeners, tlsEnabled); - URI uri = new URI(brokerUrl); + try { + String listeners = stringOptional.get(); + String brokerUrl = getBrokerUrl(listeners, tlsEnabled); - // get local listeners. - String listeners1 = kafkaConfig.getListeners(); + // get local listeners. + String localListeners = kafkaConfig.getListeners(); - if (log.isDebugEnabled()) { - log.debug("Found broker: {} for topicName: {}, local address: {}, found brokerUri: {}: {}:{}", - listeners, topic, listeners1, uri, uri.getHost(), uri.getPort()); - } + if (log.isDebugEnabled()) { + log.debug("Found broker listeners: {} for topicName: {}, " + + "localListeners: {}, found Listeners: {}", + listeners, topic, localListeners, listeners); + } - if (!topicManager.topicExists(topic.toString()) && listeners1.contains(uri.getHost())) { - pulsarService.getBrokerService().getTopic(topic.toString(), true) - .whenComplete((topicOpt, exception) -> { - if (exception != null) { - log.error("[{}] findBroker: Failed to getOrCreateTopic {}. exception:", - ctx.channel(), topic.toString(), exception); - } else { - if (topicOpt.isPresent()) { - if (log.isDebugEnabled()) { - log.debug("Add topic: {} into TopicManager while findBroker.", - topic.toString()); - } - topicManager.addTopic(topic.toString(), (PersistentTopic) topicOpt.get()); + if (!topicManager.topicExists(topic.toString()) && localListeners.contains(brokerUrl)) { + pulsarService.getBrokerService().getTopic(topic.toString(), true) + .whenComplete((topicOpt, exception) -> { + if (exception != null) { + log.error("[{}] findBroker: Failed to getOrCreateTopic {}. exception:", + ctx.channel(), topic.toString(), exception); } else { - log.error("[{}] findBroker: getOrCreateTopic get empty topic for name {}", - ctx.channel(), topic.toString()); + if (topicOpt.isPresent()) { + if (log.isDebugEnabled()) { + log.debug("Add topic: {} into TopicManager while findBroker.", + topic.toString()); + } + topicManager.addTopic(topic.toString(), (PersistentTopic) topicOpt.get()); + } else { + log.error("[{}] findBroker: getOrCreateTopic get empty topic for name {}", + ctx.channel(), topic.toString()); + } } - } - }); - } + }); + } - Node node = newNode(new InetSocketAddress( - uri.getHost(), - uri.getPort())); + URI uri = new URI(brokerUrl); + Node node = newNode(new InetSocketAddress( + uri.getHost(), + uri.getPort())); - return newPartitionMetadata(topic, node); - } catch (Exception e) { - log.error("Caught error while find Broker for topic:{} ", topic, e); + return newPartitionMetadata(topic, node); + } catch (Exception e) { + log.error("Caught error while find Broker for topic:{} ", topic, e); + return null; + } + }).exceptionally(ex -> { + log.error("Exceptionally while find Broker for topic:{} ", topic, ex); return null; - } - }).exceptionally(ex -> { - log.error("Exceptionally while find Broker for topic:{} ", topic, ex); - return null; - }); + }); + } catch (Exception e) { + log.error("Exceptionally while get pulsar client from Pulsar Broker for topic:{} ", topic, e); + CompletableFuture completableFuture = new CompletableFuture(); + completableFuture.complete(null); + return completableFuture; + } } static Node newNode(InetSocketAddress address) { diff --git a/src/main/java/io/streamnative/kop/KafkaService.java b/src/main/java/io/streamnative/kop/KafkaService.java index 05fd0ef798aa6..a615316301d5f 100644 --- a/src/main/java/io/streamnative/kop/KafkaService.java +++ b/src/main/java/io/streamnative/kop/KafkaService.java @@ -15,18 +15,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; -import io.streamnative.kop.coordinator.group.GroupConfig; import io.streamnative.kop.coordinator.group.GroupCoordinator; -import io.streamnative.kop.coordinator.group.OffsetConfig; -import io.streamnative.kop.utils.timer.SystemTimer; import java.net.InetSocketAddress; -import java.nio.ByteBuffer; import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; import lombok.Getter; @@ -34,9 +27,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.commons.lang3.builder.ReflectionToStringBuilder; -import org.apache.kafka.common.internals.Topic; -import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.utils.Time; import org.apache.pulsar.broker.BookKeeperClientFactory; import org.apache.pulsar.broker.ManagedLedgerClientFactory; import org.apache.pulsar.broker.PulsarServerException; @@ -47,17 +37,7 @@ import org.apache.pulsar.broker.stats.MetricsGenerator; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.web.WebService; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.Reader; -import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.configuration.VipStatus; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; -import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.RetentionPolicies; -import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.zookeeper.LocalZooKeeperConnectionService; import org.eclipse.jetty.servlet.ServletHolder; @@ -106,6 +86,14 @@ public void start() throws PulsarServerException { throw new IllegalArgumentException("Kafka Listeners should be provided through brokerConf.listeners"); } + if (kafkaConfig.getAdvertisedAddress() != null + && !kafkaConfig.getListeners().contains(kafkaConfig.getAdvertisedAddress())) { + String err = "Error config: advertisedAddress - " + kafkaConfig.getAdvertisedAddress() + + " and listeners - " + kafkaConfig.getListeners() + " not match."; + log.error(err); + throw new IllegalArgumentException(err); + } + setOrderedExecutor(OrderedExecutor.newBuilder().numThreads(8).name("pulsar-ordered") .build()); @@ -238,99 +226,15 @@ public Boolean get() { } } - // TODO: make group coordinator running in a distributed mode - // https://github.com/streamnative/kop/issues/32 - public void startGroupCoordinator() throws Exception { - GroupConfig groupConfig = new GroupConfig( - kafkaConfig.getGroupMinSessionTimeoutMs(), - kafkaConfig.getGroupMaxSessionTimeoutMs(), - kafkaConfig.getGroupInitialRebalanceDelayMs() - ); - - OffsetConfig offsetConfig = OffsetConfig.builder() - .offsetsTopicCompressionType(CompressionType.valueOf(kafkaConfig.getOffsetsTopicCompressionCodec())) - .maxMetadataSize(kafkaConfig.getOffsetMetadataMaxSize()) - .offsetsRetentionCheckIntervalMs(kafkaConfig.getOffsetsRetentionCheckIntervalMs()) - .offsetsRetentionMs(TimeUnit.MINUTES.toMillis(kafkaConfig.getOffsetsRetentionMinutes())) - .build(); - - createKafkaMetadataNamespaceIfNeeded(); - String offsetsTopic = createKafkaOffsetsTopic(); - - TopicName offsetsTopicName = TopicName.get(offsetsTopic); - String offsetsTopicPtn0 = offsetsTopicName.getPartition(0).toString(); - - Producer groupCoordinatorTopicProducer = getClient().newProducer(Schema.BYTEBUFFER) - .topic(offsetsTopicPtn0) - // TODO: make it configurable - .maxPendingMessages(100000) - .create(); - Reader groupCoordinatorTopicReader = getClient().newReader(Schema.BYTEBUFFER) - .topic(offsetsTopicPtn0) - .startMessageId(MessageId.earliest) - .create(); - this.groupCoordinator = GroupCoordinator.of( - groupCoordinatorTopicProducer, - groupCoordinatorTopicReader, - groupConfig, - offsetConfig, - SystemTimer.builder() - .executorName("group-coordinator-timer") - .build(), - Time.SYSTEM - ); - - this.groupCoordinator.startup(false); - } - - private void createKafkaMetadataNamespaceIfNeeded() throws PulsarServerException, PulsarAdminException { - String cluster = kafkaConfig.getClusterName(); - String kafkaMetadataTenant = kafkaConfig.getKafkaMetadataTenant(); - String kafkaMetadataNamespace = kafkaMetadataTenant + "/" + kafkaConfig.getKafkaMetadataNamespace(); - - try { - ClusterData clusterData = new ClusterData(getWebServiceAddress(), null /* serviceUrlTls */, - getBrokerServiceUrl(), null /* brokerServiceUrlTls */); - if (!getAdminClient().clusters().getClusters().contains(cluster)) { - getAdminClient().clusters().createCluster(cluster, clusterData); - } else { - getAdminClient().clusters().updateCluster(cluster, clusterData); - } - - if (!getAdminClient().tenants().getTenants().contains(kafkaMetadataTenant)) { - getAdminClient().tenants().createTenant(kafkaMetadataTenant, - new TenantInfo(Sets.newHashSet(kafkaConfig.getSuperUserRoles()), Sets.newHashSet(cluster))); - } - if (!getAdminClient().namespaces().getNamespaces(kafkaMetadataTenant).contains(kafkaMetadataNamespace)) { - Set clusters = Sets.newHashSet(kafkaConfig.getClusterName()); - getAdminClient().namespaces().createNamespace(kafkaMetadataNamespace, clusters); - getAdminClient().namespaces().setNamespaceReplicationClusters(kafkaMetadataNamespace, clusters); - getAdminClient().namespaces().setRetention(kafkaMetadataNamespace, - new RetentionPolicies(-1, -1)); - } - } catch (PulsarAdminException e) { - log.error("Failed to get retention policy for kafka metadata namespace {}", - kafkaMetadataNamespace, e); - throw e; + @Override + public void close() throws PulsarServerException { + if (groupCoordinator != null) { + this.groupCoordinator.shutdown(); } - } - - private String createKafkaOffsetsTopic() throws PulsarServerException, PulsarAdminException { - String offsetsTopic = kafkaConfig.getKafkaTenant() + "/" + kafkaConfig.getKafkaMetadataNamespace() - + "/" + Topic.GROUP_METADATA_TOPIC_NAME; - - PartitionedTopicMetadata offsetsTopicMetadata = - getAdminClient().topics().getPartitionedTopicMetadata(offsetsTopic); - if (offsetsTopicMetadata.partitions <= 0) { - log.info("Kafka group metadata topic {} doesn't exist. Creating it ...", - offsetsTopic); - getAdminClient().topics().createPartitionedTopic( - offsetsTopic, - KafkaServiceConfiguration.DefaultOffsetsTopicNumPartitions - ); - log.info("Successfully created group metadata topic {}.", offsetsTopic); + if (kafkaTopicManager != null) { + this.kafkaTopicManager.close(); } - - return offsetsTopic; + super.close(); } + } diff --git a/src/main/java/io/streamnative/kop/KafkaServiceConfiguration.java b/src/main/java/io/streamnative/kop/KafkaServiceConfiguration.java index 2f87f00fc908c..56e8d376dafd0 100644 --- a/src/main/java/io/streamnative/kop/KafkaServiceConfiguration.java +++ b/src/main/java/io/streamnative/kop/KafkaServiceConfiguration.java @@ -113,6 +113,12 @@ public class KafkaServiceConfiguration extends ServiceConfiguration { ) private String offsetsTopicCompressionCodec = CompressionType.NONE.name(); + @FieldContext( + category = CATEGORY_KOP, + doc = "Number of partitions for the offsets topic" + ) + private int offsetsTopicNumPartitions = DefaultOffsetsTopicNumPartitions; + @FieldContext( category = CATEGORY_KOP, doc = "The maximum size for a metadata entry associated with an offset commit" diff --git a/src/main/java/io/streamnative/kop/KafkaTopicManager.java b/src/main/java/io/streamnative/kop/KafkaTopicManager.java index 8736a984da355..54c9f0ade5c82 100644 --- a/src/main/java/io/streamnative/kop/KafkaTopicManager.java +++ b/src/main/java/io/streamnative/kop/KafkaTopicManager.java @@ -77,5 +77,17 @@ public PersistentTopic getTopic(String topicName) { return topics.get(topicName); } + public void close() { + consumerTopics.values() + .forEach(manager -> manager.join().getConsumers().values() + .forEach(pair -> { + try { + pair.join().getLeft().close(); + } catch (Exception e) { + log.error("Failed to close cursor for topic {}. exception:", + pair.join().getLeft().getName(), e); + } + })); + } } diff --git a/src/main/java/io/streamnative/kop/MessageFetchContext.java b/src/main/java/io/streamnative/kop/MessageFetchContext.java index 82da6c3c99dd8..bd4eaeb7ab7bc 100644 --- a/src/main/java/io/streamnative/kop/MessageFetchContext.java +++ b/src/main/java/io/streamnative/kop/MessageFetchContext.java @@ -235,7 +235,7 @@ private void readMessagesInternal(KafkaHeaderAndRequest fetch, try { Thread.sleep(waitTime); } catch (Exception e) { - log.error("Request {}: error while sleep.", + log.info("Request {}: error while sleep, this is OK.", fetch.getHeader(), e); } diff --git a/src/main/java/io/streamnative/kop/coordinator/group/GroupCoordinator.java b/src/main/java/io/streamnative/kop/coordinator/group/GroupCoordinator.java index 49d8bda35aae1..b9be86322bd2e 100644 --- a/src/main/java/io/streamnative/kop/coordinator/group/GroupCoordinator.java +++ b/src/main/java/io/streamnative/kop/coordinator/group/GroupCoordinator.java @@ -25,7 +25,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import io.streamnative.kop.KafkaServiceConfiguration; import io.streamnative.kop.coordinator.group.GroupMetadata.GroupOverview; import io.streamnative.kop.coordinator.group.GroupMetadata.GroupSummary; import io.streamnative.kop.offset.OffsetAndMetadata; @@ -44,6 +43,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; @@ -62,7 +62,9 @@ import org.apache.kafka.common.requests.TransactionResult; import org.apache.kafka.common.utils.Time; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Reader; +import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.util.FutureUtil; @@ -73,8 +75,8 @@ public class GroupCoordinator { public static GroupCoordinator of( - Producer producer, - Reader reader, + ProducerBuilder producer, + ReaderBuilder reader, GroupConfig groupConfig, OffsetConfig offsetConfig, Timer timer, @@ -85,7 +87,6 @@ public static GroupCoordinator of( .build(); GroupMetadataManager metadataManager = new GroupMetadataManager( - KafkaServiceConfiguration.DefaultOffsetsTopicNumPartitions, offsetConfig, producer, reader, @@ -113,7 +114,6 @@ public static GroupCoordinator of( ); } - static final String NoState = ""; static final String NoProtocolType = ""; static final String NoProtocol = ""; @@ -176,6 +176,25 @@ public void shutdown() { log.info("Shutdown group coordinator completely."); } + public int partitionFor(String coordinatorKey) { + return groupManager.partitionFor(coordinatorKey); + } + + public String getTopicPartitionName(int partition) { + return groupManager.getTopicPartitionName(partition); + } + + public ConcurrentMap>> getOffsetsProducers() { + return groupManager.getOffsetsProducers(); + } + + public ConcurrentMap>> getOffsetsReaders() { + return groupManager.getOffsetsReaders(); + } + + public GroupMetadataManager getGroupManager() { + return groupManager; + } public CompletableFuture handleJoinGroup( String groupId, @@ -898,14 +917,8 @@ private Optional validateGroupStatus(String groupId, return Optional.of(Errors.COORDINATOR_NOT_AVAILABLE); } else if (groupManager.isGroupLoading(groupId)) { return Optional.of(Errors.COORDINATOR_LOAD_IN_PROGRESS); - // TODO: make group coordinator running in distributed mode. - // https://github.com/streamnative/kop/issues/32 - // } else if (!groupManager.isGroupLocal(groupId) - // && api != ApiKeys.JOIN_GROUP // first time join, group may not persisted. - // && api != ApiKeys.SYNC_GROUP - // && api != ApiKeys.OFFSET_FETCH) { - // return Optional.of(Errors.NOT_COORDINATOR); - // } + } else if (!groupManager.isGroupLocal(groupId)) { + return Optional.of(Errors.NOT_COORDINATOR); } else { return Optional.empty(); } diff --git a/src/main/java/io/streamnative/kop/coordinator/group/GroupMetadataManager.java b/src/main/java/io/streamnative/kop/coordinator/group/GroupMetadataManager.java index b672a8e8cffa2..e3d4b3ba565c5 100644 --- a/src/main/java/io/streamnative/kop/coordinator/group/GroupMetadataManager.java +++ b/src/main/java/io/streamnative/kop/coordinator/group/GroupMetadataManager.java @@ -23,8 +23,8 @@ import static io.streamnative.kop.coordinator.group.GroupMetadataConstants.readMessageKey; import static io.streamnative.kop.coordinator.group.GroupMetadataConstants.readOffsetMessageValue; import static io.streamnative.kop.utils.CoreUtils.inLock; -import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME; +import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -60,7 +60,6 @@ import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.common.hash.Murmur3; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.commons.lang3.tuple.Triple; import org.apache.kafka.common.TopicPartition; @@ -81,8 +80,11 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Reader; +import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.common.util.FutureUtil; /** * Manager to manage a coordination group. @@ -161,7 +163,7 @@ public String toString() { private final byte magicValue = RecordBatch.CURRENT_MAGIC_VALUE; private final CompressionType compressionType; - private final OffsetConfig config; + private final OffsetConfig offsetConfig; private final ConcurrentMap groupMetadataCache; /* lock protecting access to loading and owned partition sets */ private final ReentrantLock partitionLock = new ReentrantLock(); @@ -176,6 +178,11 @@ public String toString() { private final AtomicBoolean shuttingDown = new AtomicBoolean(false); private final int groupMetadataTopicPartitionCount; + // Map of + private final ConcurrentMap>> offsetsProducers = + new ConcurrentHashMap<>(); + private final ConcurrentMap>> offsetsReaders = + new ConcurrentHashMap<>(); /* single-thread scheduler to handle offset/group metadata cache loading and unloading */ private final ScheduledExecutorService scheduler; @@ -187,44 +194,43 @@ public String toString() { */ private final Map> openGroupsForProducer = new HashMap<>(); - private final Producer metadataTopicProducer; - private final Reader metadataTopicReader; + private final ProducerBuilder metadataTopicProducerBuilder; + private final ReaderBuilder metadataTopicReaderBuilder; private final Time time; private final Function partitioner; - public GroupMetadataManager(int groupMetadataTopicPartitionCount, - OffsetConfig config, - Producer metadataTopicProducer, - Reader metadataTopicConsumer, + public GroupMetadataManager(OffsetConfig offsetConfig, + ProducerBuilder metadataTopicProducerBuilder, + ReaderBuilder metadataTopicReaderBuilder, ScheduledExecutorService scheduler, Time time) { this( - groupMetadataTopicPartitionCount, - config, - metadataTopicProducer, - metadataTopicConsumer, + offsetConfig, + metadataTopicProducerBuilder, + metadataTopicReaderBuilder, scheduler, time, + // Be same with kafka: abs(groupId.hashCode) % groupMetadataTopicPartitionCount + // return a partitionId groupId -> MathUtils.signSafeMod( - Murmur3.hash32(groupId.getBytes(UTF_8)), - groupMetadataTopicPartitionCount + groupId.hashCode(), + offsetConfig.offsetsTopicNumPartitions() ) ); } - GroupMetadataManager(int groupMetadataTopicPartitionCount, - OffsetConfig config, - Producer metadataTopicProducer, - Reader metadataTopicConsumer, + GroupMetadataManager(OffsetConfig offsetConfig, + ProducerBuilder metadataTopicProducerBuilder, + ReaderBuilder metadataTopicConsumerBuilder, ScheduledExecutorService scheduler, Time time, Function partitioner) { - this.config = config; - this.compressionType = config.offsetsTopicCompressionType(); + this.offsetConfig = offsetConfig; + this.compressionType = offsetConfig.offsetsTopicCompressionType(); this.groupMetadataCache = new ConcurrentHashMap<>(); - this.groupMetadataTopicPartitionCount = groupMetadataTopicPartitionCount; - this.metadataTopicProducer = metadataTopicProducer; - this.metadataTopicReader = metadataTopicConsumer; + this.groupMetadataTopicPartitionCount = offsetConfig.offsetsTopicNumPartitions(); + this.metadataTopicProducerBuilder = metadataTopicProducerBuilder; + this.metadataTopicReaderBuilder = metadataTopicConsumerBuilder; this.scheduler = scheduler; this.time = time; this.partitioner = partitioner; @@ -234,8 +240,8 @@ public void startup(boolean enableMetadataExpiration) { if (enableMetadataExpiration) { scheduler.scheduleAtFixedRate( this::cleanupGroupMetadata, - config.offsetsRetentionCheckIntervalMs(), - config.offsetsRetentionCheckIntervalMs(), + offsetConfig.offsetsRetentionCheckIntervalMs(), + offsetConfig.offsetsRetentionCheckIntervalMs(), TimeUnit.MILLISECONDS ); } @@ -244,6 +250,43 @@ public void startup(boolean enableMetadataExpiration) { public void shutdown() { shuttingDown.set(true); scheduler.shutdown(); + List> producerCloses = offsetsProducers.entrySet().stream() + .map(v -> v.getValue() + .thenComposeAsync(producer -> producer.closeAsync(), scheduler)) + .collect(Collectors.toList()); + offsetsProducers.clear(); + List> readerCloses = offsetsReaders.entrySet().stream() + .map(v -> v.getValue() + .thenComposeAsync(reader -> reader.closeAsync(), scheduler)) + .collect(Collectors.toList()); + offsetsReaders.clear(); + + FutureUtil.waitForAll(producerCloses).whenCompleteAsync((ignore, t) -> { + if (t != null) { + log.error("Error when close all the {} offsetsProducers in GroupMetadataManager", + producerCloses.size(), t); + } + if (log.isDebugEnabled()) { + log.debug("Closed all the {} offsetsProducers in GroupMetadataManager", producerCloses.size()); + } + }, scheduler); + + FutureUtil.waitForAll(readerCloses).whenCompleteAsync((ignore, t) -> { + if (t != null) { + log.error("Error when close all the {} offsetsReaders in GroupMetadataManager", + readerCloses.size(), t); + } + if (log.isDebugEnabled()) { + log.debug("Closed all the {} offsetsReaders in GroupMetadataManager.", readerCloses.size()); + } + }, scheduler); + } + + public ConcurrentMap>> getOffsetsProducers() { + return offsetsProducers; + } + public ConcurrentMap>> getOffsetsReaders() { + return offsetsReaders; } public Iterable currentGroups() { @@ -271,6 +314,18 @@ public int partitionFor(String groupId) { return partitioner.apply(groupId); } + public String getTopicPartitionName() { + return offsetConfig.offsetsTopicName(); + } + + public String getTopicPartitionName(int partitionId) { + return offsetConfig.offsetsTopicName() + PARTITIONED_TOPIC_SUFFIX + partitionId; + } + + public int getGroupMetadataTopicPartitionCount() { + return groupMetadataTopicPartitionCount; + } + public boolean isGroupLocal(String groupId) { return isPartitionOwned(partitionFor(groupId)); } @@ -292,8 +347,8 @@ public boolean groupNotExists(String groupId) { partitionLock, () -> isGroupLocal(groupId) && getGroup(groupId) - .map(group -> group.inLock(() -> group.is(GroupState.Dead))) - .orElse(true) + .map(group -> group.inLock(() -> group.is(GroupState.Dead))) + .orElse(true) ); } @@ -342,13 +397,13 @@ public CompletableFuture storeGroup(GroupMetadata group, recordsBuilder.append(timestamp, key, value); MemoryRecords records = recordsBuilder.build(); - return metadataTopicProducer - .newMessage() - .keyBytes(key) - .value(records.buffer()) - .eventTime(timestamp) - .sendAsync() - .thenApply(msgId -> { + return getOffsetsTopicProducer(group.groupId()) + .thenComposeAsync(f -> f.newMessage() + .keyBytes(key) + .value(records.buffer()) + .eventTime(timestamp).sendAsync() + , scheduler) + .thenApplyAsync(msgId -> { if (!isGroupLocal(group.groupId())) { if (log.isDebugEnabled()) { log.warn("add partition ownership for group {}", @@ -357,20 +412,21 @@ public CompletableFuture storeGroup(GroupMetadata group, addPartitionOwnership(partitionFor(group.groupId())); } return Errors.NONE; - }) + }, scheduler) .exceptionally(cause -> Errors.COORDINATOR_NOT_AVAILABLE); } // visible for mock - CompletableFuture storeOffsetMessage(byte[] key, + CompletableFuture storeOffsetMessage(String groupId, + byte[] key, ByteBuffer buffer, long timestamp) { - return metadataTopicProducer - .newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(timestamp) - .sendAsync(); + return getOffsetsTopicProducer(groupId) + .thenComposeAsync(f -> f.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(timestamp).sendAsync() + , scheduler); } public CompletableFuture> storeOffsets( @@ -406,7 +462,7 @@ public CompletableFuture> storeOffsets( group.inLock(() -> { if (!group.hasReceivedConsistentOffsetCommits()) { log.warn("group: {} with leader: {} has received offset commits from consumers as well " - + "as transactional producers. Mixing both types of offset commits will generally" + + "as transactional offsetsProducers. Mixing both types of offset commits will generally" + " result in surprises and should be avoided.", group.groupId(), group.leaderOrNull()); } @@ -470,8 +526,8 @@ public CompletableFuture> storeOffsets( // dummy offset commit key byte[] key = offsetCommitKey(group.groupId(), new TopicPartition("", -1)); - return storeOffsetMessage(key, entries.buffer(), timestamp) - .thenApply(messageId -> { + return storeOffsetMessage(group.groupId(), key, entries.buffer(), timestamp) + .thenApplyAsync(messageId -> { if (!group.is(GroupState.Dead)) { MessageIdImpl lastMessageId = (MessageIdImpl) messageId; long baseOffset = MessageIdUtils.getOffset( @@ -492,7 +548,7 @@ public CompletableFuture> storeOffsets( }); } return Errors.NONE; - }) + }, scheduler) .exceptionally(cause -> { if (!group.is(GroupState.Dead)) { if (!group.hasPendingOffsetCommitsFromProducer(producerId)) { @@ -515,7 +571,7 @@ public CompletableFuture> storeOffsets( return Errors.UNKNOWN_SERVER_ERROR; }) - .thenApply(errors -> offsetMetadata.entrySet() + .thenApplyAsync(errors -> offsetMetadata.entrySet() .stream() .collect(Collectors.toMap( e -> e.getKey(), @@ -526,8 +582,7 @@ public CompletableFuture> storeOffsets( return Errors.OFFSET_METADATA_TOO_LARGE; } } - )) - ); + )), scheduler); } /** @@ -650,41 +705,42 @@ private void removeGroupFromAllProducers(String groupId) { * Check if the offset metadata length is valid */ private boolean validateOffsetMetadataLength(String metadata) { - return metadata == null || metadata.length() <= config.maxMetadataSize(); + return metadata == null || metadata.length() <= offsetConfig.maxMetadataSize(); } public CompletableFuture scheduleLoadGroupAndOffsets(int offsetsPartition, Consumer onGroupLoaded) { - TopicPartition topicPartition = new TopicPartition( - GROUP_METADATA_TOPIC_NAME, offsetsPartition - ); + String topicPartition = offsetConfig.offsetsTopicName() + PARTITIONED_TOPIC_SUFFIX + offsetsPartition; if (addLoadingPartition(offsetsPartition)) { log.info("Scheduling loading of offsets and group metadata from {}", topicPartition); long startMs = time.milliseconds(); - return metadataTopicProducer.newMessage() - .value(ByteBuffer.allocate(0)) - .eventTime(time.milliseconds()) - .sendAsync() - .thenCompose(lastMessageId -> { + return getOffsetsTopicProducer(offsetsPartition) + .thenComposeAsync(f -> f.newMessage() + .value(ByteBuffer.allocate(0)) + .eventTime(time.milliseconds()).sendAsync() + , scheduler) + .thenComposeAsync(lastMessageId -> { if (log.isTraceEnabled()) { log.trace("Successfully write a placeholder record into {} @ {}", topicPartition, lastMessageId); } - return doLoadGroupsAndOffsets(metadataTopicReader, lastMessageId, onGroupLoaded); - }) - .whenComplete((ignored, cause) -> { - if (null == cause) { - log.info("Finished loading offsets and group metadata from {} in {} milliseconds", - topicPartition, time.milliseconds() - startMs); - } else { + return doLoadGroupsAndOffsets(getOffsetsTopicReader(offsetsPartition), + lastMessageId, onGroupLoaded); + }, scheduler) + .whenCompleteAsync((ignored, cause) -> { + if (null != cause) { log.error("Error loading offsets from {}", topicPartition, cause); + removeLoadingPartition(offsetsPartition); + return; } + log.info("Finished loading offsets and group metadata from {} in {} milliseconds", + topicPartition, time.milliseconds() - startMs); inLock(partitionLock, () -> { - ownedPartitions.add(topicPartition.partition()); - loadingPartitions.remove(topicPartition.partition()); + ownedPartitions.add(offsetsPartition); + loadingPartitions.remove(offsetsPartition); return null; }); - }); + }, scheduler); } else { log.info("Already loading offsets and group metadata from {}", topicPartition); return CompletableFuture.completedFuture(null); @@ -692,7 +748,7 @@ public CompletableFuture scheduleLoadGroupAndOffsets(int offsetsPartition, } private CompletableFuture doLoadGroupsAndOffsets( - Reader metadataConsumer, + CompletableFuture> metadataConsumer, MessageId endMessageId, Consumer onGroupLoaded ) { @@ -715,7 +771,7 @@ private CompletableFuture doLoadGroupsAndOffsets( return resultFuture; } - private void loadNextMetadataMessage(Reader metadataConsumer, + private void loadNextMetadataMessage(CompletableFuture> metadataConsumer, MessageId endMessageId, CompletableFuture resultFuture, Consumer onGroupLoaded, @@ -736,13 +792,13 @@ private void loadNextMetadataMessage(Reader metadataConsumer, removedGroups ); } catch (Throwable cause) { - log.error("Unknown exception caught when loading group and offsets from topic {}", - metadataConsumer.getTopic(), cause); + log.error("Unknown exception caught when loading group and offsets from topic", + cause); resultFuture.completeExceptionally(cause); } } - private void unsafeLoadNextMetadataMessage(Reader metadataConsumer, + private void unsafeLoadNextMetadataMessage(CompletableFuture> metadataConsumer, MessageId endMessageId, CompletableFuture resultFuture, Consumer onGroupLoaded, @@ -759,13 +815,13 @@ private void unsafeLoadNextMetadataMessage(Reader metadataConsumer, if (log.isTraceEnabled()) { log.trace("Reading the next metadata message from topic {}", - metadataConsumer.getTopic()); + metadataConsumer.join().getTopic()); } BiConsumer, Throwable> readNextComplete = (message, cause) -> { if (log.isTraceEnabled()) { log.trace("Metadata consumer received a metadata message from {} @ {}", - metadataConsumer.getTopic(), message.getMessageId()); + metadataConsumer.join().getTopic(), message.getMessageId()); } if (null != cause) { @@ -834,7 +890,7 @@ private void unsafeLoadNextMetadataMessage(Reader metadataConsumer, if (log.isTraceEnabled()) { log.trace("Applying metadata record {} received from {}", - bk, metadataConsumer.getTopic()); + bk, metadataConsumer.join().getTopic()); } if (bk instanceof OffsetKey) { @@ -905,15 +961,15 @@ private void unsafeLoadNextMetadataMessage(Reader metadataConsumer, ); }; - metadataConsumer.readNextAsync().whenComplete((message, cause) -> { + metadataConsumer.thenComposeAsync(r -> r.readNextAsync()).whenCompleteAsync((message, cause) -> { try { readNextComplete.accept(message, cause); } catch (Throwable completeCause) { log.error("Unknown exception caught when processing the received metadata message from topic {}", - metadataConsumer.getTopic(), completeCause); + metadataConsumer.join().getTopic(), completeCause); resultFuture.completeExceptionally(completeCause); } - }); + }, scheduler); } private void processLoadedAndRemovedGroups(CompletableFuture resultFuture, @@ -933,8 +989,8 @@ private void processLoadedAndRemovedGroups(CompletableFuture resultFuture, .collect(Collectors.groupingBy( e -> e.getKey().group(), Collectors.toMap( - f -> f.getKey().topicPartition(), - f -> f.getValue() + f -> f.getKey().topicPartition(), + f -> f.getValue() )) ); Map>> partitionedLoadedOffsets = @@ -975,8 +1031,8 @@ private void processLoadedAndRemovedGroups(CompletableFuture resultFuture, Map>>> partitionedPendingOffsetsByGroup = CoreUtils.partition( - pendingOffsetsByGroup, - group -> loadedGroups.containsKey(group) + pendingOffsetsByGroup, + group -> loadedGroups.containsKey(group) ); Map>> pendingGroupOffsets = partitionedPendingOffsetsByGroup.get(true); @@ -1042,7 +1098,7 @@ private void loadGroup(GroupMetadata group, OffsetAndMetadata offsetAndMetadata = commitRecordMetadataAndOffset.offsetAndMetadata(); OffsetAndMetadata updatedOffsetAndMetadata; if (offsetAndMetadata.expireTimestamp() == OffsetCommitRequest.DEFAULT_TIMESTAMP) { - long expireTimestamp = offsetAndMetadata.commitTimestamp() + config.offsetsRetentionMs(); + long expireTimestamp = offsetAndMetadata.commitTimestamp() + offsetConfig.offsetsRetentionMs(); updatedOffsetAndMetadata = OffsetAndMetadata.apply( offsetAndMetadata.offset(), offsetAndMetadata.metadata(), @@ -1070,7 +1126,7 @@ private void loadGroup(GroupMetadata group, GroupMetadata currentGroup = addGroup(group); if (group != currentGroup) { log.debug("Attempt to load group {} from log with generation {} failed " - + "because there is already a cached group with generation {}", + + "because there is already a cached group with generation {}", group.groupId(), group.generationId(), currentGroup.generationId()); } } @@ -1106,6 +1162,26 @@ public void removeGroupsForPartition(int offsetsPartition, } } + // remove related producers and readers + CompletableFuture> producer = offsetsProducers.remove(offsetsPartition); + CompletableFuture> reader = offsetsReaders.remove(offsetsPartition); + if (producer != null) { + producer.thenApplyAsync(p -> p.closeAsync()).whenCompleteAsync((ignore, t) -> { + if (t != null) { + log.error("Failed to close producer when remove partition {}.", + producer.join().getTopic()); + } + }, scheduler); + } + if (reader != null) { + reader.thenApplyAsync(p -> p.closeAsync()).whenCompleteAsync((ignore, t) -> { + if (t != null) { + log.error("Failed to close reader when remove partition {}.", + reader.join().getTopic()); + } + }, scheduler); + } + return null; }); @@ -1118,10 +1194,10 @@ CompletableFuture cleanupGroupMetadata() { final long startMs = time.milliseconds(); return cleanGroupMetadata(groupMetadataCache.values().stream(), group -> group.removeExpiredOffsets(time.milliseconds()) - ).thenAccept(offsetsRemoved -> - log.info("Removed {} expired offsets in {} milliseconds.", - offsetsRemoved, time.milliseconds() - startMs) - ); + ).thenAcceptAsync(offsetsRemoved -> + log.info("Removed {} expired offsets in {} milliseconds.", + offsetsRemoved, time.milliseconds() - startMs) + , scheduler); } CompletableFuture cleanGroupMetadata(Stream groups, @@ -1177,16 +1253,17 @@ CompletableFuture cleanGroupMetadata(Stream groups, byte[] groupKey = groupMetadataKey( group.groupId() ); - return metadataTopicProducer.newMessage() - .keyBytes(groupKey) - .value(records.buffer()) - .eventTime(timestamp) - .sendAsync() - .thenApply(ignored -> removedOffsets.size()) + return getOffsetsTopicProducer(group.groupId()) + .thenComposeAsync(f -> f.newMessage() + .keyBytes(groupKey) + .value(records.buffer()) + .eventTime(timestamp).sendAsync(), scheduler) + .thenApplyAsync(ignored -> removedOffsets.size(), scheduler) .exceptionally(cause -> { - log.error("Failed to append {} tombstones to {} for expired/deleted " - + "offsets and/or metadata for group {}", - tombstones.size(), metadataTopicProducer.getTopic(), + log.error("Failed to append {} tombstones to topic {} for expired/deleted " + + "offsets and/or metadata for group {}", + tombstones.size(), + offsetConfig.offsetsTopicName() + '-' + partitioner.apply(group.groupId()), group.groupId(), cause); // ignore and continue return 0; @@ -1196,7 +1273,7 @@ CompletableFuture cleanGroupMetadata(Stream groups, } }).collect(Collectors.toList()); return FutureUtils.collect(cleanFutures) - .thenApply(removedList -> removedList.stream().mapToInt(Integer::intValue).sum()); + .thenApplyAsync(removedList -> removedList.stream().mapToInt(Integer::intValue).sum(), scheduler); } /** @@ -1265,4 +1342,43 @@ boolean removeLoadingPartition(int partition) { return inLock(partitionLock, () -> loadingPartitions.remove(partition)); } + CompletableFuture> getOffsetsTopicProducer(String groupId) { + return offsetsProducers.computeIfAbsent(partitionFor(groupId), + partitionId -> { + if (log.isDebugEnabled()) { + log.debug("Created Partitioned producer: {} for consumer group: {}", + offsetConfig.offsetsTopicName() + PARTITIONED_TOPIC_SUFFIX + partitionId, + groupId); + } + return metadataTopicProducerBuilder.clone() + .topic(offsetConfig.offsetsTopicName() + PARTITIONED_TOPIC_SUFFIX + partitionId) + .createAsync(); + }); + } + + CompletableFuture> getOffsetsTopicProducer(int partitionId) { + return offsetsProducers.computeIfAbsent(partitionId, + id -> { + if (log.isDebugEnabled()) { + log.debug("Will create Partitioned producer: {}", + offsetConfig.offsetsTopicName() + PARTITIONED_TOPIC_SUFFIX + id); + } + return metadataTopicProducerBuilder.clone() + .topic(offsetConfig.offsetsTopicName() + PARTITIONED_TOPIC_SUFFIX + id) + .createAsync(); + }); + } + + CompletableFuture> getOffsetsTopicReader(int partitionId) { + return offsetsReaders.computeIfAbsent(partitionId, + id -> { + if (log.isDebugEnabled()) { + log.debug("Will create Partitioned reader: {}", + offsetConfig.offsetsTopicName() + PARTITIONED_TOPIC_SUFFIX + id); + } + return metadataTopicReaderBuilder.clone() + .topic(offsetConfig.offsetsTopicName() + PARTITIONED_TOPIC_SUFFIX + partitionId) + .createAsync(); + }); + } } diff --git a/src/main/java/io/streamnative/kop/coordinator/group/OffsetConfig.java b/src/main/java/io/streamnative/kop/coordinator/group/OffsetConfig.java index 714b3bf4a66a0..9b08e7e1f121c 100644 --- a/src/main/java/io/streamnative/kop/coordinator/group/OffsetConfig.java +++ b/src/main/java/io/streamnative/kop/coordinator/group/OffsetConfig.java @@ -13,6 +13,7 @@ */ package io.streamnative.kop.coordinator.group; +import io.streamnative.kop.KafkaServiceConfiguration; import lombok.Builder; import lombok.Builder.Default; import lombok.Data; @@ -30,7 +31,11 @@ public class OffsetConfig { public static final int DefaultMaxMetadataSize = 4096; public static final long DefaultOffsetsRetentionMs = 24 * 60 * 60 * 1000L; public static final long DefaultOffsetsRetentionCheckIntervalMs = 600000L; + public static final String DefaultOffsetsTopicName = "public/default/__consumer_offsets"; + public static final int DefaultOffsetsNumPartitions = KafkaServiceConfiguration.DefaultOffsetsTopicNumPartitions; + @Default + private String offsetsTopicName = DefaultOffsetsTopicName; @Default private int maxMetadataSize = DefaultMaxMetadataSize; @Default @@ -39,5 +44,6 @@ public class OffsetConfig { private long offsetsRetentionMs = DefaultOffsetsRetentionMs; @Default private long offsetsRetentionCheckIntervalMs = DefaultOffsetsRetentionCheckIntervalMs; - + @Default + private int offsetsTopicNumPartitions = DefaultOffsetsNumPartitions; } diff --git a/src/main/java/io/streamnative/kop/utils/TopicNameUtils.java b/src/main/java/io/streamnative/kop/utils/TopicNameUtils.java index 8f9849cbb32ce..42361f1f32898 100644 --- a/src/main/java/io/streamnative/kop/utils/TopicNameUtils.java +++ b/src/main/java/io/streamnative/kop/utils/TopicNameUtils.java @@ -60,6 +60,7 @@ public static String getPartitionedTopicNameWithoutPartitions(TopicName topicNam } } + // get local name without partition part public static String getKafkaTopicNameFromPulsarTopicname(TopicName topicName) { // remove partition part String localName = topicName.getPartitionedTopicName(); diff --git a/src/main/java/org/apache/pulsar/client/ReaderBuilderImpl2.java b/src/main/java/org/apache/pulsar/client/ReaderBuilderImpl2.java new file mode 100644 index 0000000000000..9b0d9399fe305 --- /dev/null +++ b/src/main/java/org/apache/pulsar/client/ReaderBuilderImpl2.java @@ -0,0 +1,170 @@ +/** + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.pulsar.client; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.AccessLevel; +import lombok.Getter; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; +import org.apache.pulsar.client.api.CryptoKeyReader; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Reader; +import org.apache.pulsar.client.api.ReaderBuilder; +import org.apache.pulsar.client.api.ReaderListener; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils; +import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; +import org.apache.pulsar.common.util.FutureUtil; + +/** + * This is a copy of ReaderBuilderImpl from Pulsar to fix the issue of clone. + * TODO: remove this class after merge pulsar PR: + * https://github.com/apache/pulsar/pull/5923 + */ +@Getter(AccessLevel.PUBLIC) +public class ReaderBuilderImpl2 implements ReaderBuilder { + + private final PulsarClientImpl client; + + private ReaderConfigurationData conf; + + private final Schema schema; + + public ReaderBuilderImpl2(PulsarClientImpl client, Schema schema) { + this(client, new ReaderConfigurationData(), schema); + } + + private ReaderBuilderImpl2(PulsarClientImpl client, ReaderConfigurationData conf, Schema schema) { + this.client = client; + this.conf = conf; + this.schema = schema; + } + + @Override + @SuppressWarnings("unchecked") + public ReaderBuilder clone() { + try { + ReaderBuilderImpl2 clone = (ReaderBuilderImpl2) super.clone(); + clone.conf = conf.clone(); + + return clone; + } catch (CloneNotSupportedException e) { + throw new RuntimeException("Failed to clone ReaderBuilderImpl"); + } +// return new ReaderBuilderImpl2<>(client, conf.clone(), schema); + } + + @Override + public Reader create() throws PulsarClientException { + try { + return createAsync().get(); + } catch (Exception e) { + throw PulsarClientException.unwrap(e); + } + } + + @Override + public CompletableFuture> createAsync() { + if (conf.getTopicName() == null) { + return FutureUtil + .failedFuture(new IllegalArgumentException("Topic name must be set on the reader builder")); + } + + if (conf.getStartMessageId() == null) { + return FutureUtil + .failedFuture(new IllegalArgumentException("Start message id must be set on the reader builder")); + } + + return client.createReaderAsync(conf, schema); + } + + @Override + public ReaderBuilder loadConf(Map config) { + MessageId startMessageId = conf.getStartMessageId(); + conf = ConfigurationDataUtils.loadData(config, conf, ReaderConfigurationData.class); + conf.setStartMessageId(startMessageId); + return this; + } + + @Override + public ReaderBuilder topic(String topicName) { + conf.setTopicName(StringUtils.trim(topicName)); + return this; + } + + @Override + public ReaderBuilder startMessageId(MessageId startMessageId) { + conf.setStartMessageId(startMessageId); + return this; + } + + @Override + public ReaderBuilder startMessageFromRollbackDuration(long rollbackDuration, TimeUnit timeunit) { + conf.setStartMessageFromRollbackDurationInSec(timeunit.toSeconds(rollbackDuration)); + return this; + } + + @Override + public ReaderBuilder startMessageIdInclusive() { + conf.setResetIncludeHead(true); + return this; + } + + @Override + public ReaderBuilder readerListener(ReaderListener readerListener) { + conf.setReaderListener(readerListener); + return this; + } + + @Override + public ReaderBuilder cryptoKeyReader(CryptoKeyReader cryptoKeyReader) { + conf.setCryptoKeyReader(cryptoKeyReader); + return this; + } + + @Override + public ReaderBuilder cryptoFailureAction(ConsumerCryptoFailureAction action) { + conf.setCryptoFailureAction(action); + return this; + } + + @Override + public ReaderBuilder receiverQueueSize(int receiverQueueSize) { + conf.setReceiverQueueSize(receiverQueueSize); + return this; + } + + @Override + public ReaderBuilder readerName(String readerName) { + conf.setReaderName(readerName); + return this; + } + + @Override + public ReaderBuilder subscriptionRolePrefix(String subscriptionRolePrefix) { + conf.setSubscriptionRolePrefix(subscriptionRolePrefix); + return this; + } + + @Override + public ReaderBuilder readCompacted(boolean readCompacted) { + conf.setReadCompacted(readCompacted); + return this; + } +} diff --git a/src/main/java/org/apache/pulsar/client/package-info.java b/src/main/java/org/apache/pulsar/client/package-info.java new file mode 100644 index 0000000000000..03ec6345b6197 --- /dev/null +++ b/src/main/java/org/apache/pulsar/client/package-info.java @@ -0,0 +1,19 @@ +/** + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * Timer related classes. + * + *

The classes under this package are ported from Kafka. + */ +package org.apache.pulsar.client; diff --git a/src/test/java/io/streamnative/kop/KafkaApisTest.java b/src/test/java/io/streamnative/kop/KafkaApisTest.java index 0f963ee39c13a..eb8a2eb466ad9 100644 --- a/src/test/java/io/streamnative/kop/KafkaApisTest.java +++ b/src/test/java/io/streamnative/kop/KafkaApisTest.java @@ -176,7 +176,8 @@ CompletableFuture checkInvalidPartition(String topic, return kafkaRequestHandler.handleOffsetCommitRequest(request); } - @Test(timeOut = 20000) + @Test(timeOut = 20000, enabled = false) + // https://github.com/streamnative/kop/issues/51 public void testOffsetCommitWithInvalidPartition() throws Exception { String topicName = "kopOffsetCommitWithInvalidPartition"; diff --git a/src/test/java/io/streamnative/kop/KafkaRequestTypeTest.java b/src/test/java/io/streamnative/kop/KafkaRequestTypeTest.java index 43485d5c811f2..dec0b09e2daf7 100644 --- a/src/test/java/io/streamnative/kop/KafkaRequestTypeTest.java +++ b/src/test/java/io/streamnative/kop/KafkaRequestTypeTest.java @@ -482,7 +482,7 @@ public void testTopicConsumerManager() throws Exception { kafkaService.getAdminClient().topics().createPartitionedTopic(kafkaTopicName, partitionNumber); int totalMsgs = 10; - String messageStrPrefix = "Message_Kop_PulsarProduceKafkaConsume_" + partitionNumber + "_"; + String messageStrPrefix = "Message_Kop_testTopicConsumerManager_" + partitionNumber + "_"; ProducerBuilder producerBuilder = pulsarClient.newProducer() .topic(pulsarTopicName) diff --git a/src/test/java/io/streamnative/kop/LogOffsetTest.java b/src/test/java/io/streamnative/kop/LogOffsetTest.java index 8f27c5db7d691..233e80b31bd51 100644 --- a/src/test/java/io/streamnative/kop/LogOffsetTest.java +++ b/src/test/java/io/streamnative/kop/LogOffsetTest.java @@ -35,7 +35,8 @@ @Slf4j public class LogOffsetTest extends KafkaApisTest { - @Test(timeOut = 20000) + @Test(timeOut = 20000, enabled = false) + // https://github.com/streamnative/kop/issues/51 public void testGetOffsetsForUnknownTopic() throws Exception { String topicName = "kopTestGetOffsetsForUnknownTopic"; diff --git a/src/test/java/io/streamnative/kop/MockKafkaServiceBaseTest.java b/src/test/java/io/streamnative/kop/MockKafkaServiceBaseTest.java index 62fd7da2b5ab3..652577db88b9e 100644 --- a/src/test/java/io/streamnative/kop/MockKafkaServiceBaseTest.java +++ b/src/test/java/io/streamnative/kop/MockKafkaServiceBaseTest.java @@ -81,11 +81,11 @@ public abstract class MockKafkaServiceBaseTest { protected URI lookupUrl; protected PulsarClient pulsarClient; - protected final int brokerWebservicePort = PortManager.nextFreePort(); - protected final int brokerWebservicePortTls = PortManager.nextFreePort(); - protected final int brokerPort = PortManager.nextFreePort(); - protected final int kafkaBrokerPort = PortManager.nextFreePort(); - protected final int kafkaBrokerPortTls = PortManager.nextFreePort(); + protected int brokerWebservicePort = PortManager.nextFreePort(); + protected int brokerWebservicePortTls = PortManager.nextFreePort(); + protected int brokerPort = PortManager.nextFreePort(); + protected int kafkaBrokerPort = PortManager.nextFreePort(); + protected int kafkaBrokerPortTls = PortManager.nextFreePort(); protected MockZooKeeper mockZookKeeper; protected NonClosableMockBookKeeper mockBookKeeper; @@ -115,6 +115,7 @@ protected void resetConfig() { this.conf.setZookeeperServers("localhost:2181"); this.conf.setConfigurationStoreServers("localhost:3181"); this.conf.setEnableGroupCoordinator(true); + this.conf.setOffsetsTopicNumPartitions(1); this.conf.setAuthenticationEnabled(false); this.conf.setAuthorizationEnabled(false); this.conf.setAllowAutoTopicCreation(true); @@ -164,7 +165,7 @@ protected final void internalCleanup() throws Exception { pulsarClient.close(); } if (kafkaService != null) { - kafkaService.close(); + stopBroker(); } if (mockBookKeeper != null) { mockBookKeeper.reallyShutdown(); @@ -431,20 +432,20 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { public static class KConsumer implements Closeable { private final KafkaConsumer consumer; private final String topic; + private final String consumerGroup; public KConsumer( String topic, String host, int port, - boolean autoCommit, String username, String password) { + boolean autoCommit, String username, String password, String consumerGroup) { Properties props = new Properties(); props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, host + ":" + port); - props.put(ConsumerConfig.GROUP_ID_CONFIG, "DemoKafkaOnPulsarConsumer"); + props.put(ConsumerConfig.GROUP_ID_CONFIG, consumerGroup); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); if (autoCommit) { props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "1000"); - props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); } else { props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); } if (null != username && null != password) { @@ -464,20 +465,25 @@ public KConsumer( this.consumer = new KafkaConsumer<>(props); this.topic = topic; + this.consumerGroup = consumerGroup; } public KConsumer(String topic, int port, boolean autoCommit) { - this(topic, "localhost", port, autoCommit, null, null); + this(topic, "localhost", port, autoCommit, null, null, "DemoKafkaOnPulsarConsumer"); } public KConsumer(String topic, String host, int port) { - this(topic, "localhost", port, false, null, null); + this(topic, "localhost", port, false, null, null, "DemoKafkaOnPulsarConsumer"); } public KConsumer(String topic, int port) { this(topic, "localhost", port); } + public KConsumer(String topic, int port, String group) { + this(topic, "localhost", port, false, null, null, group); + } + @Override public void close() { this.consumer.close(); diff --git a/src/test/java/io/streamnative/kop/SaslPlainTest.java b/src/test/java/io/streamnative/kop/SaslPlainTest.java index 5f490af7990a2..93649b53e9ff0 100644 --- a/src/test/java/io/streamnative/kop/SaslPlainTest.java +++ b/src/test/java/io/streamnative/kop/SaslPlainTest.java @@ -112,7 +112,7 @@ protected void cleanup() throws Exception { super.internalCleanup(); } - @Test(timeOut = 20000) + @Test(timeOut = 40000) void simpleProduceAndConsume() throws Exception { KProducer kProducer = new KProducer(KAFKA_TOPIC, false, "localhost", getKafkaBrokerPort(), SIMPLE_USER + "/" + NAMESPACE, "token:" + userToken); @@ -121,11 +121,11 @@ void simpleProduceAndConsume() throws Exception { for (int i = 0; i < totalMsgs; i++) { String messageStr = messageStrPrefix + i; - kProducer.getProducer().send(new ProducerRecord<>(KAFKA_TOPIC, i, messageStr)).get(); + kProducer.getProducer().send(new ProducerRecord<>(KAFKA_TOPIC, i, messageStr)); } KConsumer kConsumer = new KConsumer(KAFKA_TOPIC, "localhost", getKafkaBrokerPort(), false, - SIMPLE_USER + "/" + NAMESPACE, "token:" + userToken); + SIMPLE_USER + "/" + NAMESPACE, "token:" + userToken, "DemoKafkaOnPulsarConsumer"); kConsumer.getConsumer().subscribe(Collections.singleton(KAFKA_TOPIC)); int i = 0; @@ -147,8 +147,8 @@ void simpleProduceAndConsume() throws Exception { Map> result = kConsumer .getConsumer().listTopics(Duration.ofSeconds(1)); assertEquals(result.size(), 1); - assertTrue(result.containsKey(KAFKA_TOPIC), "list of topics " - + result.keySet().toString() + " does not contains " + KAFKA_TOPIC); + assertTrue(result.containsKey(KAFKA_TOPIC), + "list of topics " + result.keySet().toString() + " does not contains " + KAFKA_TOPIC); } @Test(timeOut = 20000) diff --git a/src/test/java/io/streamnative/kop/coordinator/group/DistributedGroupCoordinatorTest.java b/src/test/java/io/streamnative/kop/coordinator/group/DistributedGroupCoordinatorTest.java new file mode 100644 index 0000000000000..fde7ca8462620 --- /dev/null +++ b/src/test/java/io/streamnative/kop/coordinator/group/DistributedGroupCoordinatorTest.java @@ -0,0 +1,401 @@ +/** + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamnative.kop.coordinator.group; + +import static io.streamnative.kop.KafkaProtocolHandler.PLAINTEXT_PREFIX; +import static org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME; +import static org.apache.pulsar.common.naming.TopicName.PARTITIONED_TOPIC_SUFFIX; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.gson.Gson; +import io.streamnative.kop.KafkaService; +import io.streamnative.kop.KafkaServiceConfiguration; +import io.streamnative.kop.MockKafkaServiceBaseTest; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import lombok.Cleanup; +import org.apache.bookkeeper.test.PortManager; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.PartitionedTopicStats; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + + +/** + * Unit test {@link GroupCoordinator}. + */ +public class DistributedGroupCoordinatorTest extends MockKafkaServiceBaseTest { + + protected KafkaServiceConfiguration conf1; + protected KafkaServiceConfiguration conf2; + protected KafkaService kafkaService1; + protected KafkaService kafkaService2; + + protected int primaryBrokerWebservicePort; + protected int secondaryBrokerWebservicePort; + protected int primaryBrokerPort; + protected int secondaryBrokerPort; + protected int primaryKafkaBrokerPort; + protected int secondaryKafkaBrokerPort; + + protected int offsetsTopicNumPartitions; + + private static final Logger log = LoggerFactory.getLogger(DistributedGroupCoordinatorTest.class); + + protected KafkaServiceConfiguration resetConfig(int brokerPort, int webPort, int kafkaPort) { + KafkaServiceConfiguration kConfig = new KafkaServiceConfiguration(); + kConfig.setBrokerServicePort(Optional.ofNullable(brokerPort)); + kConfig.setWebServicePort(Optional.ofNullable(webPort)); + kConfig.setListeners(PLAINTEXT_PREFIX + "localhost:" + kafkaPort); + + kConfig.setOffsetsTopicNumPartitions(offsetsTopicNumPartitions); + kConfig.setEnableGroupCoordinator(true); + + kConfig.setAdvertisedAddress("localhost"); + kConfig.setClusterName(configClusterName); + kConfig.setManagedLedgerCacheSizeMB(8); + kConfig.setActiveConsumerFailoverDelayTimeMillis(0); + kConfig.setDefaultNumberOfNamespaceBundles(2); + kConfig.setZookeeperServers("localhost:2181"); + kConfig.setConfigurationStoreServers("localhost:3181"); + kConfig.setEnableGroupCoordinator(true); + kConfig.setAuthenticationEnabled(false); + kConfig.setAuthorizationEnabled(false); + kConfig.setAllowAutoTopicCreation(true); + kConfig.setAllowAutoTopicCreationType("partitioned"); + kConfig.setBrokerDeleteInactiveTopicsEnabled(false); + + return kConfig; + } + + @Override + protected void resetConfig() { + offsetsTopicNumPartitions = 16; + primaryBrokerWebservicePort = PortManager.nextFreePort(); + secondaryBrokerWebservicePort = PortManager.nextFreePort(); + primaryBrokerPort = PortManager.nextFreePort(); + secondaryBrokerPort = PortManager.nextFreePort(); + primaryKafkaBrokerPort = PortManager.nextFreePort(); + secondaryKafkaBrokerPort = PortManager.nextFreePort(); + conf1 = resetConfig( + primaryBrokerPort, + primaryBrokerWebservicePort, + primaryKafkaBrokerPort); + conf2 = resetConfig( + secondaryBrokerPort, + secondaryBrokerWebservicePort, + secondaryKafkaBrokerPort); + conf = conf1; + + brokerPort = primaryBrokerPort; + brokerWebservicePort = primaryBrokerWebservicePort; + kafkaBrokerPort = primaryKafkaBrokerPort; + + log.info("Ports -- broker1: {}, brokerWeb1:{}, kafka1: {}", + primaryBrokerPort, primaryBrokerWebservicePort, primaryKafkaBrokerPort); + log.info("Ports -- broker2: {}, brokerWeb2:{}, kafka2: {}\n", + secondaryBrokerPort, secondaryBrokerWebservicePort, secondaryKafkaBrokerPort); + } + + @Override + protected void startBroker() throws Exception { + this.kafkaService1 = startBroker(conf1); + this.kafkaService = kafkaService1; + this.kafkaService2 = startBroker(conf2); + } + + @Override + protected void stopBroker() throws Exception { + kafkaService1.close(); + kafkaService2.close(); + } + + @BeforeMethod + @Override + public void setup() throws Exception { + super.internalSetup(); + + if (!admin.clusters().getClusters().contains(configClusterName)) { + // so that clients can test short names + admin.clusters().createCluster(configClusterName, + new ClusterData("http://127.0.0.1:" + brokerWebservicePort)); + } else { + admin.clusters().updateCluster(configClusterName, + new ClusterData("http://127.0.0.1:" + brokerWebservicePort)); + } + + if (!admin.tenants().getTenants().contains("public")) { + admin.tenants().createTenant("public", + new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); + } else { + admin.tenants().updateTenant("public", + new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); + } + if (!admin.namespaces().getNamespaces("public").contains("public/default")) { + admin.namespaces().createNamespace("public/default"); + admin.namespaces().setNamespaceReplicationClusters("public/default", Sets.newHashSet("test")); + admin.namespaces().setRetention("public/default", + new RetentionPolicies(60, 1000)); + } + if (!admin.namespaces().getNamespaces("public").contains("public/__kafka")) { + admin.namespaces().createNamespace("public/__kafka"); + admin.namespaces().setNamespaceReplicationClusters("public/__kafka", Sets.newHashSet("test")); + admin.namespaces().setRetention("public/__kafka", + new RetentionPolicies(-1, -1)); + } + + List brokers = admin.brokers().getActiveBrokers(configClusterName); + Assert.assertEquals(brokers.size(), 2); + log.info("broker1: {} broker2: {}", brokers.get(0), brokers.get(1)); + } + + + @AfterMethod + @Override + public void cleanup() throws Exception { + log.info("--- Shutting down ---"); + super.internalCleanup(); + } + + protected void kafkaPublishMessage(KProducer kProducer, int numMessages, String messageStrPrefix) throws Exception { + for (int i = 0; i < numMessages; i++) { + String messageStr = messageStrPrefix + i; + ProducerRecord record = new ProducerRecord<>( + kProducer.getTopic(), + i, + messageStr); + + kProducer.getProducer() + .send(record) + .get(); + if (log.isDebugEnabled()) { + log.debug("Kafka Producer {} Sent message with header: ({}, {})", + kProducer.getTopic(), i, messageStr); + } + } + } + + protected void kafkaConsumeCommitMessage(KConsumer kConsumer, + int numMessages, + String messageStrPrefix, + List topicPartitions) { + kConsumer.getConsumer().assign(topicPartitions); + int i = 0; + while (i < numMessages) { + if (log.isDebugEnabled()) { + log.debug("kConsumer {} start poll message: {}", + kConsumer.getTopic() + kConsumer.getConsumerGroup(), i); + } + ConsumerRecords records = kConsumer.getConsumer().poll(Duration.ofSeconds(1)); + for (ConsumerRecord record : records) { + Integer key = record.key(); + assertEquals(messageStrPrefix + key.toString(), record.value()); + + if (log.isDebugEnabled()) { + log.debug("Kafka consumer get message: {}, key: {} at offset {}", + record.key(), record.value(), record.offset()); + } + i++; + } + } + + kConsumer.getConsumer().commitSync(); + + if (log.isDebugEnabled()) { + log.debug("kConsumer {} finished poll and commit message: {}", + kConsumer.getTopic() + kConsumer.getConsumerGroup(), i); + } + assertEquals(i, numMessages); + } + + @Test(timeOut = 30000) + public void testMutiBrokerAndCoordinator() throws Exception { + int partitionNumber = 10; + String kafkaTopicName = "kopMutiBrokerAndCoordinator" + partitionNumber; + String pulsarTopicName = "persistent://public/default/" + kafkaTopicName; + + String offsetNs = conf.getKafkaMetadataTenant() + "/" + conf.getKafkaMetadataNamespace(); + String offsetsTopicName = "persistent://" + offsetNs + "/" + GROUP_METADATA_TOPIC_NAME; + + // 0. Preparing: + // create partitioned topic. + kafkaService1.getAdminClient().topics().createPartitionedTopic(kafkaTopicName, partitionNumber); + // Because kafkaService1 is start firstly. all the offset topics is served in broker1. + // In setting, each ns has 2 bundles. unload the first part, and this part will be served by broker2. + kafkaService1.getAdminClient().namespaces().unloadNamespaceBundle(offsetNs, "0x00000000_0x80000000"); + + // Offsets partitions should be served by 2 brokers now. + Map> offsetTopicMap = Maps.newHashMap(); + for (int ii = 0; ii < offsetsTopicNumPartitions; ii++) { + String offsetsTopic = offsetsTopicName + PARTITIONED_TOPIC_SUFFIX + ii; + String result = admin.lookups().lookupTopic(offsetsTopic); + offsetTopicMap.putIfAbsent(result, Lists.newArrayList()); + offsetTopicMap.get(result).add(offsetsTopic); + log.info("serving broker for offset topic {} is {}", offsetsTopic, result); + } + assertEquals(offsetTopicMap.size(), 2); + + final AtomicInteger numberTopic = new AtomicInteger(0); + offsetTopicMap.values().stream().forEach(list -> numberTopic.addAndGet(list.size())); + assertEquals(numberTopic.get(), offsetsTopicNumPartitions); + + // 1. produce message with Kafka producer. + int totalMsgs = 50; + String messageStrPrefix = "Message_Kop_KafkaProduceKafkaConsume_" + partitionNumber + "_"; + @Cleanup + KProducer kProducer = new KProducer(kafkaTopicName, false, getKafkaBrokerPort()); + kafkaPublishMessage(kProducer, totalMsgs, messageStrPrefix); + + // 2. create 4 kafka consumer from different consumer groups. + // consume data and commit offsets for 4 consumer group. + @Cleanup + KConsumer kConsumer1 = new KConsumer(kafkaTopicName, getKafkaBrokerPort(), "consumer-group-1"); + @Cleanup + KConsumer kConsumer2 = new KConsumer(kafkaTopicName, getKafkaBrokerPort(), "consumer-group-2"); + @Cleanup + KConsumer kConsumer3 = new KConsumer(kafkaTopicName, getKafkaBrokerPort(), "consumer-group-3"); + @Cleanup + KConsumer kConsumer4 = new KConsumer(kafkaTopicName, getKafkaBrokerPort(), "consumer-group-4"); + + List topicPartitions = IntStream.range(0, partitionNumber) + .mapToObj(i -> new TopicPartition(kafkaTopicName, i)).collect(Collectors.toList()); + + log.info("Partition size: {}, will consume and commitOffset for 4 consumers", + topicPartitions.size()); + + kafkaConsumeCommitMessage(kConsumer1, totalMsgs, messageStrPrefix, topicPartitions); + kafkaConsumeCommitMessage(kConsumer2, totalMsgs, messageStrPrefix, topicPartitions); + kafkaConsumeCommitMessage(kConsumer3, totalMsgs, messageStrPrefix, topicPartitions); + kafkaConsumeCommitMessage(kConsumer4, totalMsgs, messageStrPrefix, topicPartitions); + + // 3. use a map for serving broker and topics , verify both broker has messages served. + Map> topicMap = Maps.newHashMap(); + for (int ii = 0; ii < partitionNumber; ii++) { + String topicName = pulsarTopicName + PARTITIONED_TOPIC_SUFFIX + ii; + String result = admin.lookups().lookupTopic(topicName); + topicMap.putIfAbsent(result, Lists.newArrayList()); + topicMap.get(result).add(topicName); + log.info("serving broker for topic {} is {}", topicName, result); + } + assertTrue(topicMap.size() == 2); + + final AtomicInteger numberTopic2 = new AtomicInteger(0); + topicMap.values().stream().forEach(list -> numberTopic2.addAndGet(list.size())); + assertTrue(numberTopic2.get() == partitionNumber); + + final PartitionedTopicStats topicStats = admin.topics().getPartitionedStats(pulsarTopicName, true); + log.info("PartitionedTopicStats for topic {} : {}", pulsarTopicName, new Gson().toJson(topicStats)); + + topicMap.forEach((broker, topics) -> { + AtomicLong brokerStorageSize = new AtomicLong(0); + topics.forEach(topic -> { + brokerStorageSize.addAndGet(topicStats.partitions.get(topic).storageSize); + }); + log.info("get data topics served by broker {}, broker storage size: {}", broker, brokerStorageSize.get()); + assertTrue(brokerStorageSize.get() > 0L); + }); + + offsetTopicMap = Maps.newHashMap(); + for (int ii = 0; ii < offsetsTopicNumPartitions; ii++) { + String offsetsTopic = offsetsTopicName + PARTITIONED_TOPIC_SUFFIX + ii; + String result = admin.lookups().lookupTopic(offsetsTopic); + offsetTopicMap.putIfAbsent(result, Lists.newArrayList()); + offsetTopicMap.get(result).add(offsetsTopic); + log.info("serving broker for offset topic {} is {}", offsetsTopic, result); + } + + log.info("producer size1: {}, size2: {}", + kafkaService1.getGroupCoordinator().getOffsetsProducers().size(), + kafkaService2.getGroupCoordinator().getOffsetsProducers().size()); + log.info("reader size1: {}, size2: {}", + kafkaService1.getGroupCoordinator().getOffsetsReaders().size(), + kafkaService2.getGroupCoordinator().getOffsetsReaders().size()); + + // 4. unload ns, coordinator will be on another broker + // verify consumer group still keep the old offset, and consumers will poll no data. + log.info("Unload offset namespace, this will trigger another reload. After reload verify offset."); + kafkaService1.getAdminClient().namespaces().unload(offsetNs); + + // verify offset be kept and no more records could read. + ConsumerRecords records = kConsumer1.getConsumer().poll(Duration.ofMillis(200)); + assertTrue(records.isEmpty()); + records = kConsumer2.getConsumer().poll(Duration.ofMillis(200)); + assertTrue(records.isEmpty()); + records = kConsumer3.getConsumer().poll(Duration.ofMillis(200)); + assertTrue(records.isEmpty()); + records = kConsumer4.getConsumer().poll(Duration.ofMillis(200)); + assertTrue(records.isEmpty()); + + // 5. another round publish and consume after ns unload. + kafkaPublishMessage(kProducer, totalMsgs, messageStrPrefix); + kafkaConsumeCommitMessage(kConsumer1, totalMsgs, messageStrPrefix, topicPartitions); + kafkaConsumeCommitMessage(kConsumer2, totalMsgs, messageStrPrefix, topicPartitions); + kafkaConsumeCommitMessage(kConsumer3, totalMsgs, messageStrPrefix, topicPartitions); + kafkaConsumeCommitMessage(kConsumer4, totalMsgs, messageStrPrefix, topicPartitions); + + offsetTopicMap = Maps.newHashMap(); + for (int ii = 0; ii < offsetsTopicNumPartitions; ii++) { + String offsetsTopic = offsetsTopicName + PARTITIONED_TOPIC_SUFFIX + ii; + String result = admin.lookups().lookupTopic(offsetsTopic); + offsetTopicMap.putIfAbsent(result, Lists.newArrayList()); + offsetTopicMap.get(result).add(offsetsTopic); + log.info("serving broker for offset topic {} is {}", offsetsTopic, result); + } + + log.info("producer broker 1 size : {}, broker 2 : {}", + kafkaService1.getGroupCoordinator().getOffsetsProducers().size(), + kafkaService2.getGroupCoordinator().getOffsetsProducers().size()); + log.info("reader broker 1 size : {}, broker 2 : {}", + kafkaService1.getGroupCoordinator().getOffsetsReaders().size(), + kafkaService2.getGroupCoordinator().getOffsetsReaders().size()); + + assertTrue(kafkaService2.getGroupCoordinator().getOffsetsProducers().size() > 0); + + // 6. unload ns, coordinator will be on another broker + // verify consumer group still keep the old offset, and consumers will poll no data. + log.info("Unload offset namespace, this will trigger another reload"); + kafkaService1.getAdminClient().namespaces().unload(offsetNs); + + // verify offset be kept and no more records could read. + records = kConsumer1.getConsumer().poll(Duration.ofMillis(200)); + assertTrue(records.isEmpty()); + records = kConsumer2.getConsumer().poll(Duration.ofMillis(200)); + assertTrue(records.isEmpty()); + records = kConsumer3.getConsumer().poll(Duration.ofMillis(200)); + assertTrue(records.isEmpty()); + records = kConsumer4.getConsumer().poll(Duration.ofMillis(200)); + assertTrue(records.isEmpty()); + } +} diff --git a/src/test/java/io/streamnative/kop/coordinator/group/GroupCoordinatorTest.java b/src/test/java/io/streamnative/kop/coordinator/group/GroupCoordinatorTest.java index fa0075c935a60..d84e3e521a1cc 100644 --- a/src/test/java/io/streamnative/kop/coordinator/group/GroupCoordinatorTest.java +++ b/src/test/java/io/streamnative/kop/coordinator/group/GroupCoordinatorTest.java @@ -51,8 +51,8 @@ import org.apache.kafka.common.requests.TransactionResult; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.Reader; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.common.policies.data.ClusterData; @@ -68,8 +68,6 @@ */ public class GroupCoordinatorTest extends MockKafkaServiceBaseTest { - private static final String ClientId = "consumer-test"; - private static final String ClientHost = "localhost"; private static final int ConsumerMinSessionTimeout = 10; private static final int ConsumerMaxSessionTimeout = 10000; private static final int DefaultRebalanceTimeout = 500; @@ -83,8 +81,10 @@ public class GroupCoordinatorTest extends MockKafkaServiceBaseTest { String topicName; MockTimer timer = null; GroupCoordinator groupCoordinator = null; - Producer producer; - Reader reader; + + ProducerBuilder producerBuilder; + ReaderBuilder readerBuilder; + Consumer consumer; OrderedScheduler scheduler; GroupMetadataManager groupMetadataManager; @@ -127,15 +127,13 @@ public void setup() throws Exception { ConsumerMaxSessionTimeout, GroupInitialRebalanceDelay ); - OffsetConfig offsetConfig = OffsetConfig.builder().build(); - - timer = new MockTimer(); topicName = "test-coordinator-" + System.currentTimeMillis(); + OffsetConfig offsetConfig = OffsetConfig.builder().offsetsTopicName(topicName).build(); - producer = pulsarClient.newProducer(Schema.BYTEBUFFER) - .topic(topicName) - .create(); + timer = new MockTimer(); + + producerBuilder = pulsarClient.newProducer(Schema.BYTEBUFFER); consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) .topic(topicName) @@ -143,19 +141,17 @@ public void setup() throws Exception { .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); - reader = pulsarClient.newReader(Schema.BYTEBUFFER) - .topic(topicName) - .startMessageId(MessageId.earliest) - .create(); + readerBuilder = pulsarClient.newReader(Schema.BYTEBUFFER) + .startMessageId(MessageId.earliest); groupPartitionId = 0; otherGroupPartitionId = 1; otherGroupId = "otherGroupId"; + offsetConfig.offsetsTopicNumPartitions(4); groupMetadataManager = spy(new GroupMetadataManager( - 4, offsetConfig, - producer, - reader, + producerBuilder, + readerBuilder, scheduler, timer.time(), id -> { @@ -203,8 +199,6 @@ public void setup() throws Exception { public void cleanup() throws Exception { groupCoordinator.shutdown(); groupMetadataManager.shutdown(); - producer.close(); - reader.close(); consumer.close(); scheduler.shutdown(); super.internalCleanup(); @@ -280,7 +274,7 @@ public void testJoinGroupUnknowMemberId() throws Exception { JoinGroupResult joinGroupResult = joinGroup( otherGroupId, memberId, protocolType, protocols ); - assertEquals(Errors.NONE, joinGroupResult.getError()); + assertEquals(Errors.NOT_COORDINATOR, joinGroupResult.getError()); } @Test @@ -389,7 +383,7 @@ public void testJoinGroupUnknownConsumerExistingGroup() throws Exception { @Test public void testHeartbeatWrongCoordinator() throws Exception { Errors error = groupCoordinator.handleHeartbeat(otherGroupId, memberId, -1).get(); - assertEquals(Errors.UNKNOWN_MEMBER_ID, error); + assertEquals(Errors.NOT_COORDINATOR, error); } @Test @@ -737,8 +731,7 @@ public void testSyncGroupEmptyAssignment() throws Exception { assertEquals(Errors.NONE, heartbeatResult); } - @Test(enabled = false) - // TODO: https://github.com/streamnative/kop/issues/32 + @Test public void testSyncGroupOtherGroupId() throws Exception { int generation = 1; KeyValue syncGroupResult = groupCoordinator.handleSyncGroup( @@ -1542,8 +1535,7 @@ public void testFetchOffsetForUnknownPartition() { assertEquals(OffsetFetchResponse.INVALID_OFFSET, fetchOffsetsResult.getValue().get(tp).offset); } - @Test(enabled = false) - // TODO: https://github.com/streamnative/kop/issues/32 + @Test public void testFetchOffsetNotCoordinatorForGroup() { TopicPartition tp = new TopicPartition("topic", 0); KeyValue> fetchOffsetsResult = @@ -1678,7 +1670,7 @@ public void testLeaveGroupWrongCoordinator() throws Exception { Errors leaveGroupResult = groupCoordinator.handleLeaveGroup( otherGroupId, JoinGroupRequest.UNKNOWN_MEMBER_ID ).get(); - assertEquals(Errors.UNKNOWN_MEMBER_ID, leaveGroupResult); + assertEquals(Errors.NOT_COORDINATOR, leaveGroupResult); } @Test @@ -1764,7 +1756,7 @@ groupId, memberId, protocolType, newProtocols() @Test public void testDescribeGroupWrongCoordinator() { KeyValue describeGroupResult = groupCoordinator.handleDescribeGroup(otherGroupId); - assertEquals(Errors.NONE, describeGroupResult.getKey()); + assertEquals(Errors.NOT_COORDINATOR, describeGroupResult.getKey()); } @Test diff --git a/src/test/java/io/streamnative/kop/coordinator/group/GroupMetadataManagerTest.java b/src/test/java/io/streamnative/kop/coordinator/group/GroupMetadataManagerTest.java index 10066b567a654..17b68c43ed548 100644 --- a/src/test/java/io/streamnative/kop/coordinator/group/GroupMetadataManagerTest.java +++ b/src/test/java/io/streamnative/kop/coordinator/group/GroupMetadataManagerTest.java @@ -20,17 +20,17 @@ import static io.streamnative.kop.coordinator.group.GroupState.Empty; import static io.streamnative.kop.coordinator.group.GroupState.PreparingRebalance; import static io.streamnative.kop.coordinator.group.GroupState.Stable; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -59,7 +59,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.CompressionType; @@ -73,19 +72,21 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; +import org.apache.kafka.common.utils.Time; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.Reader; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfo; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; /** * Unit test {@link GroupMetadataManager}. @@ -95,27 +96,18 @@ public class GroupMetadataManagerTest extends MockKafkaServiceBaseTest { private static final String groupId = "foo"; private static final int groupPartitionId = 0; - private static final TopicPartition groupTopicPartition = - new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId); + private static final String protocolType = "protocolType"; private static final int rebalanceTimeout = 60000; private static final int sessionTimeout = 10000; - MockTime time = null; GroupMetadataManager groupMetadataManager = null; - Producer producer = null; - Reader consumer = null; + ProducerBuilder producer = null; + ReaderBuilder consumer = null; OffsetConfig offsetConfig = OffsetConfig.builder().build(); OrderedScheduler scheduler; - @Override - protected void resetConfig() { - super.resetConfig(); - // since this test mock all Group Coordinator, we disable the one in Kafka broker. - this.conf.setEnableGroupCoordinator(false); - } - - @Before + @BeforeMethod @Override public void setup() throws Exception { super.internalSetup(); @@ -125,30 +117,46 @@ public void setup() throws Exception { .numThreads(1) .build(); - admin.clusters().createCluster("test", - new ClusterData("http://127.0.0.1:" + brokerWebservicePort)); - admin.tenants().createTenant("public", - new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); - admin.namespaces().createNamespace("public/default"); - admin.namespaces().setNamespaceReplicationClusters("public/default", Sets.newHashSet("test")); - admin.namespaces().setRetention("public/default", - new RetentionPolicies(20, 100)); + if (!admin.clusters().getClusters().contains(configClusterName)) { + // so that clients can test short names + admin.clusters().createCluster(configClusterName, + new ClusterData("http://127.0.0.1:" + brokerWebservicePort)); + } else { + admin.clusters().updateCluster(configClusterName, + new ClusterData("http://127.0.0.1:" + brokerWebservicePort)); + } - time = new MockTime(); - groupMetadataManager = new GroupMetadataManager( - 1, - offsetConfig, - producer, - consumer, - scheduler, - time - ); + if (!admin.tenants().getTenants().contains("public")) { + admin.tenants().createTenant("public", + new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); + } else { + admin.tenants().updateTenant("public", + new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); + } + if (!admin.namespaces().getNamespaces("public").contains("public/default")) { + admin.namespaces().createNamespace("public/default"); + admin.namespaces().setNamespaceReplicationClusters("public/default", Sets.newHashSet("test")); + admin.namespaces().setRetention("public/default", + new RetentionPolicies(60, 1000)); + } + if (!admin.namespaces().getNamespaces("public").contains("public/__kafka")) { + admin.namespaces().createNamespace("public/__kafka"); + admin.namespaces().setNamespaceReplicationClusters("public/__kafka", Sets.newHashSet("test")); + admin.namespaces().setRetention("public/__kafka", + new RetentionPolicies(20, 100)); + } + + groupMetadataManager = kafkaService.getGroupCoordinator().getGroupManager(); } - @After + @AfterMethod @Override public void cleanup() throws Exception { + if (groupMetadataManager != null) { + groupMetadataManager.shutdown(); + } + scheduler.shutdown(); super.internalCleanup(); } @@ -239,7 +247,7 @@ private ByteBuffer newMemoryRecordsBuffer(List records, short producerEpoch, boolean isTxnOffsetCommit) { TimestampType timestampType = TimestampType.CREATE_TIME; - long timestamp = time.milliseconds(); + long timestamp = Time.SYSTEM.milliseconds(); ByteBuffer buffer = ByteBuffer.allocate( AbstractRecords.estimateSizeInBytes( @@ -291,7 +299,7 @@ private int completeTransactionalOffsetCommit(ByteBuffer buffer, boolean isCommit) { MemoryRecordsBuilder builder = MemoryRecords.builder( buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, - TimestampType.LOG_APPEND_TIME, baseOffset, time.milliseconds(), + TimestampType.LOG_APPEND_TIME, baseOffset, Time.SYSTEM.milliseconds(), producerId, producerEpoch, 0, true, true, RecordBatch.NO_PARTITION_LEADER_EPOCH); ControlRecordType controlRecordType; @@ -300,15 +308,13 @@ private int completeTransactionalOffsetCommit(ByteBuffer buffer, } else { controlRecordType = ControlRecordType.ABORT; } - builder.appendEndTxnMarker(time.milliseconds(), new EndTransactionMarker(controlRecordType, 0)); + builder.appendEndTxnMarker(Time.SYSTEM.milliseconds(), new EndTransactionMarker(controlRecordType, 0)); builder.build(); return 1; } @Test public void testLoadOffsetsWithoutGroup() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; - Map committedOffsets = new HashMap<>(); committedOffsets.put( new TopicPartition("foo", 0), 23L); @@ -324,37 +330,35 @@ public void testLoadOffsetsWithoutGroup() throws Exception { ByteBuffer buffer = newMemoryRecordsBuffer(offsetCommitRecords); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata group = onLoadedFuture.get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - assertSame(group, groupInCache); - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - assertEquals(committedOffsets.size(), group.allOffsets().size()); - committedOffsets.forEach((tp, offset) -> - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); - }); + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata group = onLoadedFuture.get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + assertSame(group, groupInCache); + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + assertEquals(committedOffsets.size(), group.allOffsets().size()); + committedOffsets.forEach((tp, offset) -> + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); + } @Test public void testLoadEmptyGroupWithOffsets() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; int generation = 15; String protocolType = "consumer"; @@ -376,41 +380,38 @@ public void testLoadEmptyGroupWithOffsets() throws Exception { ByteBuffer buffer = newMemoryRecordsBuffer(offsetCommitRecords); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata group = onLoadedFuture.get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - assertSame(group, groupInCache); - - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - assertEquals(generation, group.generationId()); - assertEquals(Optional.of(protocolType), group.protocolType()); - assertEquals(committedOffsets.size(), group.allOffsets().size()); - assertNull(group.leaderOrNull()); - assertNull(group.protocolOrNull()); - committedOffsets.forEach((tp, offset) -> - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); - }); + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata group = onLoadedFuture.get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + assertSame(group, groupInCache); + + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + assertEquals(generation, group.generationId()); + assertEquals(Optional.of(protocolType), group.protocolType()); + assertEquals(committedOffsets.size(), group.allOffsets().size()); + assertNull(group.leaderOrNull()); + assertNull(group.protocolOrNull()); + committedOffsets.forEach((tp, offset) -> + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); } @Test public void testLoadTransactionalOffsetsWithoutGroup() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; long producerId = 1000L; short producerEpoch = 2; @@ -433,37 +434,34 @@ public void testLoadTransactionalOffsetsWithoutGroup() throws Exception { buffer.flip(); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata group = onLoadedFuture.get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - assertSame(group, groupInCache); - - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - assertEquals(committedOffsets.size(), group.allOffsets().size()); - committedOffsets.forEach((tp, offset) -> - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); - }); + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata group = onLoadedFuture.get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + assertSame(group, groupInCache); + + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + assertEquals(committedOffsets.size(), group.allOffsets().size()); + committedOffsets.forEach((tp, offset) -> + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); } @Test public void testDoNotLoadAbortedTransactionalOffsetCommits() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; long producerId = 1000L; short producerEpoch = 2; @@ -483,26 +481,23 @@ public void testDoNotLoadAbortedTransactionalOffsetCommits() throws Exception { byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> {} - ).get(); - Optional groupInCache = groupMetadataManager.getGroup(groupId); - assertFalse(groupInCache.isPresent()); - }); + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> {} + ).get(); + Optional groupInCache = groupMetadataManager.getGroup(groupId); + assertFalse(groupInCache.isPresent()); } @Test public void testGroupLoadedWithPendingCommits() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; long producerId = 1000L; short producerEpoch = 2; @@ -521,37 +516,35 @@ public void testGroupLoadedWithPendingCommits() throws Exception { byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - GroupMetadata group = onLoadedFuture.get(); - assertSame(group, groupInCache); - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - // Ensure that no offsets are materialized, but that we have offsets pending. - assertEquals(0, group.allOffsets().size()); - assertTrue(group.hasOffsets()); - assertTrue(group.hasPendingOffsetCommitsFromProducer(producerId)); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + GroupMetadata group = onLoadedFuture.get(); + assertSame(group, groupInCache); + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + // Ensure that no offsets are materialized, but that we have offsets pending. + assertEquals(0, group.allOffsets().size()); + assertTrue(group.hasOffsets()); + assertTrue(group.hasPendingOffsetCommitsFromProducer(producerId)); } @Test public void testLoadWithCommitedAndAbortedTransactionOffsetCommits() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; long producerId = 1000L; short producerEpoch = 2; @@ -581,41 +574,39 @@ public void testLoadWithCommitedAndAbortedTransactionOffsetCommits() throws Exce byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - GroupMetadata group = onLoadedFuture.get(); - assertSame(group, groupInCache); - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - // Ensure that only the committed offsets are materialized, and that there are no pending - // commits for the producer. This allows us to be certain that the aborted offset commits - // - // are truly discarded. - assertEquals(committedOffsets.size(), group.allOffsets().size()); - committedOffsets.forEach((tp, offset) -> - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); - assertFalse(group.hasPendingOffsetCommitsFromProducer(producerId)); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + GroupMetadata group = onLoadedFuture.get(); + assertSame(group, groupInCache); + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + // Ensure that only the committed offsets are materialized, and that there are no pending + // commits for the producer. This allows us to be certain that the aborted offset commits + // + // are truly discarded. + assertEquals(committedOffsets.size(), group.allOffsets().size()); + committedOffsets.forEach((tp, offset) -> + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); + assertFalse(group.hasPendingOffsetCommitsFromProducer(producerId)); } @Test public void testLoadWithCommitedAndAbortedAndPendingTransactionOffsetCommits() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; long producerId = 1000L; short producerEpoch = 2; @@ -654,51 +645,50 @@ public void testLoadWithCommitedAndAbortedAndPendingTransactionOffsetCommits() t byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - GroupMetadata group = onLoadedFuture.get(); - assertSame(group, groupInCache); - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - - // Ensure that only the committed offsets are materialized, and that there are no pending commits - // for the producer. This allows us to be certain that the aborted offset commits are truly discarded. - assertEquals(committedOffsets.size(), group.allOffsets().size()); - committedOffsets.forEach((tp, offset) -> - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); - - // We should have pending commits. - assertTrue(group.hasPendingOffsetCommitsFromProducer(producerId)); - - // The loaded pending commits should materialize after a commit marker comes in. - groupMetadataManager.handleTxnCompletion( - producerId, - Sets.newHashSet(groupMetadataTopicPartition.partition()), - true); - assertFalse(group.hasPendingOffsetCommitsFromProducer(producerId)); - pendingOffsets.forEach((tp, offset) -> - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + GroupMetadata group = onLoadedFuture.get(); + assertSame(group, groupInCache); + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + + // Ensure that only the committed offsets are materialized, and that there are no pending commits + // for the producer. This allows us to be certain that the aborted offset commits are truly discarded. + assertEquals(committedOffsets.size(), group.allOffsets().size()); + committedOffsets.forEach((tp, offset) -> + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); + + // We should have pending commits. + assertTrue(group.hasPendingOffsetCommitsFromProducer(producerId)); + + // The loaded pending commits should materialize after a commit marker comes in. + groupMetadataManager.handleTxnCompletion( + producerId, + Sets.newHashSet(groupPartitionId), + true); + assertFalse(group.hasPendingOffsetCommitsFromProducer(producerId)); + pendingOffsets.forEach((tp, offset) -> + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset))); + } @Test public void testLoadTransactionalOffsetCommitsFromMultipleProducers() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; long firstProducerId = 1000L; short firstProducerEpoch = 2; long secondProducerId = 1001L; @@ -739,51 +729,50 @@ public void testLoadTransactionalOffsetCommitsFromMultipleProducers() throws Exc buffer.flip(); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata group = onLoadedFuture.get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - assertSame(group, groupInCache); - - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - - // Ensure that only the committed offsets are materialized, and that there are no pending commits - // for the producer. This allows us to be certain that the aborted offset commits are truly discarded. - assertEquals(committedOffsetsFirstProducer.size() + committedOffsetsSecondProducer.size(), - group.allOffsets().size()); - committedOffsetsFirstProducer.forEach((tp, offset) -> { - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); - assertEquals( - Optional.of((long) firstProduceRecordOffset), - group.offsetWithRecordMetadata(tp).flatMap(CommitRecordMetadataAndOffset::appendedBatchOffset)); - }); - committedOffsetsSecondProducer.forEach((tp, offset) -> { - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); - assertEquals( - Optional.of((long) secondProduceRecordOffset), - group.offsetWithRecordMetadata(tp).flatMap(CommitRecordMetadataAndOffset::appendedBatchOffset)); - }); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata group = onLoadedFuture.get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + assertSame(group, groupInCache); + + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + + // Ensure that only the committed offsets are materialized, and that there are no pending commits + // for the producer. This allows us to be certain that the aborted offset commits are truly discarded. + assertEquals(committedOffsetsFirstProducer.size() + committedOffsetsSecondProducer.size(), + group.allOffsets().size()); + committedOffsetsFirstProducer.forEach((tp, offset) -> { + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); + assertEquals( + Optional.of((long) firstProduceRecordOffset), + group.offsetWithRecordMetadata(tp).flatMap(CommitRecordMetadataAndOffset::appendedBatchOffset)); + }); + committedOffsetsSecondProducer.forEach((tp, offset) -> { + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); + assertEquals( + Optional.of((long) secondProduceRecordOffset), + group.offsetWithRecordMetadata(tp).flatMap(CommitRecordMetadataAndOffset::appendedBatchOffset)); + }); + } @Test public void testGroupLoadWithConsumerAndTransactionalOffsetCommitsTransactionWins() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; long producerId = 1000L; short producerEpoch = 2; @@ -809,42 +798,49 @@ public void testGroupLoadWithConsumerAndTransactionalOffsetCommitsTransactionWin buffer.flip(); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata group = onLoadedFuture.get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - assertSame(group, groupInCache); - - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - - // The group should be loaded with pending offsets. - assertEquals(1, group.allOffsets().size()); - assertTrue(group.hasOffsets()); - assertFalse(group.hasPendingOffsetCommitsFromProducer(producerId)); - assertEquals(consumerOffsetCommits.size(), group.allOffsets().size()); - transactionalOffsetCommits.forEach((tp, offset) -> { - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); - }); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata group = onLoadedFuture.get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + assertSame(group, groupInCache); + + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + + // The group should be loaded with pending offsets. + assertEquals(1, group.allOffsets().size()); + assertTrue(group.hasOffsets()); + assertFalse(group.hasPendingOffsetCommitsFromProducer(producerId)); + assertEquals(consumerOffsetCommits.size(), group.allOffsets().size()); + transactionalOffsetCommits.forEach((tp, offset) -> { + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); + }); + } @Test public void testGroupNotExits() { + groupMetadataManager = new GroupMetadataManager( + offsetConfig, + producer, + consumer, + scheduler, + new MockTime() + ); // group is not owned assertFalse(groupMetadataManager.groupNotExists(groupId)); @@ -865,7 +861,6 @@ public void testGroupNotExits() { @Test public void testLoadOffsetsWithTombstones() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; TopicPartition tombstonePartition = new TopicPartition("foo", 1); Map committedOffsets = new HashMap<>(); @@ -886,44 +881,43 @@ public void testLoadOffsetsWithTombstones() throws Exception { ByteBuffer buffer = newMemoryRecordsBuffer(offsetCommitRecords); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata group = onLoadedFuture.get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - assertSame(group, groupInCache); - - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - - // The group should be loaded with pending offsets. - assertEquals(committedOffsets.size() - 1, group.allOffsets().size()); - committedOffsets.forEach((tp, offset) -> { - if (tp == tombstonePartition) { - assertEquals(Optional.empty(), group.offset(tp)); - } else { - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); - } - }); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata group = onLoadedFuture.get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + assertSame(group, groupInCache); + + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + + // The group should be loaded with pending offsets. + assertEquals(committedOffsets.size() - 1, group.allOffsets().size()); + committedOffsets.forEach((tp, offset) -> { + if (tp == tombstonePartition) { + assertEquals(Optional.empty(), group.offset(tp)); + } else { + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); + } + }); + } @Test public void testLoadOffsetsAndGroup() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; int generation = 935; String protocolType = "consumer"; String protocol = "range"; @@ -949,47 +943,46 @@ public void testLoadOffsetsAndGroup() throws Exception { ByteBuffer buffer = newMemoryRecordsBuffer(offsetCommitRecords); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata group = onLoadedFuture.get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - assertSame(group, groupInCache); - - assertEquals(groupId, group.groupId()); - assertEquals(Stable, group.currentState()); - assertEquals(memberId, group.leaderOrNull()); - assertEquals(generation, group.generationId()); - assertEquals(Optional.of(protocolType), group.protocolType()); - assertEquals( - Lists.newArrayList(memberId), - group.allMembers().stream().collect(Collectors.toList())); - assertEquals( - committedOffsets.size(), - group.allOffsets().size() - ); - committedOffsets.forEach((tp, offset) -> { - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); - }); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata group = onLoadedFuture.get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + assertSame(group, groupInCache); + + assertEquals(groupId, group.groupId()); + assertEquals(Stable, group.currentState()); + assertEquals(memberId, group.leaderOrNull()); + assertEquals(generation, group.generationId()); + assertEquals(Optional.of(protocolType), group.protocolType()); + assertEquals( + Lists.newArrayList(memberId), + group.allMembers().stream().collect(Collectors.toList())); + assertEquals( + committedOffsets.size(), + group.allOffsets().size() + ); + committedOffsets.forEach((tp, offset) -> { + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); + }); + } @Test public void testLoadGroupWithTombstone() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; int generation = 935; String memberId = "98098230493"; String protocolType = "consumer"; @@ -1011,20 +1004,20 @@ public void testLoadGroupWithTombstone() throws Exception { )); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> {} - ).get(); - assertFalse(groupMetadataManager.getGroup(groupId).isPresent()); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> {} + ).get(); + assertFalse(groupMetadataManager.getGroup(groupId).isPresent()); + } @Test @@ -1032,8 +1025,6 @@ public void testOffsetWriteAfterGroupRemoved() throws Exception { // this test case checks the following scenario: // 1. the group exists at some point in time, but is later removed (because all members left) // 2. a "simple" consumer (i.e. not a consumer group) then uses the same groupId to commit some offsets - - TopicPartition groupMetadataTopicPartition = groupTopicPartition; int generation = 293; String memberId = "98098230493"; String protocolType = "consumer"; @@ -1067,38 +1058,36 @@ public void testOffsetWriteAfterGroupRemoved() throws Exception { ByteBuffer buffer = newMemoryRecordsBuffer(newOffsetCommitRecords); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata group = onLoadedFuture.get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - assertSame(group, groupInCache); - - assertEquals(groupId, group.groupId()); - assertEquals(Empty, group.currentState()); - assertEquals(committedOffsets.size(), group.allOffsets().size()); - committedOffsets.forEach((tp, offset) -> { - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); - }); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata group = onLoadedFuture.get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + assertSame(group, groupInCache); + + assertEquals(groupId, group.groupId()); + assertEquals(Empty, group.currentState()); + assertEquals(committedOffsets.size(), group.allOffsets().size()); + committedOffsets.forEach((tp, offset) -> { + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); + }); } @Test public void testLoadGroupAndOffsetsFromDifferentSegments() throws Exception { - TopicPartition groupMetadataTopicPartition = groupTopicPartition; int generation = 293; String protocolType = "consumer"; String protocol = "range"; @@ -1138,54 +1127,61 @@ public void testLoadGroupAndOffsetsFromDifferentSegments() throws Exception { ByteBuffer segment2Buffer = newMemoryRecordsBuffer(segment2Records); byte[] key = groupMetadataKey(groupId); - runGroupMetadataManagerProducerTester("test-load-offsets-without-group", - (groupMetadataManager, producer) -> { - producer.newMessage() - .keyBytes(key) - .value(segment1Buffer) - .eventTime(time.milliseconds()) - .send(); - - producer.newMessage() - .keyBytes(key) - .value(segment2Buffer) - .eventTime(time.milliseconds()) - .send(); - - CompletableFuture onLoadedFuture = new CompletableFuture<>(); - groupMetadataManager.scheduleLoadGroupAndOffsets( - groupMetadataTopicPartition.partition(), - groupMetadata -> onLoadedFuture.complete(groupMetadata) - ).get(); - GroupMetadata group = onLoadedFuture.get(); - GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { - fail("Group was not loaded into the cache"); - return null; - }); - assertSame(group, groupInCache); - - assertEquals(groupId, group.groupId()); - assertEquals(Stable, group.currentState()); - - assertEquals("segment2 group record member should be elected", - segment2MemberId, group.leaderOrNull()); - assertEquals("segment2 group record member should be only member", - Lists.newArrayList(segment2MemberId), - group.allMembers().stream().collect(Collectors.toList())); - - // offsets of segment1 should be overridden by segment2 offsets of the same topic partitions - Map committedOffsets = new HashMap<>(); - committedOffsets.putAll(segment1Offsets); - committedOffsets.putAll(segment2Offsets); - assertEquals(committedOffsets.size(), group.allOffsets().size()); - committedOffsets.forEach((tp, offset) -> { - assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); - }); - }); + + Producer producer = groupMetadataManager.getOffsetsTopicProducer(groupPartitionId).get(); + producer.newMessage() + .keyBytes(key) + .value(segment1Buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + producer.newMessage() + .keyBytes(key) + .value(segment2Buffer) + .eventTime(Time.SYSTEM.milliseconds()) + .send(); + + CompletableFuture onLoadedFuture = new CompletableFuture<>(); + groupMetadataManager.scheduleLoadGroupAndOffsets( + groupPartitionId, + groupMetadata -> onLoadedFuture.complete(groupMetadata) + ).get(); + GroupMetadata group = onLoadedFuture.get(); + GroupMetadata groupInCache = groupMetadataManager.getGroup(groupId).orElseGet(() -> { + fail("Group was not loaded into the cache"); + return null; + }); + assertSame(group, groupInCache); + + assertEquals(groupId, group.groupId()); + assertEquals(Stable, group.currentState()); + + assertEquals(segment2MemberId, group.leaderOrNull(), + "segment2 group record member should be elected"); + assertEquals(Lists.newArrayList(segment2MemberId), + group.allMembers().stream().collect(Collectors.toList()), + "segment2 group record member should be only member"); + + // offsets of segment1 should be overridden by segment2 offsets of the same topic partitions + Map committedOffsets = new HashMap<>(); + committedOffsets.putAll(segment1Offsets); + committedOffsets.putAll(segment2Offsets); + assertEquals(committedOffsets.size(), group.allOffsets().size()); + committedOffsets.forEach((tp, offset) -> { + assertEquals(Optional.of(offset), group.offset(tp).map(OffsetAndMetadata::offset)); + }); + } @Test public void testAddGroup() { + groupMetadataManager = new GroupMetadataManager( + offsetConfig, + producer, + consumer, + scheduler, + new MockTime() + ); GroupMetadata group = new GroupMetadata("foo", Empty); assertEquals(group, groupMetadataManager.addGroup(group)); assertEquals(group, groupMetadataManager.addGroup( @@ -1195,760 +1191,741 @@ public void testAddGroup() { @Test public void testStoreEmptyGroup() throws Exception { - final String topicName = "test-store-empty-group"; - - runGroupMetadataManagerConsumerTester(topicName, (groupMetadataManager, consumer) -> { - int generation = 27; - String protocolType = "consumer"; - GroupMetadata group = GroupMetadata.loadGroup( - groupId, - Empty, - generation, - protocolType, - null, - null, - Collections.emptyList() - ); - groupMetadataManager.addGroup(group); - - Errors errors = groupMetadataManager.storeGroup(group, Collections.emptyMap()).get(); - assertEquals(Errors.NONE, errors); - - Message message = consumer.receive(); - assertTrue(message.getEventTime() > 0L); - assertTrue(message.hasKey()); - byte[] key = message.getKeyBytes(); - BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); - assertTrue(groupKey instanceof GroupMetadataKey); - GroupMetadataKey groupMetadataKey = (GroupMetadataKey) groupKey; - assertEquals(groupId, groupMetadataKey.key()); - - ByteBuffer value = message.getValue(); - MemoryRecords memRecords = MemoryRecords.readableRecords(value); - AtomicBoolean verified = new AtomicBoolean(false); - memRecords.batches().forEach(batch -> { - for (Record record : batch) { - assertFalse(verified.get()); - BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); - assertTrue(bk instanceof GroupMetadataKey); - GroupMetadataKey gmk = (GroupMetadataKey) bk; - assertEquals(groupId, gmk.key()); - - GroupMetadata gm = GroupMetadataConstants.readGroupMessageValue( - groupId, record.value() - ); - assertTrue(gm.is(Empty)); - assertEquals(generation, gm.generationId()); - assertEquals(Optional.of(protocolType), gm.protocolType()); - verified.set(true); - } - }); - assertTrue(verified.get()); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) + .topic(groupMetadataManager.getTopicPartitionName()) + .subscriptionName("test-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + int generation = 27; + String protocolType = "consumer"; + GroupMetadata group = GroupMetadata.loadGroup( + groupId, + Empty, + generation, + protocolType, + null, + null, + Collections.emptyList() + ); + groupMetadataManager.addGroup(group); + + Errors errors = groupMetadataManager.storeGroup(group, Collections.emptyMap()).get(); + assertEquals(Errors.NONE, errors); + + Message message = consumer.receive(); + while (message.getValue().array().length == 0) { + // bypass above place holder message. + message = consumer.receive(); + } + assertTrue(message.getEventTime() > 0L); + assertTrue(message.hasKey()); + byte[] key = message.getKeyBytes(); + BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); + assertTrue(groupKey instanceof GroupMetadataKey); + GroupMetadataKey groupMetadataKey = (GroupMetadataKey) groupKey; + assertEquals(groupId, groupMetadataKey.key()); + + ByteBuffer value = message.getValue(); + MemoryRecords memRecords = MemoryRecords.readableRecords(value); + AtomicBoolean verified = new AtomicBoolean(false); + memRecords.batches().forEach(batch -> { + for (Record record : batch) { + assertFalse(verified.get()); + BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); + assertTrue(bk instanceof GroupMetadataKey); + GroupMetadataKey gmk = (GroupMetadataKey) bk; + assertEquals(groupId, gmk.key()); + + GroupMetadata gm = GroupMetadataConstants.readGroupMessageValue( + groupId, record.value() + ); + assertTrue(gm.is(Empty)); + assertEquals(generation, gm.generationId()); + assertEquals(Optional.of(protocolType), gm.protocolType()); + verified.set(true); + } }); + assertTrue(verified.get()); + } @Test public void testStoreEmptySimpleGroup() throws Exception { - final String topicName = "test-store-empty-simple-group"; - - runGroupMetadataManagerConsumerTester(topicName, (groupMetadataManager, consumer) -> { - - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); - - Errors errors = groupMetadataManager.storeGroup(group, Collections.emptyMap()).get(); - assertEquals(Errors.NONE, errors); - - Message message = consumer.receive(); - assertTrue(message.getEventTime() > 0L); - assertTrue(message.hasKey()); - byte[] key = message.getKeyBytes(); - - BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); - assertTrue(groupKey instanceof GroupMetadataKey); - GroupMetadataKey groupMetadataKey = (GroupMetadataKey) groupKey; - assertEquals(groupId, groupMetadataKey.key()); - - ByteBuffer value = message.getValue(); - MemoryRecords memRecords = MemoryRecords.readableRecords(value); - AtomicBoolean verified = new AtomicBoolean(false); - memRecords.batches().forEach(batch -> { - for (Record record : batch) { - assertFalse(verified.get()); - BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); - assertTrue(bk instanceof GroupMetadataKey); - GroupMetadataKey gmk = (GroupMetadataKey) bk; - assertEquals(groupId, gmk.key()); - - GroupMetadata gm = GroupMetadataConstants.readGroupMessageValue( - groupId, record.value() - ); - assertTrue(gm.is(Empty)); - assertEquals(0, gm.generationId()); - assertEquals(Optional.empty(), gm.protocolType()); - verified.set(true); - } - }); - assertTrue(verified.get()); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) + .topic(groupMetadataManager.getTopicPartitionName()) + .subscriptionName("test-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + GroupMetadata group = new GroupMetadata(groupId, Empty); + groupMetadataManager.addGroup(group); + + Errors errors = groupMetadataManager.storeGroup(group, Collections.emptyMap()).get(); + assertEquals(Errors.NONE, errors); + + Message message = consumer.receive(); + while (message.getValue().array().length == 0) { + // bypass above place holder message. + message = consumer.receive(); + } + assertTrue(message.getEventTime() > 0L); + assertTrue(message.hasKey()); + byte[] key = message.getKeyBytes(); + + BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); + assertTrue(groupKey instanceof GroupMetadataKey); + GroupMetadataKey groupMetadataKey = (GroupMetadataKey) groupKey; + assertEquals(groupId, groupMetadataKey.key()); + + ByteBuffer value = message.getValue(); + MemoryRecords memRecords = MemoryRecords.readableRecords(value); + AtomicBoolean verified = new AtomicBoolean(false); + memRecords.batches().forEach(batch -> { + for (Record record : batch) { + assertFalse(verified.get()); + BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); + assertTrue(bk instanceof GroupMetadataKey); + GroupMetadataKey gmk = (GroupMetadataKey) bk; + assertEquals(groupId, gmk.key()); + + GroupMetadata gm = GroupMetadataConstants.readGroupMessageValue( + groupId, record.value() + ); + assertTrue(gm.is(Empty)); + assertEquals(0, gm.generationId()); + assertEquals(Optional.empty(), gm.protocolType()); + verified.set(true); + } }); + assertTrue(verified.get()); } @Test public void testStoreNoneEmptyGroup() throws Exception { - final String topicName = "test-store-non-empty-group"; - - runGroupMetadataManagerConsumerTester(topicName, (groupMetadataManager, consumer) -> { - String memberId = "memberId"; - String clientId = "clientId"; - String clientHost = "localhost"; - - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); - - Map protocols = new HashMap<>(); - protocols.put("protocol", new byte[0]); - MemberMetadata member = new MemberMetadata( - memberId, - groupId, - clientId, - clientHost, - rebalanceTimeout, - sessionTimeout, - protocolType, - protocols - ); - CompletableFuture joinFuture = new CompletableFuture<>(); - member.awaitingJoinCallback(joinFuture); - group.add(member); - group.transitionTo(GroupState.PreparingRebalance); - group.initNextGeneration(); - - Map assignments = new HashMap<>(); - assignments.put(memberId, new byte[0]); - Errors errors = groupMetadataManager.storeGroup(group, assignments).get(); - assertEquals(Errors.NONE, errors); - - Message message = consumer.receive(); - assertTrue(message.getEventTime() > 0L); - assertTrue(message.hasKey()); - byte[] key = message.getKeyBytes(); - BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); - assertTrue(groupKey instanceof GroupMetadataKey); - GroupMetadataKey groupMetadataKey = (GroupMetadataKey) groupKey; - assertEquals(groupId, groupMetadataKey.key()); - - ByteBuffer value = message.getValue(); - MemoryRecords memRecords = MemoryRecords.readableRecords(value); - AtomicBoolean verified = new AtomicBoolean(false); - memRecords.batches().forEach(batch -> { - for (Record record : batch) { - assertFalse(verified.get()); - BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); - assertTrue(bk instanceof GroupMetadataKey); - GroupMetadataKey gmk = (GroupMetadataKey) bk; - assertEquals(groupId, gmk.key()); - - GroupMetadata gm = GroupMetadataConstants.readGroupMessageValue( - groupId, record.value() - ); - assertEquals(Stable, gm.currentState()); - assertEquals(1, gm.generationId()); - assertEquals(Optional.of(protocolType), gm.protocolType()); - assertEquals("protocol", gm.protocolOrNull()); - assertTrue(gm.has(memberId)); - verified.set(true); - } - }); - assertTrue(verified.get()); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) + .topic(groupMetadataManager.getTopicPartitionName()) + .subscriptionName("test-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + String memberId = "memberId"; + String clientId = "clientId"; + String clientHost = "localhost"; + + GroupMetadata group = new GroupMetadata(groupId, Empty); + groupMetadataManager.addGroup(group); + + Map protocols = new HashMap<>(); + protocols.put("protocol", new byte[0]); + MemberMetadata member = new MemberMetadata( + memberId, + groupId, + clientId, + clientHost, + rebalanceTimeout, + sessionTimeout, + protocolType, + protocols + ); + CompletableFuture joinFuture = new CompletableFuture<>(); + member.awaitingJoinCallback(joinFuture); + group.add(member); + group.transitionTo(GroupState.PreparingRebalance); + group.initNextGeneration(); + + Map assignments = new HashMap<>(); + assignments.put(memberId, new byte[0]); + Errors errors = groupMetadataManager.storeGroup(group, assignments).get(); + assertEquals(Errors.NONE, errors); + + Message message = consumer.receive(); + while (message.getValue().array().length == 0) { + // bypass above place holder message. + message = consumer.receive(); + } + assertTrue(message.getEventTime() > 0L); + assertTrue(message.hasKey()); + byte[] key = message.getKeyBytes(); + BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); + assertTrue(groupKey instanceof GroupMetadataKey); + GroupMetadataKey groupMetadataKey = (GroupMetadataKey) groupKey; + assertEquals(groupId, groupMetadataKey.key()); + + ByteBuffer value = message.getValue(); + MemoryRecords memRecords = MemoryRecords.readableRecords(value); + AtomicBoolean verified = new AtomicBoolean(false); + memRecords.batches().forEach(batch -> { + for (Record record : batch) { + assertFalse(verified.get()); + BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); + assertTrue(bk instanceof GroupMetadataKey); + GroupMetadataKey gmk = (GroupMetadataKey) bk; + assertEquals(groupId, gmk.key()); + + GroupMetadata gm = GroupMetadataConstants.readGroupMessageValue( + groupId, record.value() + ); + assertEquals(Stable, gm.currentState()); + assertEquals(1, gm.generationId()); + assertEquals(Optional.of(protocolType), gm.protocolType()); + assertEquals("protocol", gm.protocolOrNull()); + assertTrue(gm.has(memberId)); + verified.set(true); + } }); + assertTrue(verified.get()); } @Test public void testCommitOffset() throws Exception { - runGroupMetadataManagerConsumerTester("test-commit-offset", (groupMetadataManager, consumer) -> { - String memberId = ""; - TopicPartition topicPartition = new TopicPartition("foo", 0); - groupMetadataManager.addPartitionOwnership(groupPartitionId); - long offset = 37L; - - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); - - Map offsets = ImmutableMap.builder() - .put(topicPartition, OffsetAndMetadata.apply(offset)) - .build(); - - Map commitErrors = groupMetadataManager.storeOffsets( - group, memberId, offsets - ).get(); - - assertTrue(group.hasOffsets()); - assertFalse(commitErrors.isEmpty()); - Errors maybeError = commitErrors.get(topicPartition); - assertEquals(Errors.NONE, maybeError); - assertTrue(group.hasOffsets()); - - Map cachedOffsets = groupMetadataManager.getOffsets( - groupId, - Optional.of(Lists.newArrayList(topicPartition)) - ); - PartitionData maybePartitionResponse = cachedOffsets.get(topicPartition); - assertNotNull(maybePartitionResponse); - - assertEquals(Errors.NONE, maybePartitionResponse.error); - assertEquals(offset, maybePartitionResponse.offset); - - Message message = consumer.receive(); - assertTrue(message.getEventTime() > 0L); - assertTrue(message.hasKey()); - byte[] key = message.getKeyBytes(); - BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); - assertTrue(groupKey instanceof OffsetKey); - - ByteBuffer value = message.getValue(); - MemoryRecords memRecords = MemoryRecords.readableRecords(value); - AtomicBoolean verified = new AtomicBoolean(false); - memRecords.batches().forEach(batch -> { - for (Record record : batch) { - assertFalse(verified.get()); - BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); - assertTrue(bk instanceof OffsetKey); - OffsetKey ok = (OffsetKey) bk; - GroupTopicPartition gtp = ok.key(); - assertEquals(groupId, gtp.group()); - assertEquals(topicPartition, gtp.topicPartition()); - - OffsetAndMetadata gm = GroupMetadataConstants.readOffsetMessageValue( - record.value() - ); - assertEquals(offset, gm.offset()); - verified.set(true); - } - }); - assertTrue(verified.get()); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) + .topic(groupMetadataManager.getTopicPartitionName()) + .subscriptionName("test-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + String memberId = "fakeMemberId"; + TopicPartition topicPartition = new TopicPartition("foo", 0); + groupMetadataManager.addPartitionOwnership(groupPartitionId); + long offset = 37L; + + GroupMetadata group = new GroupMetadata(groupId, Empty); + groupMetadataManager.addGroup(group); + + Map offsets = ImmutableMap.builder() + .put(topicPartition, OffsetAndMetadata.apply(offset)) + .build(); + + Map commitErrors = groupMetadataManager.storeOffsets( + group, memberId, offsets + ).get(); + + assertTrue(group.hasOffsets()); + assertFalse(commitErrors.isEmpty()); + Errors maybeError = commitErrors.get(topicPartition); + assertEquals(Errors.NONE, maybeError); + assertTrue(group.hasOffsets()); + + Map cachedOffsets = groupMetadataManager.getOffsets( + groupId, + Optional.of(Lists.newArrayList(topicPartition)) + ); + PartitionData maybePartitionResponse = cachedOffsets.get(topicPartition); + assertNotNull(maybePartitionResponse); + + assertEquals(Errors.NONE, maybePartitionResponse.error); + assertEquals(offset, maybePartitionResponse.offset); + + Message message = consumer.receive(); + while (message.getValue().array().length == 0) { + // bypass above place holder message. + message = consumer.receive(); + } + assertTrue(message.getEventTime() > 0L); + assertTrue(message.hasKey()); + byte[] key = message.getKeyBytes(); + BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); + assertTrue(groupKey instanceof OffsetKey); + + ByteBuffer value = message.getValue(); + MemoryRecords memRecords = MemoryRecords.readableRecords(value); + AtomicBoolean verified = new AtomicBoolean(false); + memRecords.batches().forEach(batch -> { + for (Record record : batch) { + assertFalse(verified.get()); + BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); + assertTrue(bk instanceof OffsetKey); + OffsetKey ok = (OffsetKey) bk; + GroupTopicPartition gtp = ok.key(); + assertEquals(groupId, gtp.group()); + assertEquals(topicPartition, gtp.topicPartition()); + + OffsetAndMetadata gm = GroupMetadataConstants.readOffsetMessageValue( + record.value() + ); + assertEquals(offset, gm.offset()); + verified.set(true); + } }); + assertTrue(verified.get()); } @Test public void testTransactionalCommitOffsetCommitted() throws Exception { - runGroupMetadataManagerConsumerTester("test-commit-offset", (groupMetadataManager, consumer) -> { - String memberId = ""; - TopicPartition topicPartition = new TopicPartition("foo", 0); - long offset = 37L; - long producerId = 232L; - short producerEpoch = 0; - - groupMetadataManager.addPartitionOwnership(groupPartitionId); - - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); - - Map offsets = ImmutableMap.builder() - .put(topicPartition, OffsetAndMetadata.apply(offset)) - .build(); - - CompletableFuture writeOffsetMessageFuture = new CompletableFuture<>(); - AtomicReference> realWriteFutureRef = new AtomicReference<>(); - doAnswer(invocationOnMock -> { - CompletableFuture realWriteFuture = - (CompletableFuture) invocationOnMock.callRealMethod(); - realWriteFutureRef.set(realWriteFuture); - return writeOffsetMessageFuture; - }).when(groupMetadataManager).storeOffsetMessage( - any(byte[].class), any(ByteBuffer.class), anyLong() - ); - - CompletableFuture> storeFuture = groupMetadataManager.storeOffsets( - group, memberId, offsets, producerId, producerEpoch - ); - - assertTrue(group.hasOffsets()); - assertTrue(group.allOffsets().isEmpty()); - - // complete the write message - writeOffsetMessageFuture.complete(realWriteFutureRef.get().get()); - Map commitErrors = storeFuture.get(); - - assertFalse(commitErrors.isEmpty()); - Errors maybeError = commitErrors.get(topicPartition); - assertEquals(Errors.NONE, maybeError); - assertTrue(group.hasOffsets()); - assertTrue(group.allOffsets().isEmpty()); - - group.completePendingTxnOffsetCommit(producerId, true); - assertTrue(group.hasOffsets()); - assertFalse(group.allOffsets().isEmpty()); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) + .topic(groupMetadataManager.getTopicPartitionName()) + .subscriptionName("test-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + String memberId = ""; + TopicPartition topicPartition = new TopicPartition("foo", 0); + long offset = 37L; + long producerId = 232L; + short producerEpoch = 0; + + GroupMetadataManager spyGroupManager = spy(groupMetadataManager); + spyGroupManager.addPartitionOwnership(groupPartitionId); + + GroupMetadata group = new GroupMetadata(groupId, Empty); + spyGroupManager.addGroup(group); + + Map offsets = ImmutableMap.builder() + .put(topicPartition, OffsetAndMetadata.apply(offset)) + .build(); + + CompletableFuture writeOffsetMessageFuture = new CompletableFuture<>(); + AtomicReference> realWriteFutureRef = new AtomicReference<>(); + doAnswer(invocationOnMock -> { + CompletableFuture realWriteFuture = + (CompletableFuture) invocationOnMock.callRealMethod(); + realWriteFutureRef.set(realWriteFuture); + return writeOffsetMessageFuture; + }).when(spyGroupManager).storeOffsetMessage( + any(String.class), any(byte[].class), any(ByteBuffer.class), anyLong() + ); + + CompletableFuture> storeFuture = spyGroupManager.storeOffsets( + group, memberId, offsets, producerId, producerEpoch + ); + + assertTrue(group.hasOffsets()); + assertTrue(group.allOffsets().isEmpty()); + + // complete the write message + writeOffsetMessageFuture.complete(realWriteFutureRef.get().get()); + Map commitErrors = storeFuture.get(); + + assertFalse(commitErrors.isEmpty()); + Errors maybeError = commitErrors.get(topicPartition); + assertEquals(Errors.NONE, maybeError); + assertTrue(group.hasOffsets()); + assertTrue(group.allOffsets().isEmpty()); + + group.completePendingTxnOffsetCommit(producerId, true); + assertTrue(group.hasOffsets()); + assertFalse(group.allOffsets().isEmpty()); + + assertEquals( + Optional.of(OffsetAndMetadata.apply(offset)), + group.offset(topicPartition) + ); - assertEquals( - Optional.of(OffsetAndMetadata.apply(offset)), - group.offset(topicPartition) - ); - }); } @Test public void testTransactionalCommitOffsetAppendFailure() throws Exception { - runGroupMetadataManagerConsumerTester("test-commit-offset", (groupMetadataManager, consumer) -> { - String memberId = ""; - TopicPartition topicPartition = new TopicPartition("foo", 0); - long offset = 37L; - long producerId = 232L; - short producerEpoch = 0; - - groupMetadataManager.addPartitionOwnership(groupPartitionId); - - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); - - Map offsets = ImmutableMap.builder() - .put(topicPartition, OffsetAndMetadata.apply(offset)) - .build(); - - CompletableFuture writeOffsetMessageFuture = new CompletableFuture<>(); - AtomicReference> realWriteFutureRef = new AtomicReference<>(); - doAnswer(invocationOnMock -> { - CompletableFuture realWriteFuture = - (CompletableFuture) invocationOnMock.callRealMethod(); - realWriteFutureRef.set(realWriteFuture); - return writeOffsetMessageFuture; - }).when(groupMetadataManager).storeOffsetMessage( - any(byte[].class), any(ByteBuffer.class), anyLong() - ); - - CompletableFuture> storeFuture = groupMetadataManager.storeOffsets( - group, memberId, offsets, producerId, producerEpoch - ); - - assertTrue(group.hasOffsets()); - assertTrue(group.allOffsets().isEmpty()); - - // complete the write message - writeOffsetMessageFuture.completeExceptionally( - new Exception("Not enought replicas") - ); - Map commitErrors = storeFuture.get(); - - assertFalse(commitErrors.isEmpty()); - Errors maybeError = commitErrors.get(topicPartition); - assertEquals(Errors.UNKNOWN_SERVER_ERROR, maybeError); - assertFalse(group.hasOffsets()); - assertTrue(group.allOffsets().isEmpty()); - - group.completePendingTxnOffsetCommit(producerId, false); - assertFalse(group.hasOffsets()); - assertTrue(group.allOffsets().isEmpty()); - }); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) + .topic(groupMetadataManager.getTopicPartitionName()) + .subscriptionName("test-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + String memberId = ""; + TopicPartition topicPartition = new TopicPartition("foo", 0); + long offset = 37L; + long producerId = 232L; + short producerEpoch = 0; + + GroupMetadataManager spyGroupManager = spy(groupMetadataManager); + spyGroupManager.addPartitionOwnership(groupPartitionId); + + GroupMetadata group = new GroupMetadata(groupId, Empty); + spyGroupManager.addGroup(group); + + Map offsets = ImmutableMap.builder() + .put(topicPartition, OffsetAndMetadata.apply(offset)) + .build(); + + CompletableFuture writeOffsetMessageFuture = new CompletableFuture<>(); + AtomicReference> realWriteFutureRef = new AtomicReference<>(); + doAnswer(invocationOnMock -> { + CompletableFuture realWriteFuture = + (CompletableFuture) invocationOnMock.callRealMethod(); + realWriteFutureRef.set(realWriteFuture); + return writeOffsetMessageFuture; + }).when(spyGroupManager).storeOffsetMessage( + any(String.class), any(byte[].class), any(ByteBuffer.class), anyLong() + ); + + CompletableFuture> storeFuture = spyGroupManager.storeOffsets( + group, memberId, offsets, producerId, producerEpoch + ); + + assertTrue(group.hasOffsets()); + assertTrue(group.allOffsets().isEmpty()); + + // complete the write message + writeOffsetMessageFuture.completeExceptionally( + new Exception("Not enought replicas") + ); + Map commitErrors = storeFuture.get(); + + assertFalse(commitErrors.isEmpty()); + Errors maybeError = commitErrors.get(topicPartition); + assertEquals(Errors.UNKNOWN_SERVER_ERROR, maybeError); + assertFalse(group.hasOffsets()); + assertTrue(group.allOffsets().isEmpty()); + + group.completePendingTxnOffsetCommit(producerId, false); + assertFalse(group.hasOffsets()); + assertTrue(group.allOffsets().isEmpty()); + } @Test public void testTransactionalCommitOffsetAborted() throws Exception { - runGroupMetadataManagerConsumerTester("test-commit-offset", (groupMetadataManager, consumer) -> { - String memberId = ""; - TopicPartition topicPartition = new TopicPartition("foo", 0); - long offset = 37L; - long producerId = 232L; - short producerEpoch = 0; - - groupMetadataManager.addPartitionOwnership(groupPartitionId); - - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); - - Map offsets = ImmutableMap.builder() - .put(topicPartition, OffsetAndMetadata.apply(offset)) - .build(); - - CompletableFuture writeOffsetMessageFuture = new CompletableFuture<>(); - AtomicReference> realWriteFutureRef = new AtomicReference<>(); - doAnswer(invocationOnMock -> { - CompletableFuture realWriteFuture = - (CompletableFuture) invocationOnMock.callRealMethod(); - realWriteFutureRef.set(realWriteFuture); - return writeOffsetMessageFuture; - }).when(groupMetadataManager).storeOffsetMessage( - any(byte[].class), any(ByteBuffer.class), anyLong() - ); - - CompletableFuture> storeFuture = groupMetadataManager.storeOffsets( - group, memberId, offsets, producerId, producerEpoch - ); - - assertTrue(group.hasOffsets()); - assertTrue(group.allOffsets().isEmpty()); - - // complete the write message - writeOffsetMessageFuture.complete(realWriteFutureRef.get().get()); - Map commitErrors = storeFuture.get(); - - assertFalse(commitErrors.isEmpty()); - Errors maybeError = commitErrors.get(topicPartition); - assertEquals(Errors.NONE, maybeError); - assertTrue(group.hasOffsets()); - assertTrue(group.allOffsets().isEmpty()); - - group.completePendingTxnOffsetCommit(producerId, false); - assertFalse(group.hasOffsets()); - assertTrue(group.allOffsets().isEmpty()); - }); - } + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) + .topic(groupMetadataManager.getTopicPartitionName()) + .subscriptionName("test-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + String memberId = ""; + TopicPartition topicPartition = new TopicPartition("foo", 0); + long offset = 37L; + long producerId = 232L; + short producerEpoch = 0; - @Test - public void testExpiredOffset() throws Exception { - runGroupMetadataManagerConsumerTester("test-commit-offset", (groupMetadataManager, consumer) -> { - String memberId = ""; - TopicPartition topicPartition1 = new TopicPartition("foo", 0); - TopicPartition topicPartition2 = new TopicPartition("foo", 1); - groupMetadataManager.addPartitionOwnership(groupPartitionId); - long offset = 37L; - - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); - - long startMs = time.milliseconds(); - Map offsets = ImmutableMap.builder() - .put(topicPartition1, OffsetAndMetadata.apply( - offset, "", startMs, startMs + 1)) - .put(topicPartition2, OffsetAndMetadata.apply( - offset, "", startMs, startMs + 3)) - .build(); - - Map commitErrors = groupMetadataManager.storeOffsets( - group, memberId, offsets - ).get(); - assertTrue(group.hasOffsets()); - - assertFalse(commitErrors.isEmpty()); - Errors maybeError = commitErrors.get(topicPartition1); - assertEquals(Errors.NONE, maybeError); - - // expire only one of the offsets - time.sleep(2); - - groupMetadataManager.cleanupGroupMetadata(); - - assertEquals(Optional.of(group), groupMetadataManager.getGroup(groupId)); - assertEquals(Optional.empty(), group.offset(topicPartition1)); - assertEquals(Optional.of(offset), group.offset(topicPartition2).map(OffsetAndMetadata::offset)); - - Map cachedOffsets = groupMetadataManager.getOffsets( - groupId, - Optional.of(Lists.newArrayList( - topicPartition1, - topicPartition2 - )) - ); - assertEquals( - OffsetFetchResponse.INVALID_OFFSET, - cachedOffsets.get(topicPartition1).offset); - assertEquals( - offset, - cachedOffsets.get(topicPartition2).offset); - }); - } + GroupMetadataManager spyGroupManager = spy(groupMetadataManager); + spyGroupManager.addPartitionOwnership(groupPartitionId); - @Test - public void testGroupMetadataRemoval() throws Exception { - runGroupMetadataManagerConsumerTester("test-commit-offset", (groupMetadataManager, consumer) -> { - TopicPartition topicPartition1 = new TopicPartition("foo", 0); - TopicPartition topicPartition2 = new TopicPartition("foo", 1); - - groupMetadataManager.addPartitionOwnership(groupPartitionId); - - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); - group.generationId(5); - - groupMetadataManager.cleanupGroupMetadata().get(); - - Message message = consumer.receive(); - assertTrue(message.getEventTime() > 0L); - assertTrue(message.hasKey()); - byte[] key = message.getKeyBytes(); - - BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); - assertTrue(groupKey instanceof GroupMetadataKey); - GroupMetadataKey groupMetadataKey = (GroupMetadataKey) groupKey; - assertEquals(groupId, groupMetadataKey.key()); - - ByteBuffer value = message.getValue(); - MemoryRecords memRecords = MemoryRecords.readableRecords(value); - AtomicBoolean verified = new AtomicBoolean(false); - memRecords.batches().forEach(batch -> { - assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, batch.magic()); - assertEquals(TimestampType.CREATE_TIME, batch.timestampType()); - for (Record record : batch) { - assertFalse(verified.get()); - assertTrue(record.hasKey()); - assertFalse(record.hasValue()); - assertTrue(record.timestamp() > 0); - BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); - assertTrue(bk instanceof GroupMetadataKey); - GroupMetadataKey gmk = (GroupMetadataKey) bk; - assertEquals(groupId, gmk.key()); - verified.set(true); - } - }); - assertTrue(verified.get()); - assertEquals(Optional.empty(), groupMetadataManager.getGroup(groupId)); - Map cachedOffsets = groupMetadataManager.getOffsets( - groupId, - Optional.of(Lists.newArrayList( - topicPartition1, - topicPartition2 - )) - ); - assertEquals( - OffsetFetchResponse.INVALID_OFFSET, - cachedOffsets.get(topicPartition1).offset); - assertEquals( - OffsetFetchResponse.INVALID_OFFSET, - cachedOffsets.get(topicPartition2).offset); - }); - } + GroupMetadata group = new GroupMetadata(groupId, Empty); + spyGroupManager.addGroup(group); - @Test - public void testExpireGroupWithOffsetsOnly() throws Exception { - runGroupMetadataManagerConsumerTester("test-commit-offset", (groupMetadataManager, consumer) -> { - // verify that the group is removed properly, but no tombstone is written if - // this is a group which is only using kafka for offset storage + Map offsets = ImmutableMap.builder() + .put(topicPartition, OffsetAndMetadata.apply(offset)) + .build(); - String memberId = ""; - TopicPartition topicPartition1 = new TopicPartition("foo", 0); - TopicPartition topicPartition2 = new TopicPartition("foo", 1); - long offset = 37; + CompletableFuture writeOffsetMessageFuture = new CompletableFuture<>(); + AtomicReference> realWriteFutureRef = new AtomicReference<>(); + doAnswer(invocationOnMock -> { + CompletableFuture realWriteFuture = + (CompletableFuture) invocationOnMock.callRealMethod(); + realWriteFutureRef.set(realWriteFuture); + return writeOffsetMessageFuture; + }).when(spyGroupManager).storeOffsetMessage( + any(String.class), any(byte[].class), any(ByteBuffer.class), anyLong() + ); - groupMetadataManager.addPartitionOwnership(groupPartitionId); + CompletableFuture> storeFuture = spyGroupManager.storeOffsets( + group, memberId, offsets, producerId, producerEpoch + ); - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); + assertTrue(group.hasOffsets()); + assertTrue(group.allOffsets().isEmpty()); - long startMs = time.milliseconds(); - Map offsets = ImmutableMap.builder() - .put(topicPartition1, OffsetAndMetadata.apply(offset, "", startMs, startMs + 1)) - .put(topicPartition2, OffsetAndMetadata.apply(offset, "", startMs, startMs + 3)) - .build(); + // complete the write message + writeOffsetMessageFuture.complete(realWriteFutureRef.get().get()); + Map commitErrors = storeFuture.get(); - Map commitErrors = - groupMetadataManager.storeOffsets(group, memberId, offsets).get(); - assertTrue(group.hasOffsets()); + assertFalse(commitErrors.isEmpty()); + Errors maybeError = commitErrors.get(topicPartition); + assertEquals(Errors.NONE, maybeError); + assertTrue(group.hasOffsets()); + assertTrue(group.allOffsets().isEmpty()); - assertFalse(commitErrors.isEmpty()); - assertEquals( - Errors.NONE, - commitErrors.get(topicPartition1) - ); - - // expire all of the offsets - time.sleep(4); - - groupMetadataManager.cleanupGroupMetadata().get(); - - // skip `storeOffsets` op - consumer.receive(); - - Message message = consumer.receive(); - assertTrue(message.getEventTime() > 0L); - assertTrue(message.hasKey()); - byte[] key = message.getKeyBytes(); - - BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); - assertTrue(groupKey instanceof GroupMetadataKey); - GroupMetadataKey gmk = (GroupMetadataKey) groupKey; - assertEquals(groupId, gmk.key()); - - ByteBuffer value = message.getValue(); - MemoryRecords memRecords = MemoryRecords.readableRecords(value); - AtomicInteger verified = new AtomicInteger(2); - memRecords.batches().forEach(batch -> { - assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, batch.magic()); - assertEquals(TimestampType.CREATE_TIME, batch.timestampType()); - for (Record record : batch) { - verified.decrementAndGet(); - assertTrue(record.hasKey()); - assertFalse(record.hasValue()); - assertTrue(record.timestamp() > 0); - BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); - assertTrue(bk instanceof OffsetKey); - OffsetKey ok = (OffsetKey) bk; - assertEquals(groupId, ok.key().group()); - assertEquals("foo", ok.key().topicPartition().topic()); - } - }); - assertEquals(0, verified.get()); - assertEquals(Optional.empty(), groupMetadataManager.getGroup(groupId)); - Map cachedOffsets = groupMetadataManager.getOffsets( - groupId, - Optional.of(Lists.newArrayList( - topicPartition1, - topicPartition2 - )) - ); - assertEquals( - OffsetFetchResponse.INVALID_OFFSET, - cachedOffsets.get(topicPartition1).offset); - assertEquals( - OffsetFetchResponse.INVALID_OFFSET, - cachedOffsets.get(topicPartition2).offset); - }); + group.completePendingTxnOffsetCommit(producerId, false); + assertFalse(group.hasOffsets()); + assertTrue(group.allOffsets().isEmpty()); } @Test - public void testExpireOffsetsWithActiveGroup() throws Exception { - runGroupMetadataManagerConsumerTester("test-commit-offset", (groupMetadataManager, consumer) -> { - String memberId = "memberId"; - String clientId = "clientId"; - String clientHost = "localhost"; - TopicPartition topicPartition1 = new TopicPartition("foo", 0); - TopicPartition topicPartition2 = new TopicPartition("foo", 1); - long offset = 37; - - groupMetadataManager.addPartitionOwnership(groupPartitionId); - - GroupMetadata group = new GroupMetadata(groupId, Empty); - groupMetadataManager.addGroup(group); - - MemberMetadata member = new MemberMetadata( - memberId, groupId, clientId, clientHost, - rebalanceTimeout, - sessionTimeout, - protocolType, - ImmutableMap.builder() - .put("protocol", new byte[0]) - .build() - ); - CompletableFuture memberJoinFuture = new CompletableFuture<>(); - member.awaitingJoinCallback(memberJoinFuture); - group.add(member); - group.transitionTo(PreparingRebalance); - group.initNextGeneration(); - - long startMs = time.milliseconds(); - Map offsets = ImmutableMap.builder() - .put(topicPartition1, OffsetAndMetadata.apply(offset, "", startMs, startMs + 1)) - .put(topicPartition2, OffsetAndMetadata.apply(offset, "", startMs, startMs + 3)) - .build(); - - Map commitErrors = - groupMetadataManager.storeOffsets(group, memberId, offsets).get(); - assertTrue(group.hasOffsets()); - - assertFalse(commitErrors.isEmpty()); - assertEquals( - Errors.NONE, - commitErrors.get(topicPartition1) - ); + public void testExpiredOffset() throws Exception { + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) + .topic(groupMetadataManager.getTopicPartitionName()) + .subscriptionName("test-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + String memberId = "fakeMemberId"; + TopicPartition topicPartition1 = new TopicPartition("foo", 0); + TopicPartition topicPartition2 = new TopicPartition("foo", 1); + groupMetadataManager.addPartitionOwnership(groupPartitionId); + long offset = 37L; - // expire all of the offsets - time.sleep(4); + GroupMetadata group = new GroupMetadata(groupId, Empty); + groupMetadataManager.addGroup(group); - groupMetadataManager.cleanupGroupMetadata().get(); + // 1 offset expire soon. the other expire after 5 + Map offsets = ImmutableMap.builder() + .put(topicPartition1, OffsetAndMetadata.apply( + offset, "", Time.SYSTEM.milliseconds(), Time.SYSTEM.milliseconds() + 1)) + .put(topicPartition2, OffsetAndMetadata.apply( + offset, "", Time.SYSTEM.milliseconds(), Time.SYSTEM.milliseconds() + 5000)) + .build(); - // group should still be there, but the offsets should be gone - assertEquals( - Optional.of(group), - groupMetadataManager.getGroup(groupId) - ); - assertEquals( - Optional.empty(), - group.offset(topicPartition1) - ); - assertEquals( - Optional.empty(), - group.offset(topicPartition2) - ); - - Map cachedOffsets = groupMetadataManager.getOffsets( - groupId, - Optional.of(Lists.newArrayList( - topicPartition1, - topicPartition2 - )) - ); - assertEquals( - OffsetFetchResponse.INVALID_OFFSET, - cachedOffsets.get(topicPartition1).offset); - assertEquals( - OffsetFetchResponse.INVALID_OFFSET, - cachedOffsets.get(topicPartition2).offset); - }); - } + Map commitErrors = groupMetadataManager.storeOffsets( + group, memberId, offsets + ).get(); + assertTrue(group.hasOffsets()); + + assertFalse(commitErrors.isEmpty()); + Errors maybeError = commitErrors.get(topicPartition1); + assertEquals(Errors.NONE, maybeError); - /** - * A group metadata manager test runner. - */ - @FunctionalInterface - public interface GroupMetadataManagerProducerTester { + groupMetadataManager.cleanupGroupMetadata(); - void test(GroupMetadataManager groupMetadataManager, - Producer consumer) throws Exception; + assertEquals(Optional.of(group), groupMetadataManager.getGroup(groupId)); + assertEquals(Optional.empty(), group.offset(topicPartition1)); + assertEquals(Optional.of(offset), group.offset(topicPartition2).map(OffsetAndMetadata::offset)); + + Map cachedOffsets = groupMetadataManager.getOffsets( + groupId, + Optional.of(Lists.newArrayList( + topicPartition1, + topicPartition2 + )) + ); + assertEquals( + OffsetFetchResponse.INVALID_OFFSET, + cachedOffsets.get(topicPartition1).offset); + assertEquals( + offset, + cachedOffsets.get(topicPartition2).offset); } - /** - * A group metadata manager test runner. - */ - @FunctionalInterface - public interface GroupMetadataManagerConsumerTester { + @Test + public void testGroupMetadataRemoval() throws Exception { + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) + .topic(groupMetadataManager.getTopicPartitionName()) + .subscriptionName("test-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + TopicPartition topicPartition1 = new TopicPartition("foo", 0); + TopicPartition topicPartition2 = new TopicPartition("foo", 1); + + groupMetadataManager.addPartitionOwnership(groupPartitionId); - void test(GroupMetadataManager groupMetadataManager, - Consumer consumer) throws Exception; + GroupMetadata group = new GroupMetadata(groupId, Empty); + groupMetadataManager.addGroup(group); + group.generationId(5); + + groupMetadataManager.cleanupGroupMetadata().get(); + + Message message = consumer.receive(); + while (message.getValue().array().length == 0) { + // bypass above place holder message. + message = consumer.receive(); + } + assertTrue(message.getEventTime() > 0L); + assertTrue(message.hasKey()); + byte[] key = message.getKeyBytes(); + + BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); + assertTrue(groupKey instanceof GroupMetadataKey); + GroupMetadataKey groupMetadataKey = (GroupMetadataKey) groupKey; + assertEquals(groupId, groupMetadataKey.key()); + + ByteBuffer value = message.getValue(); + MemoryRecords memRecords = MemoryRecords.readableRecords(value); + AtomicBoolean verified = new AtomicBoolean(false); + memRecords.batches().forEach(batch -> { + assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, batch.magic()); + assertEquals(TimestampType.CREATE_TIME, batch.timestampType()); + for (Record record : batch) { + assertFalse(verified.get()); + assertTrue(record.hasKey()); + assertFalse(record.hasValue()); + assertTrue(record.timestamp() > 0); + BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); + assertTrue(bk instanceof GroupMetadataKey); + GroupMetadataKey gmk = (GroupMetadataKey) bk; + assertEquals(groupId, gmk.key()); + verified.set(true); + } + }); + assertTrue(verified.get()); + assertEquals(Optional.empty(), groupMetadataManager.getGroup(groupId)); + Map cachedOffsets = groupMetadataManager.getOffsets( + groupId, + Optional.of(Lists.newArrayList( + topicPartition1, + topicPartition2 + )) + ); + assertEquals( + OffsetFetchResponse.INVALID_OFFSET, + cachedOffsets.get(topicPartition1).offset); + assertEquals( + OffsetFetchResponse.INVALID_OFFSET, + cachedOffsets.get(topicPartition2).offset); } - void runGroupMetadataManagerProducerTester(final String topicName, - GroupMetadataManagerProducerTester tester) throws Exception { - @Cleanup - Producer producer = pulsarClient.newProducer(Schema.BYTEBUFFER) - .topic(topicName) - .create(); + @Test + public void testExpireGroupWithOffsetsOnly() throws Exception { @Cleanup Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) - .topic(topicName) + .topic(groupMetadataManager.getTopicPartitionName()) .subscriptionName("test-sub") .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); - @Cleanup - Reader reader = pulsarClient.newReader(Schema.BYTEBUFFER) - .topic(topicName) - .startMessageId(MessageId.earliest) - .create(); - groupMetadataManager = spy(new GroupMetadataManager( - 1, - offsetConfig, - producer, - reader, - scheduler, - time - )); - tester.test(groupMetadataManager, producer); + // verify that the group is removed properly, but no tombstone is written if + // this is a group which is only using kafka for offset storage + + String memberId = ""; + TopicPartition topicPartition1 = new TopicPartition("foo", 0); + TopicPartition topicPartition2 = new TopicPartition("foo", 1); + long offset = 37; + + groupMetadataManager.addPartitionOwnership(groupPartitionId); + + GroupMetadata group = new GroupMetadata(groupId, Empty); + groupMetadataManager.addGroup(group); + + long startMs = Time.SYSTEM.milliseconds(); + Map offsets = ImmutableMap.builder() + .put(topicPartition1, OffsetAndMetadata.apply(offset, "", startMs, startMs + 1)) + .put(topicPartition2, OffsetAndMetadata.apply(offset, "", startMs, startMs + 3)) + .build(); + + Map commitErrors = + groupMetadataManager.storeOffsets(group, memberId, offsets).get(); + assertTrue(group.hasOffsets()); + + assertFalse(commitErrors.isEmpty()); + assertEquals( + Errors.NONE, + commitErrors.get(topicPartition1) + ); + + groupMetadataManager.cleanupGroupMetadata().get(); + + Message message = consumer.receive(); + // skip `storeOffsets` op, bypass place holder message. + while (!message.hasKey() + || GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(message.getKeyBytes())) instanceof OffsetKey) { + message = consumer.receive(); + } + + assertTrue(message.getEventTime() > 0L); + assertTrue(message.hasKey()); + byte[] key = message.getKeyBytes(); + + BaseKey groupKey = GroupMetadataConstants.readMessageKey(ByteBuffer.wrap(key)); + assertTrue(groupKey instanceof GroupMetadataKey); + GroupMetadataKey gmk = (GroupMetadataKey) groupKey; + assertEquals(groupId, gmk.key()); + + ByteBuffer value = message.getValue(); + MemoryRecords memRecords = MemoryRecords.readableRecords(value); + AtomicInteger verified = new AtomicInteger(2); + memRecords.batches().forEach(batch -> { + assertEquals(RecordBatch.CURRENT_MAGIC_VALUE, batch.magic()); + assertEquals(TimestampType.CREATE_TIME, batch.timestampType()); + for (Record record : batch) { + verified.decrementAndGet(); + assertTrue(record.hasKey()); + assertFalse(record.hasValue()); + assertTrue(record.timestamp() > 0); + BaseKey bk = GroupMetadataConstants.readMessageKey(record.key()); + assertTrue(bk instanceof OffsetKey); + OffsetKey ok = (OffsetKey) bk; + assertEquals(groupId, ok.key().group()); + assertEquals("foo", ok.key().topicPartition().topic()); + } + }); + assertEquals(0, verified.get()); + assertEquals(Optional.empty(), groupMetadataManager.getGroup(groupId)); + Map cachedOffsets = groupMetadataManager.getOffsets( + groupId, + Optional.of(Lists.newArrayList( + topicPartition1, + topicPartition2 + )) + ); + assertEquals( + OffsetFetchResponse.INVALID_OFFSET, + cachedOffsets.get(topicPartition1).offset); + assertEquals( + OffsetFetchResponse.INVALID_OFFSET, + cachedOffsets.get(topicPartition2).offset); } - void runGroupMetadataManagerConsumerTester(final String topicName, - GroupMetadataManagerConsumerTester tester) throws Exception { - @Cleanup - Producer producer = pulsarClient.newProducer(Schema.BYTEBUFFER) - .topic(topicName) - .create(); + @Test + public void testExpireOffsetsWithActiveGroup() throws Exception { @Cleanup Consumer consumer = pulsarClient.newConsumer(Schema.BYTEBUFFER) - .topic(topicName) + .topic(groupMetadataManager.getTopicPartitionName()) .subscriptionName("test-sub") .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); - @Cleanup - Reader reader = pulsarClient.newReader(Schema.BYTEBUFFER) - .topic(topicName) - .startMessageId(MessageId.earliest) - .create(); - groupMetadataManager = spy(new GroupMetadataManager( - 1, - offsetConfig, - producer, - reader, - scheduler, - time - )); - tester.test(groupMetadataManager, consumer); - } + String memberId = "memberId"; + String clientId = "clientId"; + String clientHost = "localhost"; + TopicPartition topicPartition1 = new TopicPartition("foo", 0); + TopicPartition topicPartition2 = new TopicPartition("foo", 1); + long offset = 37; + + groupMetadataManager.addPartitionOwnership(groupPartitionId); + + GroupMetadata group = new GroupMetadata(groupId, Empty); + groupMetadataManager.addGroup(group); + + MemberMetadata member = new MemberMetadata( + memberId, groupId, clientId, clientHost, + rebalanceTimeout, + sessionTimeout, + protocolType, + ImmutableMap.builder() + .put("protocol", new byte[0]) + .build() + ); + CompletableFuture memberJoinFuture = new CompletableFuture<>(); + member.awaitingJoinCallback(memberJoinFuture); + group.add(member); + group.transitionTo(PreparingRebalance); + group.initNextGeneration(); + + long startMs = Time.SYSTEM.milliseconds(); + Map offsets = ImmutableMap.builder() + .put(topicPartition1, OffsetAndMetadata.apply(offset, "", startMs, startMs + 1)) + .put(topicPartition2, OffsetAndMetadata.apply(offset, "", startMs, startMs + 3)) + .build(); + + Map commitErrors = + groupMetadataManager.storeOffsets(group, memberId, offsets).get(); + assertTrue(group.hasOffsets()); + assertFalse(commitErrors.isEmpty()); + assertEquals( + Errors.NONE, + commitErrors.get(topicPartition1) + ); + + groupMetadataManager.cleanupGroupMetadata().get(); + + // group should still be there, but the offsets should be gone + assertEquals( + Optional.of(group), + groupMetadataManager.getGroup(groupId) + ); + assertEquals( + Optional.empty(), + group.offset(topicPartition1) + ); + assertEquals( + Optional.empty(), + group.offset(topicPartition2) + ); + + Map cachedOffsets = groupMetadataManager.getOffsets( + groupId, + Optional.of(Lists.newArrayList( + topicPartition1, + topicPartition2 + )) + ); + assertEquals( + OffsetFetchResponse.INVALID_OFFSET, + cachedOffsets.get(topicPartition1).offset); + assertEquals( + OffsetFetchResponse.INVALID_OFFSET, + cachedOffsets.get(topicPartition2).offset); + } }