Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-30 Thread via GitHub


dajac merged PR #14638:
URL: https://github.com/apache/kafka/pull/14638


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


kirktrue commented on code in PR #14638:
URL: https://github.com/apache/kafka/pull/14638#discussion_r1372381836


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java:
##
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link ConsumerRebalanceListener}
+ * interface. When consumer group partition assignment changes, these methods 
are invoked. This class wraps those
+ * callback calls with some logging, optional {@link Sensor} updates, etc.
+ */
+class ConsumerRebalanceListenerInvoker {
+
+private final Logger log;
+private final SubscriptionState subscriptions;
+private final Time time;
+private final ConsumerCoordinatorMetrics coordinatorMetrics;
+
+ConsumerRebalanceListenerInvoker(LogContext logContext,
+ SubscriptionState subscriptions,
+ Time time,
+ ConsumerCoordinatorMetrics 
coordinatorMetrics) {
+this.log = logContext.logger(getClass());
+this.subscriptions = subscriptions;
+this.time = time;
+this.coordinatorMetrics = coordinatorMetrics;
+}
+
+Exception invokePartitionsAssigned(final SortedSet 
assignedPartitions) {
+log.info("Adding newly assigned partitions: {}", 
Utils.join(assignedPartitions, ", "));
+
+Optional listener = 
subscriptions.rebalanceListener();
+
+if (listener.isPresent()) {
+try {
+final long startMs = time.milliseconds();
+listener.get().onPartitionsAssigned(assignedPartitions);
+
coordinatorMetrics.assignCallbackSensor.record(time.milliseconds() - startMs);
+} catch (WakeupException | InterruptException e) {
+throw e;
+} catch (Exception e) {
+log.error("User provided listener {} failed on invocation of 
onPartitionsAssigned for partitions {}",
+listener.getClass().getName(), assignedPartitions, e);
+return e;
+}
+}
+
+return null;
+}
+
+Exception invokePartitionsRevoked(final SortedSet 
revokedPartitions) {
+log.info("Revoke previously assigned partitions {}", 
Utils.join(revokedPartitions, ", "));
+Set revokePausedPartitions = 
subscriptions.pausedPartitions();
+revokePausedPartitions.retainAll(revokedPartitions);
+if (!revokePausedPartitions.isEmpty())
+log.info("The pause flag in partitions [{}] will be removed due to 
revocation.", Utils.join(revokePausedPartitions, ", "));
+
+Optional listener = 
subscriptions.rebalanceListener();
+
+if (listener.isPresent()) {
+try {
+final long startMs = time.milliseconds();
+listener.get().onPartitionsRevoked(revokedPartitions);
+
coordinatorMetrics.revokeCallbackSensor.record(time.milliseconds() - startMs);
+} catch (WakeupException | InterruptException e) {
+throw e;
+} catch (Exception e) {

Review Comment:
   I can't speak to all the different use cases, but this is what is done in 
the existing `ConsumerCoordinator`, so this was done to keep the existing 
behavior equivalent.



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -1081,4 +1026,74 @@ boolean updateAssignmentMetadataIfNeeded(Timer timer) {
 // logic
  

Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


philipnee commented on PR #14638:
URL: https://github.com/apache/kafka/pull/14638#issuecomment-1779768168

   Hi @kirktrue - I made a pass and left some comments. Let me know if anything 
is unclear!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


philipnee commented on code in PR #14638:
URL: https://github.com/apache/kafka/pull/14638#discussion_r1372118635


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java:
##
@@ -162,13 +162,13 @@ else if (this.subscriptionType != type)
 throw new IllegalStateException(SUBSCRIPTION_EXCEPTION_MESSAGE);
 }
 
-public synchronized boolean subscribe(Set topics, 
ConsumerRebalanceListener listener) {
+public synchronized boolean subscribe(Set topics, 
Optional listener) {

Review Comment:
   Actually - is listener actually optional here? It seems like if the listener 
is null, it should complain? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


philipnee commented on code in PR #14638:
URL: https://github.com/apache/kafka/pull/14638#discussion_r1372117884


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java:
##
@@ -285,10 +285,8 @@ public synchronized void 
assignFromSubscribed(Collection assignm
 this.assignment.set(assignedPartitionStates);
 }
 
-private void registerRebalanceListener(ConsumerRebalanceListener listener) 
{
-if (listener == null)
-throw new IllegalArgumentException("RebalanceListener cannot be 
null");
-this.rebalanceListener = listener;
+private void registerRebalanceListener(Optional 
listener) {
+this.rebalanceListener = Objects.requireNonNull(listener, 
"RebalanceListener cannot be null");

Review Comment:
   Does Objects.requireNonNull checks the lister.get() or the listener? Since 
you are passing in an Optional.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


philipnee commented on code in PR #14638:
URL: https://github.com/apache/kafka/pull/14638#discussion_r1372115691


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java:
##
@@ -162,13 +162,13 @@ else if (this.subscriptionType != type)
 throw new IllegalStateException(SUBSCRIPTION_EXCEPTION_MESSAGE);
 }
 
-public synchronized boolean subscribe(Set topics, 
ConsumerRebalanceListener listener) {
+public synchronized boolean subscribe(Set topics, 
Optional listener) {

Review Comment:
   Another suggestion is to not change the interface, then we do:
   
   registerRebalanceListener(Optional.ofNullabe(listener)) - this prevents us 
to do a requireNonNull check.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


philipnee commented on code in PR #14638:
URL: https://github.com/apache/kafka/pull/14638#discussion_r1372112998


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -1081,4 +1026,74 @@ boolean updateAssignmentMetadataIfNeeded(Timer timer) {
 // logic
 return updateFetchPositions(timer);
 }
+
+@Override
+public void subscribe(Collection topics) {
+subscribeInternal(topics, Optional.empty());
+}
+
+@Override
+public void subscribe(Collection topics, ConsumerRebalanceListener 
listener) {
+if (listener == null)
+throw new IllegalArgumentException("RebalanceListener cannot be 
null");
+
+subscribeInternal(topics, Optional.of(listener));
+}
+
+@Override
+public void subscribe(Pattern pattern) {
+subscribeInternal(pattern, Optional.empty());
+}
+
+@Override
+public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) 
{
+if (listener == null)
+throw new IllegalArgumentException("RebalanceListener cannot be 
null");
+
+subscribeInternal(pattern, Optional.of(listener));
+}
+
+private void subscribeInternal(Pattern pattern, 
Optional listener) {
+maybeThrowInvalidGroupIdException();
+if (pattern == null || pattern.toString().isEmpty())
+throw new IllegalArgumentException("Topic pattern to subscribe to 
cannot be " + (pattern == null ?
+"null" : "empty"));
+
+throwIfNoAssignorsConfigured();
+log.info("Subscribed to pattern: '{}'", pattern);
+subscriptions.subscribe(pattern, listener);
+updatePatternSubscription(metadata.fetch());
+metadata.requestUpdateForNewTopics();
+}
+
+private void subscribeInternal(Collection topics, 
Optional listener) {
+maybeThrowInvalidGroupIdException();
+if (topics == null)
+throw new IllegalArgumentException("Topic collection to subscribe 
to cannot be null");
+if (topics.isEmpty()) {
+// treat subscribing to empty topic list as the same as 
unsubscribing
+unsubscribe();
+} else {
+for (String topic : topics) {
+if (isBlank(topic))
+throw new IllegalArgumentException("Topic collection to 
subscribe to cannot contain null or empty topic");
+}
+
+throwIfNoAssignorsConfigured();
+
+// Clear the buffered data which are not a part of newly assigned 
topics
+final Set currentTopicPartitions = new HashSet<>();
+
+for (TopicPartition tp : subscriptions.assignedPartitions()) {
+if (topics.contains(tp.topic()))

Review Comment:
   ah, seems like this is done for my comment above



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


philipnee commented on code in PR #14638:
URL: https://github.com/apache/kafka/pull/14638#discussion_r1372104074


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java:
##
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link ConsumerRebalanceListener}
+ * interface. When consumer group partition assignment changes, these methods 
are invoked. This class wraps those
+ * callback calls with some logging, optional {@link Sensor} updates, etc.
+ */
+class ConsumerRebalanceListenerInvoker {
+
+private final Logger log;
+private final SubscriptionState subscriptions;
+private final Time time;
+private final ConsumerCoordinatorMetrics coordinatorMetrics;
+
+ConsumerRebalanceListenerInvoker(LogContext logContext,
+ SubscriptionState subscriptions,
+ Time time,
+ ConsumerCoordinatorMetrics 
coordinatorMetrics) {
+this.log = logContext.logger(getClass());
+this.subscriptions = subscriptions;
+this.time = time;
+this.coordinatorMetrics = coordinatorMetrics;
+}
+
+Exception invokePartitionsAssigned(final SortedSet 
assignedPartitions) {
+log.info("Adding newly assigned partitions: {}", 
Utils.join(assignedPartitions, ", "));
+
+Optional listener = 
subscriptions.rebalanceListener();
+
+if (listener.isPresent()) {
+try {
+final long startMs = time.milliseconds();
+listener.get().onPartitionsAssigned(assignedPartitions);
+
coordinatorMetrics.assignCallbackSensor.record(time.milliseconds() - startMs);
+} catch (WakeupException | InterruptException e) {
+throw e;
+} catch (Exception e) {
+log.error("User provided listener {} failed on invocation of 
onPartitionsAssigned for partitions {}",
+listener.getClass().getName(), assignedPartitions, e);
+return e;
+}
+}
+
+return null;
+}
+
+Exception invokePartitionsRevoked(final SortedSet 
revokedPartitions) {
+log.info("Revoke previously assigned partitions {}", 
Utils.join(revokedPartitions, ", "));
+Set revokePausedPartitions = 
subscriptions.pausedPartitions();
+revokePausedPartitions.retainAll(revokedPartitions);
+if (!revokePausedPartitions.isEmpty())
+log.info("The pause flag in partitions [{}] will be removed due to 
revocation.", Utils.join(revokePausedPartitions, ", "));
+
+Optional listener = 
subscriptions.rebalanceListener();
+
+if (listener.isPresent()) {
+try {
+final long startMs = time.milliseconds();
+listener.get().onPartitionsRevoked(revokedPartitions);
+
coordinatorMetrics.revokeCallbackSensor.record(time.milliseconds() - startMs);
+} catch (WakeupException | InterruptException e) {
+throw e;
+} catch (Exception e) {

Review Comment:
   we should definitely return all KafkaExceptions; are there java exceptions 
that we want to throw back immediately to the user though? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this 

Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


kirktrue commented on PR #14638:
URL: https://github.com/apache/kafka/pull/14638#issuecomment-1779709409

   @dajac @philipnee Can you add the `ctr` and `KIP-848` tags, please? Thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


kirktrue opened a new pull request, #14638:
URL: https://github.com/apache/kafka/pull/14638

   Straightforward refactoring to extract an inner class and methods related to 
`ConsumerRebalanceListener` for reuse in the KIP-848 implementation of the 
consumer group protocol. Also using `Optional` to explicitly mark when a 
`ConsumerRebalanceListener` is in use or not, allowing us to make some 
(forthcoming) optimizations when there is no listener to invoke.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


kirktrue commented on PR #14565:
URL: https://github.com/apache/kafka/pull/14565#issuecomment-1779699978

   This is now outdated. Please see #14638.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-25 Thread via GitHub


kirktrue closed pull request #14565: KAFKA-15628: Refactor 
ConsumerRebalanceListener invocation for reuse
URL: https://github.com/apache/kafka/pull/14565


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-24 Thread via GitHub


kirktrue commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1370618922


##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -976,15 +1053,14 @@ public void subscribe(Collection topics) {
  * Group rebalances only take place during an active call to {@link 
#poll(Duration)}.
  *
  * @param pattern Pattern to subscribe to
- * @param listener Non-null listener instance to get notifications on 
partition assignment/revocation for the
- * subscribed topics
+ * @param listener {@link Optional} listener instance to get notifications 
on partition assignment/revocation
+ * for the subscribed topics
  * @throws IllegalArgumentException If pattern or listener is null
  * @throws IllegalStateException If {@code subscribe()} is called 
previously with topics, or assign is called
  *   previously (without a subsequent call to 
{@link #unsubscribe()}), or if not
  *   configured at-least one partition 
assignment strategy
  */
-@Override
-public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) 
{
+private void subscribe(Pattern pattern, 
Optional listener) {
 maybeThrowInvalidGroupIdException();
 if (pattern == null || pattern.toString().equals(""))

Review Comment:
   Agreed. I'm loath to change it at this point due to some weird side effect 
of which I'm unaware.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-24 Thread via GitHub


kirktrue commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1370615968


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java:
##
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link ConsumerRebalanceListener}
+ * interface. When consumer group partition assignment changes, these methods 
are invoked. This class wraps those
+ * callback calls with some logging, optional {@link Sensor} updates, etc.
+ */
+class ConsumerRebalanceListenerInvoker {
+
+private final Logger log;
+private final SubscriptionState subscriptions;
+private final Time time;
+private final ConsumerCoordinatorMetrics coordinatorMetrics;
+
+ConsumerRebalanceListenerInvoker(LogContext logContext,
+ SubscriptionState subscriptions,
+ Time time,
+ ConsumerCoordinatorMetrics 
coordinatorMetrics) {
+this.log = logContext.logger(getClass());
+this.subscriptions = subscriptions;
+this.time = time;
+this.coordinatorMetrics = coordinatorMetrics;
+}
+
+Exception invokePartitionsAssigned(final SortedSet 
assignedPartitions) {
+log.info("Adding newly assigned partitions: {}", 
Utils.join(assignedPartitions, ", "));
+
+Optional listener = 
subscriptions.rebalanceListener();
+
+if (listener.isPresent()) {
+try {
+final long startMs = time.milliseconds();
+listener.get().onPartitionsAssigned(assignedPartitions);
+
coordinatorMetrics.assignCallbackSensor.record(time.milliseconds() - startMs);
+} catch (WakeupException | InterruptException e) {
+throw e;
+} catch (Exception e) {
+log.error("User provided listener {} failed on invocation of 
onPartitionsAssigned for partitions {}",
+listener.getClass().getName(), assignedPartitions, e);
+return e;

Review Comment:
   The caller does that. The `Exception` type is what is used by the existing 
code, so 路‍♂️ 



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java:
##
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.metrics.Sensor;
+import 

Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-24 Thread via GitHub


philipnee commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1370505973


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java:
##
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link ConsumerRebalanceListener}
+ * interface. When consumer group partition assignment changes, these methods 
are invoked. This class wraps those
+ * callback calls with some logging, optional {@link Sensor} updates, etc.
+ */
+class ConsumerRebalanceListenerInvoker {
+
+private final Logger log;
+private final SubscriptionState subscriptions;
+private final Time time;
+private final ConsumerCoordinatorMetrics coordinatorMetrics;
+
+ConsumerRebalanceListenerInvoker(LogContext logContext,
+ SubscriptionState subscriptions,
+ Time time,
+ ConsumerCoordinatorMetrics 
coordinatorMetrics) {
+this.log = logContext.logger(getClass());
+this.subscriptions = subscriptions;
+this.time = time;
+this.coordinatorMetrics = coordinatorMetrics;
+}
+
+Exception invokePartitionsAssigned(final SortedSet 
assignedPartitions) {
+log.info("Adding newly assigned partitions: {}", 
Utils.join(assignedPartitions, ", "));
+
+Optional listener = 
subscriptions.rebalanceListener();
+
+if (listener.isPresent()) {
+try {
+final long startMs = time.milliseconds();
+listener.get().onPartitionsAssigned(assignedPartitions);
+
coordinatorMetrics.assignCallbackSensor.record(time.milliseconds() - startMs);
+} catch (WakeupException | InterruptException e) {
+throw e;
+} catch (Exception e) {
+log.error("User provided listener {} failed on invocation of 
onPartitionsAssigned for partitions {}",
+listener.getClass().getName(), assignedPartitions, e);
+return e;

Review Comment:
   Do we want to wrap it in KafkaException?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-24 Thread via GitHub


philipnee commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1370492706


##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -976,15 +1053,14 @@ public void subscribe(Collection topics) {
  * Group rebalances only take place during an active call to {@link 
#poll(Duration)}.
  *
  * @param pattern Pattern to subscribe to
- * @param listener Non-null listener instance to get notifications on 
partition assignment/revocation for the
- * subscribed topics
+ * @param listener {@link Optional} listener instance to get notifications 
on partition assignment/revocation
+ * for the subscribed topics
  * @throws IllegalArgumentException If pattern or listener is null
  * @throws IllegalStateException If {@code subscribe()} is called 
previously with topics, or assign is called
  *   previously (without a subsequent call to 
{@link #unsubscribe()}), or if not
  *   configured at-least one partition 
assignment strategy
  */
-@Override
-public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) 
{
+private void subscribe(Pattern pattern, 
Optional listener) {
 maybeThrowInvalidGroupIdException();
 if (pattern == null || pattern.toString().equals(""))

Review Comment:
   it is a bit weird to use toString() instead of `pattern()` even though both 
implementation return the same field, `pattern`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-24 Thread via GitHub


dajac commented on PR #14565:
URL: https://github.com/apache/kafka/pull/14565#issuecomment-1777355576

   One of the build was red. I just triggered another one.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-23 Thread via GitHub


kirktrue commented on PR #14565:
URL: https://github.com/apache/kafka/pull/14565#issuecomment-1775972229

   @dajac This is ready for another review. Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-23 Thread via GitHub


kirktrue commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1369002478


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java:
##
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.common.metrics.Measurable;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Meter;
+import org.apache.kafka.common.metrics.stats.WindowedCount;
+
+class ConsumerCoordinatorMetrics {
+
+final Sensor commitSensor;
+final Sensor revokeCallbackSensor;
+final Sensor assignCallbackSensor;
+final Sensor loseCallbackSensor;
+
+ConsumerCoordinatorMetrics(SubscriptionState subscriptions,
+   Metrics metrics,
+   String metricGrpPrefix) {
+String metricGrpName = metricGrpPrefix + "-coordinator-metrics";
+
+this.commitSensor = metrics.sensor("commit-latency");
+this.commitSensor.add(metrics.metricName("commit-latency-avg",
+metricGrpName,
+"The average time taken for a commit request"), new Avg());
+this.commitSensor.add(metrics.metricName("commit-latency-max",
+metricGrpName,
+"The max time taken for a commit request"), new Max());
+this.commitSensor.add(new Meter(new WindowedCount(),
+metrics.metricName("commit-rate", metricGrpName,
+"The number of commit calls per second"),
+metrics.metricName("commit-total", metricGrpName,
+"The total number of commit calls")));
+this.revokeCallbackSensor = 
metrics.sensor("partition-revoked-latency");

Review Comment:
   Agreed. Good catch. Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-23 Thread via GitHub


kirktrue commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1368991272


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java:
##
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link ConsumerRebalanceListener}
+ * interface. When consumer group partition assignment changes, these methods 
are invoked. This class wraps those
+ * callback calls with some logging, optional {@link Sensor} updates, etc.
+ */
+class ConsumerRebalanceListenerInvoker {
+
+private final Logger log;
+private final SubscriptionState subscriptions;
+private final Time time;
+private final ConsumerCoordinatorMetrics coordinatorMetrics;
+
+ConsumerRebalanceListenerInvoker(LogContext logContext,
+ SubscriptionState subscriptions,
+ Time time,
+ ConsumerCoordinatorMetrics 
coordinatorMetrics) {
+this.log = logContext.logger(getClass());
+this.subscriptions = subscriptions;
+this.time = time;
+this.coordinatorMetrics = coordinatorMetrics;
+}
+
+Exception invokePartitionsAssigned(final SortedSet 
assignedPartitions) {
+log.info("Adding newly assigned partitions: {}", 
Utils.join(assignedPartitions, ", "));
+
+Optional listener = 
subscriptions.rebalanceListener();
+
+if (listener.isPresent()) {
+try {
+final long startMs = time.milliseconds();
+listener.get().onPartitionsAssigned(assignedPartitions);
+
coordinatorMetrics.assignCallbackSensor.record(time.milliseconds() - startMs);
+} catch (WakeupException | InterruptException e) {
+throw e;
+} catch (Exception e) {
+log.error("User provided listener {} failed on invocation of 
onPartitionsAssigned for partitions {}",
+listener.getClass().getName(), assignedPartitions, e);
+return e;
+}
+}
+
+return null;
+}
+
+Exception invokePartitionsRevoked(final SortedSet 
revokedPartitions) {
+log.info("Revoke previously assigned partitions {}", 
Utils.join(revokedPartitions, ", "));
+Set revokePausedPartitions = 
subscriptions.pausedPartitions();
+revokePausedPartitions.retainAll(revokedPartitions);
+if (!revokePausedPartitions.isEmpty())
+log.info("The pause flag in partitions [{}] will be removed due to 
revocation.", org.apache.kafka.common.utils.Utils.join(revokePausedPartitions, 
", "));

Review Comment:
   Yes. Fixed.



##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java:
##
@@ -143,7 +143,7 @@
 public class FetcherTest {
 private static final double EPSILON = 0.0001;
 
-private ConsumerRebalanceListener listener = new 
NoOpConsumerRebalanceListener();
+private Optional listener = Optional.empty();

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-23 Thread via GitHub


kirktrue commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1368990080


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##
@@ -227,6 +224,12 @@ public ConsumerCoordinator(GroupRebalanceConfig 
rebalanceConfig,
 protocol = null;
 }
 
+this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
+logContext,
+subscriptions,
+time,
+coordinatorMetrics

Review Comment:
   Sorry about that. Not sure why my IDE wants so many spaces 路‍♂️ 



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##
@@ -227,6 +224,12 @@ public ConsumerCoordinator(GroupRebalanceConfig 
rebalanceConfig,
 protocol = null;
 }
 
+this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
+logContext,
+subscriptions,
+time,
+coordinatorMetrics

Review Comment:
   Sorry about that. Not sure why my IDE wants so many spaces 路‍♂️ 



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##
@@ -227,6 +224,12 @@ public ConsumerCoordinator(GroupRebalanceConfig 
rebalanceConfig,
 protocol = null;
 }
 
+this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
+logContext,
+subscriptions,
+time,
+coordinatorMetrics

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-23 Thread via GitHub


kirktrue commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1368989675


##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -961,7 +960,7 @@ public void subscribe(Collection topics, 
ConsumerRebalanceListener liste
  */
 @Override
 public void subscribe(Collection topics) {
-subscribe(topics, new NoOpConsumerRebalanceListener());
+subscribe(topics, null);

Review Comment:
   You're right. I reworked it a bit so that we still keep the same semantics.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-23 Thread via GitHub


dajac commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1368427893


##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -961,7 +960,7 @@ public void subscribe(Collection topics, 
ConsumerRebalanceListener liste
  */
 @Override
 public void subscribe(Collection topics) {
-subscribe(topics, new NoOpConsumerRebalanceListener());
+subscribe(topics, null);

Review Comment:
   The javadoc of `subscribe` says:
   
   ```
* @param listener Non-null listener instance to get notifications on 
partition assignment/revocation for the
* subscribed topics
* @throws IllegalArgumentException If topics is null or contains null 
or empty elements, or if listener is null
   ```
   
   With these changes, we break the contract. We should try to maintain it. 
What do you think?



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##
@@ -227,6 +224,12 @@ public ConsumerCoordinator(GroupRebalanceConfig 
rebalanceConfig,
 protocol = null;
 }
 
+this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
+logContext,
+subscriptions,
+time,
+coordinatorMetrics

Review Comment:
   nit: I think that we indent arguments with 4 spaces in this case.



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java:
##
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link ConsumerRebalanceListener}
+ * interface. When consumer group partition assignment changes, these methods 
are invoked. This class wraps those
+ * callback calls with some logging, optional {@link Sensor} updates, etc.
+ */
+class ConsumerRebalanceListenerInvoker {
+
+private final Logger log;
+private final SubscriptionState subscriptions;
+private final Time time;
+private final ConsumerCoordinatorMetrics coordinatorMetrics;
+
+ConsumerRebalanceListenerInvoker(LogContext logContext,
+ SubscriptionState subscriptions,
+ Time time,
+ ConsumerCoordinatorMetrics 
coordinatorMetrics) {
+this.log = logContext.logger(getClass());
+this.subscriptions = subscriptions;
+this.time = time;
+this.coordinatorMetrics = coordinatorMetrics;
+}
+
+Exception invokePartitionsAssigned(final SortedSet 
assignedPartitions) {
+log.info("Adding newly assigned partitions: {}", 
Utils.join(assignedPartitions, ", "));
+
+Optional listener = 
subscriptions.rebalanceListener();
+
+if (listener.isPresent()) {
+try {
+final long startMs = time.milliseconds();
+listener.get().onPartitionsAssigned(assignedPartitions);
+
coordinatorMetrics.assignCallbackSensor.record(time.milliseconds() - startMs);
+} catch (WakeupException | InterruptException e) {
+throw e;
+} catch (Exception e) {
+log.error("User provided listener {} failed on invocation of 
onPartitionsAssigned for partitions {}",
+listener.getClass().getName(), assignedPartitions, e);
+return e;
+}
+}
+
+return null;
+}
+
+Exception 

Re: [PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-17 Thread via GitHub


kirktrue commented on PR #14565:
URL: https://github.com/apache/kafka/pull/14565#issuecomment-1767347909

   @dajac or @philipnee Can I get the KIP-848 tag on this, please?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[PR] KAFKA-15628: Refactor ConsumerRebalanceListener invocation for reuse [kafka]

2023-10-17 Thread via GitHub


kirktrue opened a new pull request, #14565:
URL: https://github.com/apache/kafka/pull/14565

   Straightforward refactoring to extract an inner class and methods related to 
`ConsumerRebalanceListener` for reuse in the KIP-848 implementation of the 
consumer group protocol.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org