From 435321ff7dff33a9f178c235b6fcd0a8d34efca9 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Mon, 5 Aug 2024 18:29:41 +0200 Subject: [PATCH 01/19] [KIP-848] integration tests passing - Mock handler implementation - Rename current consumer protocol from generic to classic - Mock handler with automatic or manual assignment - More consumer group metadata getters - Test helpers - Expedite next HB after FindCoordinator doing it with an exponential backoff to avoid tight loops - Configurable session timeout and HB interval - Fix mock handler ListOffsets response LeaderEpoch instead of CurrentLeaderEpoch - Integration tests passing with AK trunk - Improve documentation and KIP 848 specific mock tests - Add mock tests for unknown topic id in metadata request and partial reconciliation - Make test 0147 more reliable - Fix test 0106 after HB timeout change - Exclude test case with AK trunk - Rename rd_kafka_buf_write_tags to rd_kafka_buf_write_tags_empty - Trivup 0.12.5 can run a KafkaCluster directly with KRaft and AK trunk - Trivup 0.12.6 build with a specific commit --- tests/0016-client_swname.c | 5 + tests/0147-consumer_group_consumer_mock.c | 676 ++++++++++++++++++++++ tests/CMakeLists.txt | 1 + tests/test.c | 2 + tests/trivup/trivup-0.12.6.tar.gz | Bin 0 -> 32862 bytes win32/tests/tests.vcxproj | 1 + 6 files changed, 685 insertions(+) create mode 100644 tests/0147-consumer_group_consumer_mock.c create mode 100644 tests/trivup/trivup-0.12.6.tar.gz diff --git a/tests/0016-client_swname.c b/tests/0016-client_swname.c index 335925e328..170e213ed0 100644 --- a/tests/0016-client_swname.c +++ b/tests/0016-client_swname.c @@ -120,6 +120,11 @@ int main_0016_client_swname(int argc, char **argv) { const char *jmx_port; const char *reason = NULL; + if (test_broker_version > TEST_BRKVER(3, 6, 0, 0)) { + TEST_SKIP("FIXME: check this case with AK trunk\n"); + return 0; + } + /* If available, use the Kafka JmxTool to query software name * in broker JMX metrics */ if (!(broker = test_getenv("BROKER_ADDRESS_2", NULL))) diff --git a/tests/0147-consumer_group_consumer_mock.c b/tests/0147-consumer_group_consumer_mock.c new file mode 100644 index 0000000000..7af526646b --- /dev/null +++ b/tests/0147-consumer_group_consumer_mock.c @@ -0,0 +1,676 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2024, Confluent Inc. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * 1. Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +#include "test.h" + +#include "../src/rdkafka_proto.h" + +#include + + +/** + * @name Mock tests specific of the KIP-848 group consumer protocol + */ + + +static int allowed_error; +static int rebalance_cnt; +static rd_kafka_resp_err_t rebalance_exp_event; +static rd_bool_t rebalance_exp_lost = rd_false; + +/** + * @brief Decide what error_cb's will cause the test to fail. + */ +static int +error_is_fatal_cb(rd_kafka_t *rk, rd_kafka_resp_err_t err, const char *reason) { + if (err == allowed_error || + /* If transport errors are allowed then it is likely + * that we'll also see ALL_BROKERS_DOWN. */ + (allowed_error == RD_KAFKA_RESP_ERR__TRANSPORT && + err == RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN)) { + TEST_SAY("Ignoring allowed error: %s: %s\n", + rd_kafka_err2name(err), reason); + return 0; + } + return 1; +} + +/** + * @brief Rebalance callback saving number of calls and verifying expected + * event. + */ +static void rebalance_cb(rd_kafka_t *rk, + rd_kafka_resp_err_t err, + rd_kafka_topic_partition_list_t *parts, + void *opaque) { + + rebalance_cnt++; + TEST_SAY("Rebalance #%d: %s: %d partition(s)\n", rebalance_cnt, + rd_kafka_err2name(err), parts->cnt); + + TEST_ASSERT( + err == rebalance_exp_event, "Expected rebalance event %s, not %s", + rd_kafka_err2name(rebalance_exp_event), rd_kafka_err2name(err)); + + if (rebalance_exp_lost) { + TEST_ASSERT(rd_kafka_assignment_lost(rk), + "Expected partitions lost"); + TEST_SAY("Partitions were lost\n"); + } + + if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS) { + test_consumer_assign("assign", rk, parts); + } else { + test_consumer_unassign("unassign", rk); + } + + /* Make sure only one rebalance callback is served per poll() + * so that expect_rebalance() returns to the test logic on each + * rebalance. */ + rd_kafka_yield(rk); +} + +static rd_bool_t is_heartbeat_request(rd_kafka_mock_request_t *request, + void *opaque) { + return rd_kafka_mock_request_api_key(request) == + RD_KAFKAP_ConsumerGroupHeartbeat; +} + +/** + * @brief Wait at least \p num heartbeats + * have been received by the mock cluster + * plus \p confidence_interval has passed + * + * @return Number of heartbeats received. + */ +static int wait_all_heartbeats_done(rd_kafka_mock_cluster_t *mcluster, + int num, + int confidence_interval) { + return test_mock_wait_matching_requests( + mcluster, num, confidence_interval, is_heartbeat_request, NULL); +} + +static rd_kafka_t *create_consumer(const char *bootstraps, + const char *topic, + rd_bool_t with_rebalance_cb) { + rd_kafka_conf_t *conf; + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "bootstrap.servers", bootstraps); + test_conf_set(conf, "group.protocol", "consumer"); + test_conf_set(conf, "auto.offset.reset", "earliest"); + return test_create_consumer( + topic, with_rebalance_cb ? rebalance_cb : NULL, conf, NULL); +} + +/** + * @brief Test heartbeat behavior with fatal errors, + * ensuring: + * - a fatal error is received on poll and consumer close + * - no rebalance cb is called + * - no final leave group heartbeat is sent + * + * @param err The error code to test. + * @param variation See calling code. + */ +static void +do_test_consumer_group_heartbeat_fatal_error(rd_kafka_resp_err_t err, + int variation) { + rd_kafka_mock_cluster_t *mcluster; + const char *bootstraps; + rd_kafka_topic_partition_list_t *subscription; + rd_kafka_t *c; + rd_kafka_message_t *rkmessage; + int expected_heartbeats, found_heartbeats, expected_rebalance_cnt; + test_timing_t timing; + rebalance_cnt = 0; + rebalance_exp_lost = rd_false; + rebalance_exp_event = RD_KAFKA_RESP_ERR_NO_ERROR; + const char *topic = test_mk_topic_name(__FUNCTION__, 0); + + SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); + + mcluster = test_mock_cluster_new(1, &bootstraps); + rd_kafka_mock_set_default_heartbeat_interval(mcluster, 1000); + rd_kafka_mock_topic_create(mcluster, topic, 1, 1); + + TIMING_START(&timing, "consumer_group_heartbeat_fatal_error"); + + if (variation == 1) { + /* First HB returns assignment */ + rd_kafka_mock_broker_push_request_error_rtts( + mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, + RD_KAFKA_RESP_ERR_NO_ERROR, 0); + } + + rd_kafka_mock_broker_push_request_error_rtts( + mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, err, 0); + + c = create_consumer(bootstraps, topic, rd_true); + + /* Subscribe to the input topic */ + subscription = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subscription, topic, + /* The partition is ignored in + * rd_kafka_subscribe() */ + RD_KAFKA_PARTITION_UA); + + TEST_SAY("Subscribing to topic\n"); + rd_kafka_mock_start_request_tracking(mcluster); + TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription)); + rd_kafka_topic_partition_list_destroy(subscription); + + expected_heartbeats = 1; + if (variation == 1) + expected_heartbeats++; + + TEST_SAY("Awaiting first HBs\n"); + TEST_ASSERT((found_heartbeats = + wait_all_heartbeats_done(mcluster, expected_heartbeats, + 200)) == expected_heartbeats, + "Expected %d heartbeats, got %d", expected_heartbeats, + found_heartbeats); + + rd_kafka_mock_clear_requests(mcluster); + + expected_rebalance_cnt = 0; + if (variation == 1) { + expected_rebalance_cnt++; + rebalance_exp_event = RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS; + + /* Trigger rebalance cb */ + rkmessage = rd_kafka_consumer_poll(c, 500); + TEST_ASSERT(!rkmessage, "No message should be returned"); + } + + TEST_SAY("Consume from c, a fatal error is returned\n"); + rkmessage = rd_kafka_consumer_poll(c, 500); + TEST_ASSERT(rkmessage != NULL, "An error message should be returned"); + TEST_ASSERT(rkmessage->err == RD_KAFKA_RESP_ERR__FATAL, + "Expected a _FATAL error, got %s", + rd_kafka_err2name(rkmessage->err)); + rd_kafka_message_destroy(rkmessage); + + TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, + "Expected %d rebalance events, got %d", + expected_rebalance_cnt, rebalance_cnt); + + expected_rebalance_cnt = 0; + if (variation == 1) { + expected_rebalance_cnt++; + rebalance_exp_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS; + rebalance_exp_lost = rd_true; + } + + /* Close c, a fatal error is returned */ + TEST_ASSERT(rd_kafka_consumer_close(c) == RD_KAFKA_RESP_ERR__FATAL, + "Expected a _FATAL error, got %s", rd_kafka_err2name(err)); + + TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, + "Expected %d rebalance events, got %d", + expected_rebalance_cnt, rebalance_cnt); + + TEST_SAY("Ensuring there are no leave group HBs\n"); + TEST_ASSERT((found_heartbeats = + wait_all_heartbeats_done(mcluster, 0, 200)) == 0, + "Expected no leave group heartbeat, got %d", + found_heartbeats); + + rd_kafka_mock_stop_request_tracking(mcluster); + rd_kafka_destroy(c); + test_mock_cluster_destroy(mcluster); + + TIMING_ASSERT(&timing, 500, 2000); + SUB_TEST_PASS(); +} + +/** + * @brief Test all kind of fatal errors in a ConsumerGroupHeartbeat call. + * variation 0: errors on first HB + * variation 1: errors on second HB + */ +static void do_test_consumer_group_heartbeat_fatal_errors(void) { + rd_kafka_resp_err_t fatal_errors[] = { + RD_KAFKA_RESP_ERR_INVALID_REQUEST, + RD_KAFKA_RESP_ERR_GROUP_MAX_SIZE_REACHED, + RD_KAFKA_RESP_ERR_UNSUPPORTED_ASSIGNOR, + RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION, + RD_KAFKA_RESP_ERR_UNRELEASED_INSTANCE_ID, + RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED}; + size_t i; + for (i = 0; i < RD_ARRAY_SIZE(fatal_errors); i++) { + do_test_consumer_group_heartbeat_fatal_error(fatal_errors[i], + 0); + do_test_consumer_group_heartbeat_fatal_error(fatal_errors[i], + 1); + } +} + +/** + * @brief Test heartbeat behavior with retriable errors, + * ensuring: + * - no error is received on poll and consumer close + * - rebalance cb is called to assign and revoke + * - final leave group heartbeat is sent + * + * @param err The error code to test. + * @param variation See calling code. + */ +static void +do_test_consumer_group_heartbeat_retriable_error(rd_kafka_resp_err_t err, + int variation) { + rd_kafka_mock_cluster_t *mcluster; + const char *bootstraps; + rd_kafka_topic_partition_list_t *subscription; + rd_kafka_t *c; + int expected_heartbeats, found_heartbeats; + test_timing_t timing; + const char *topic = test_mk_topic_name(__FUNCTION__, 0); + test_curr->is_fatal_cb = error_is_fatal_cb; + rebalance_cnt = 0; + rebalance_exp_lost = rd_false; + allowed_error = RD_KAFKA_RESP_ERR__TRANSPORT; + + SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); + + + mcluster = test_mock_cluster_new(1, &bootstraps); + rd_kafka_mock_set_default_heartbeat_interval(mcluster, 1000); + rd_kafka_mock_topic_create(mcluster, topic, 1, 1); + + c = create_consumer(bootstraps, topic, rd_true); + + TIMING_START(&timing, "consumer_group_heartbeat_retriable_error"); + + if (variation == 1) { + /* First HB returns assignment */ + rd_kafka_mock_broker_push_request_error_rtts( + mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, + RD_KAFKA_RESP_ERR_NO_ERROR, 0); + } + + rd_kafka_mock_broker_push_request_error_rtts( + mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, err, 0); + + /* Subscribe to the input topic */ + subscription = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subscription, topic, + /* The partition is ignored in + * rd_kafka_subscribe() */ + RD_KAFKA_PARTITION_UA); + + TEST_SAY("Subscribing to topic\n"); + rd_kafka_mock_start_request_tracking(mcluster); + TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription)); + rd_kafka_topic_partition_list_destroy(subscription); + + /* First HB and retry */ + expected_heartbeats = 2; + rebalance_exp_event = RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS; + if (variation == 1) { + TEST_SAY( + "Consume from c, no message is returned, " + "but assign callback is processed\n"); + test_consumer_poll_no_msgs("after heartbeat", c, 0, 200); + + /* wait 1 HB interval more */ + expected_heartbeats += 1; + rebalance_exp_event = RD_KAFKA_RESP_ERR_NO_ERROR; + } + + TEST_SAY("Awaiting first HBs\n"); + TEST_ASSERT((found_heartbeats = + wait_all_heartbeats_done(mcluster, expected_heartbeats, + 200)) == expected_heartbeats, + "Expected %d heartbeats, got %d", expected_heartbeats, + found_heartbeats); + + TEST_SAY("Consume from c, no message is returned\n"); + test_consumer_poll_no_msgs("after heartbeat", c, 0, 250); + + TEST_ASSERT(rebalance_cnt > 0, "Expected > 0 rebalance events, got %d", + rebalance_cnt); + + rebalance_exp_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS; + + rd_kafka_mock_clear_requests(mcluster); + rebalance_cnt = 0; + /* Close c without errors */ + TEST_ASSERT(rd_kafka_consumer_close(c) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Expected NO_ERROR, got %s", rd_kafka_err2name(err)); + TEST_ASSERT(rebalance_cnt > 0, "Expected > 0 rebalance events, got %d", + rebalance_cnt); + rebalance_exp_event = RD_KAFKA_RESP_ERR_NO_ERROR; + + TEST_SAY("Awaiting leave group HB\n"); + TEST_ASSERT((found_heartbeats = + wait_all_heartbeats_done(mcluster, 1, 200)) == 1, + "Expected 1 leave group heartbeat, got %d", + found_heartbeats); + + rd_kafka_mock_stop_request_tracking(mcluster); + rd_kafka_destroy(c); + test_mock_cluster_destroy(mcluster); + + TIMING_ASSERT(&timing, 500, 2000); + + test_curr->is_fatal_cb = NULL; + allowed_error = RD_KAFKA_RESP_ERR_NO_ERROR; + + SUB_TEST_PASS(); +} + +/** + * @brief Test all kind of retriable errors in a ConsumerGroupHeartbeat call. + * variation 0: errors on first HB + * variation 1: errors on second HB + */ +static void do_test_consumer_group_heartbeat_retriable_errors(void) { + rd_kafka_resp_err_t retriable_errors[] = { + RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS, + RD_KAFKA_RESP_ERR__SSL, RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE}; + size_t i; + for (i = 0; i < RD_ARRAY_SIZE(retriable_errors); i++) { + do_test_consumer_group_heartbeat_retriable_error( + retriable_errors[i], 0); + do_test_consumer_group_heartbeat_retriable_error( + retriable_errors[i], 1); + } +} + +/** + * @brief Test heartbeat behavior with consumer fenced errors, + * ensuring: + * - no error is received on poll and consumer close + * - rebalance callbacks are called, with partitions lost when + * necessary + * - a final leave group heartbeat is sent + * + * @param err The error code to test. + * @param variation See calling code. + */ +static void +do_test_consumer_group_heartbeat_fenced_error(rd_kafka_resp_err_t err, + int variation) { + rd_kafka_mock_cluster_t *mcluster; + const char *bootstraps; + rd_kafka_topic_partition_list_t *subscription; + rd_kafka_t *c; + rd_kafka_message_t *rkmessage; + int expected_heartbeats, found_heartbeats, expected_rebalance_cnt; + test_timing_t timing; + rebalance_cnt = 0; + rebalance_exp_lost = rd_false; + rebalance_exp_event = RD_KAFKA_RESP_ERR_NO_ERROR; + const char *topic = test_mk_topic_name(__FUNCTION__, 0); + + SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); + + mcluster = test_mock_cluster_new(1, &bootstraps); + rd_kafka_mock_set_default_heartbeat_interval(mcluster, 1000); + rd_kafka_mock_topic_create(mcluster, topic, 1, 1); + + if (variation == 1) { + /* First HB returns assignment */ + rd_kafka_mock_broker_push_request_error_rtts( + mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, + RD_KAFKA_RESP_ERR_NO_ERROR, 0); + } + + rd_kafka_mock_broker_push_request_error_rtts( + mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, err, 0); + + c = create_consumer(bootstraps, topic, rd_true); + + TIMING_START(&timing, "consumer_group_heartbeat_fenced_error"); + + /* Subscribe to the input topic */ + subscription = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subscription, topic, + /* The partition is ignored in + * rd_kafka_subscribe() */ + RD_KAFKA_PARTITION_UA); + + TEST_SAY("Subscribing to topic\n"); + rd_kafka_mock_start_request_tracking(mcluster); + TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription)); + rd_kafka_topic_partition_list_destroy(subscription); + + /*First HB is fenced and second receives assignment*/ + expected_heartbeats = 2; + if (variation == 1) + /*First HB receives assignment*/ + expected_heartbeats = 1; + + TEST_SAY("Awaiting initial HBs\n"); + TEST_ASSERT((found_heartbeats = + wait_all_heartbeats_done(mcluster, expected_heartbeats, + 200)) == expected_heartbeats, + "Expected %d heartbeats, got %d", expected_heartbeats, + found_heartbeats); + + expected_rebalance_cnt = 0; + /* variation 0: Second HB assigned */ + if (variation == 1) { + expected_rebalance_cnt++; + rebalance_exp_event = RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS; + + /* First HB assigned */ + rkmessage = rd_kafka_consumer_poll(c, 100); + TEST_ASSERT(!rkmessage, "No message should be returned"); + + TEST_SAY("Awaiting partition lost callback\n"); + /* Second HB acks and loses partitions */ + expected_rebalance_cnt++; + rebalance_exp_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS; + rebalance_exp_lost = rd_true; + /* Needs to wait HB interval */ + rkmessage = rd_kafka_consumer_poll(c, 750); + TEST_ASSERT(!rkmessage, "No message should be returned"); + + /* Third HB assigns again */ + } + + expected_rebalance_cnt++; + rebalance_exp_event = RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS; + rebalance_exp_lost = rd_false; + + TEST_SAY("Clearing mock requests\n"); + rd_kafka_mock_clear_requests(mcluster); + expected_heartbeats = 0; + + TEST_SAY("Awaiting rebalance callback\n"); + /* Consume from c, partitions are lost if assigned */ + rkmessage = rd_kafka_consumer_poll(c, 500); + TEST_ASSERT(!rkmessage, "No message should be returned"); + + TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, + "Expected %d rebalance events, got %d", + expected_rebalance_cnt, rebalance_cnt); + + if (variation == 0) { + /* Ack for assignment HB */ + expected_heartbeats++; + } else if (variation == 1) { + /* First HB assigns again + * Second HB acks assignment */ + expected_heartbeats += 2; + } + + TEST_SAY("Awaiting acknowledge heartbeat\n"); + TEST_ASSERT((found_heartbeats = + wait_all_heartbeats_done(mcluster, expected_heartbeats, + 100)) == expected_heartbeats, + "Expected %d heartbeats, got %d", expected_heartbeats, + found_heartbeats); + + expected_rebalance_cnt++; + rebalance_exp_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS; + + rd_kafka_mock_clear_requests(mcluster); + /* Close c, no error is returned */ + TEST_CALL_ERR__(rd_kafka_consumer_close(c)); + + TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, + "Expected %d rebalance events, got %d", + expected_rebalance_cnt, rebalance_cnt); + + TEST_SAY("Awaiting leave group heartbeat\n"); + /* After closing the consumer, 1 heartbeat should been sent */ + TEST_ASSERT((found_heartbeats = + wait_all_heartbeats_done(mcluster, 1, 200)) == 1, + "Expected 1 leave group heartbeat, got %d", + found_heartbeats); + + rd_kafka_mock_stop_request_tracking(mcluster); + rd_kafka_destroy(c); + test_mock_cluster_destroy(mcluster); + + TIMING_ASSERT(&timing, 500, 2000); + SUB_TEST_PASS(); +} + +/** + * @brief Test all kind of consumer fenced errors in a ConsumerGroupHeartbeat + * call. + * variation 0: errors on first HB + * variation 1: errors on second HB + */ +static void do_test_consumer_group_heartbeat_fenced_errors(void) { + rd_kafka_resp_err_t fenced_errors[] = { + RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID, + RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH}; + size_t i; + for (i = 0; i < RD_ARRAY_SIZE(fenced_errors); i++) { + do_test_consumer_group_heartbeat_fenced_error(fenced_errors[i], + 0); + do_test_consumer_group_heartbeat_fenced_error(fenced_errors[i], + 1); + } +} + +/** + * @brief Test consumer group behavior with missing topic id when retrieving + * metadata for assigned topics. + * ensuring: + * - initially a partial acknoledgement is started, with an empty list + * (variation 0) or a single topic (variation 1) + * - fetch doesn't start until broker returns an unknown topic id error + * - when error isn't returned anymore the client finishes assigning + * the partition and reads a message. + * + * @param variation See calling code. + */ +static void do_test_metadata_unknown_topic_id_error(int variation) { + rd_kafka_mock_cluster_t *mcluster; + const char *bootstraps; + rd_kafka_topic_partition_list_t *subscription; + rd_kafka_t *c; + test_timing_t timing; + const char *topic = "do_test_metadata_unknown_topic_id_error"; + const char *topic2 = "do_test_metadata_unknown_topic_id_error2"; + + SUB_TEST_QUICK("variation: %d", variation); + + mcluster = test_mock_cluster_new(1, &bootstraps); + rd_kafka_mock_set_default_heartbeat_interval(mcluster, 500); + rd_kafka_mock_topic_create(mcluster, topic, 1, 1); + if (variation == 1) { + rd_kafka_mock_topic_create(mcluster, topic2, 1, 1); + } + + c = create_consumer(bootstraps, topic, rd_false); + + /* Seed the topic with messages */ + test_produce_msgs_easy_v(topic, 0, 0, 0, 1, 1000, "bootstrap.servers", + bootstraps, NULL); + + TIMING_START(&timing, "do_test_metadata_unknown_topic_id_error"); + + subscription = rd_kafka_topic_partition_list_new(1); + rd_kafka_topic_partition_list_add(subscription, topic, + RD_KAFKA_PARTITION_UA); + if (variation == 1) { + rd_kafka_topic_partition_list_add(subscription, topic2, + RD_KAFKA_PARTITION_UA); + } + + rd_kafka_mock_topic_set_error(mcluster, topic, + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_ID); + + TEST_SAY("Subscribing to topic\n"); + TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription)); + rd_kafka_topic_partition_list_destroy(subscription); + + TEST_SAY( + "Cannot fetch until Metadata calls replies with " + "UNKNOWN_TOPIC_ID\n"); + test_consumer_poll_no_msgs("no messages", c, 0, 1000); + + rd_kafka_mock_topic_set_error(mcluster, topic, + RD_KAFKA_RESP_ERR_NO_ERROR); + + TEST_SAY("Reconciliation and fetch is now possible\n"); + test_consumer_poll_timeout("message", c, 0, 0, 0, 1, NULL, 2000); + + rd_kafka_destroy(c); + test_mock_cluster_destroy(mcluster); + + TIMING_ASSERT(&timing, 500, 4000); + SUB_TEST_PASS(); +} + +/** + * @brief Test these variations of a UNKNOWN_TOPIC_ID in a Metadata call + * before reconciliation. + * + * variation 0: single topic + * variation 1: two topics: first gives this error, second exists. + */ +static void do_test_metadata_unknown_topic_id_tests(void) { + do_test_metadata_unknown_topic_id_error(0); + do_test_metadata_unknown_topic_id_error(1); +} + +int main_0147_consumer_group_consumer_mock(int argc, char **argv) { + TEST_SKIP_MOCK_CLUSTER(0); + + if (test_consumer_group_protocol_classic()) { + TEST_SKIP("Test only for group.protocol=consumer\n"); + return 0; + } + + do_test_consumer_group_heartbeat_fatal_errors(); + + do_test_consumer_group_heartbeat_retriable_errors(); + + do_test_consumer_group_heartbeat_fenced_errors(); + + do_test_metadata_unknown_topic_id_tests(); + + return 0; +} diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 93ec0d57d8..15c1cf6eda 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -137,6 +137,7 @@ set( 0144-idempotence_mock.c 0145-pause_resume_mock.c 0146-metadata_mock.c + 0147-consumer_group_consumer_mock.c 0150-telemetry_mock.c 8000-idle.cpp 8001-fetch_from_follower_mock_manual.c diff --git a/tests/test.c b/tests/test.c index aa03310efa..1a1a4116f6 100644 --- a/tests/test.c +++ b/tests/test.c @@ -262,6 +262,7 @@ _TEST_DECL(0143_exponential_backoff_mock); _TEST_DECL(0144_idempotence_mock); _TEST_DECL(0145_pause_resume_mock); _TEST_DECL(0146_metadata_mock); +_TEST_DECL(0147_consumer_group_consumer_mock); _TEST_DECL(0150_telemetry_mock); /* Manual tests */ @@ -521,6 +522,7 @@ struct test tests[] = { _TEST(0144_idempotence_mock, TEST_F_LOCAL, TEST_BRKVER(0, 11, 0, 0)), _TEST(0145_pause_resume_mock, TEST_F_LOCAL), _TEST(0146_metadata_mock, TEST_F_LOCAL), + _TEST(0147_consumer_group_consumer_mock, TEST_F_LOCAL), _TEST(0150_telemetry_mock, 0), diff --git a/tests/trivup/trivup-0.12.6.tar.gz b/tests/trivup/trivup-0.12.6.tar.gz new file mode 100644 index 0000000000000000000000000000000000000000..5417120a69a5cb6cf73cb0d5ecdc8f6ace6fd4d0 GIT binary patch literal 32862 zcmV)bK&ihUiwFp%?>J@x|8#O`c6D$qFfK7NE;cT7VR8WMy?cM#xUn$Yf2&V{D#@Xg zO0?u_qE`D`#dZ>J9NVv@>~`z;(V;}j=9(f^lJaF!pU?iz3;+_mNwVWjw=K18ERn!q zFc=I5gTaixrmuZ}V%)yB4Ab)e@SA*E{8RL|)!NvMzvJ_EYrVbx2X_01Z}6E1q2WXG zfB5hGtiNK@(4Jb|_D*}N)!Ny3y`yh!@2u}^Zdd+Le*WV>p>JQ$XU&%0Ue~wRe!Ka< zv$chP+dEq=^=~xi+nd{M_-|)>a~tOW<~lrLTiRgJCh~t>%Kz=H_1BwQ>szn%Rw?=aOXdIJ!S4Q1fB!d_|J$3JMe=`rbA1Qo z|N3ToW2d#XwGQ)ttG%=N2iE$Q^Z&n8{_lFTJKw&z3|VblXX~x@b`$=2-C!S$3wIu{ zebaI;ZO5tf9EVZCfcaKn`PY`IS1PBLX$QbVqj_k1E;C${%>#?sE(^T5Kep(}$aW3? zj!is&8Z_9A9bPier@!7jtV})Ap4ejp0W_H5TWn_eQ#%YTlg)hZ+BV@!cxi<2j|D&+ z$GfrJ3pVy#)5b~xRhn91r&4*zk{AWdn@Ci~o(aWZyrBc30VpCoW8_`qD@g^{3+=Jh zfOmFKVax$o2tsYlHB*>CgJZ`4A?oXS7(h!31tbR0HFFMFEkFX=M>HOa1QT5|z43f% zxgk*&U_qTVpt}d}d^R;g%eM_Dh-pj%LlqVDp+|%F2YuE*emnT7ce>Bu`^o9?j|Y4E zd+g26Z18@c?H-@}e0uQi{eZncKHS?s?X%v|9y~o7oF2USG&qI_)m|TJRx9|Tcl0yc z|J%vwe!tI-Puao8lfwf52GCD?M}vd?euEtx?H+#GJ2-mRU;uy}9S_*y!N-FE6dN2j zu+d7k8asZ=KJK6HzK5T^HwT9YgP#eJw+DkGZ0jvF)?+8V)4{>+r^DVUJNa~aa@^l% z(5=eeL4WtKckpq4Plwi_MYjLr{?UN--}ep=b9;%nC;IqiACT<5Ioz*MOVG=`gVX)p z0d^<;whNR4{0I0GH}pWgHb2ZK+8efIA7cn=9(>F=NZ2=cxE7j}5uCu)7_?>E?9Z_uNL0R)f= zUc>h{pZW(xw1cC;{^{wblfl99Q5|~z6OiS!vfG2IdqmyiBkC$p==k(!1d22u;y2h& z@Au*PDH4<@)k6~WfnvJ@rC0^p0|E||KC+|zcZUb>_K$Y=5u0O#^3y?ozYb${(8mG? zyxE_6(B>!VGL8{oRiUp6$2Mpr*}+@Z+xzhV(H4ba?E43TPl#l@?-?hz{@fG-S>Z zYsPFAu$R5rZ0%(wE@a)Rh1kpgD@8@Q{(trQU!MQ`>(>AB{HI+1qxJvf!@K6e(c9y1 zYX57uX#H2uf40`Qx61YZTYNrRp^epe z*c01=$|oGDDO$@84fkRWt+Nh1zXKpHe%p9p`SnEkLwjtw0Tns!A23jzeDB&a@%Nkl zo_w9Cay+wqBg6*!cR-dYugeZx)0$cEj~iA*wdPyf0a^(IedW~p$K3XOo4EcL%{@OdUpIn4#?oH3g@;)FJ+R4RWubL^3CUXel{Tdu^d`NqxK z)IgYPdJJGl0Q8xAQLntL^)3Gz)uCX1aRJYv!TQTeMWAS-8aZ*STYKbK4K{P2kU?J^ zJT`6KSojBC7=8%7a19$e1VF|OHUUM>y7Bz0%Ea@j%eHC;UPWtI`x&>D6X3Re{REv8&tuFgQ^H0{a6@=+@s z`Q8=4zUHbqV))iF!nqIXCK~QKl?r$wwW4tQt*>%-fr(LA*-@YMJYUw^Rtk>z^<+hQN~c8LJ* z`u*O?!CLgj0L2~Z3kqYUF!P zV06}IFpfrOv9aZc+=swo7nVypH9)OYUcT(_?|wSP6TY8%r$+}z?_R!?`x>AGx-`sP zHg`c&53O-%ndqKiT1^kMZ{Q=(^PRiq7da zPj6t~z>f9@KOLWb=mWDgl(K_HXXo@S+@qVF1#6dEST9YXXqT(iN-!X^2l@W zc`h~=8>~4Ml-31+Kqq*+0$j5x*q3eSZM_A*1HV!^f!z`N-)8gDa%TTaB$#@>B{&ut zy$9SmY6kG$;X^pG7|QTax-b~yF%l7`CQR`Nd6-u-&jq|;Q>tS7{6p{Uhu-j{H+a7` zvfVYBZOzb|+2cSDE&;1wJ(z3A`t14NKMZ?&d*r$BS&+Wj9QzhBUXxd%Ne(|y2nlD6 zP59|s5pOgn1`Y3qz<^qWSFAZ=&)=LLe?Sk4<{|VnKo)|rvAOR{+;999j|!&&h=e&h zkC@&Kcf>I*#{!~6Jw-Tj=uKtF`KJSXo04Z^Qq^T0a=q2K(W3AHN@2A&?2b zy@ggm6m%F))F5!=vy_T?4?MZ+;QuB;edb55Y> z-81I-4bH!T7f~qc3XLtH)L8_oIhi|-7;hBeIM5**oF4r6>123%d_3s>pIDh;HA4|! zL#M1uSYJ?%prl~qSPOlyKdf?ifs>3L~f!!j5KL`d$1vd|ML8&JpU;_zn}eoV1=lzO)kFW{=e03@1)Luwp;6^|Hn7^oK3|6(0RqQ zg0XMUxQc4SKDNsudr+lvW?ftiZFl0GS3=`rIGWpz2_@)ne^|G%mI zU#$PHHcI>dfBE`9yLj0s0zxo)gHI6WcY1(yqY^T{zxIzQ*sB3hEW07nt@qtko;VYz99x&T$&OJcDQF z@DOh}L5XMVEr3POiGXaJWUM#wJ`=*LX^Ly=8oH)xb#4KPLH)VQ&Th|cbwsbGQ9&ql zPG+jx7|;u-udx@*tmha|?FqX*Yqp`*p8(ze)6Odu>VM5OKu${hr7c+4FuNL`LeX-Y`|1dgif2-9N;0^M8>1rvDn5cRFen_4%T`lVw6BV5M@P7H}O;q0D z{wPD#5QFZSCOlHKb6G$#MK{4BwR-A3Dvd@L|B)|*p=&72W&m6n98J2oXsQujlMXJ) zr?GpBCLTxFjU(@yB>d_M-a*yI!z!#NE3D=pl&rJdK`1JJUVqH8UD|(2`%h{ADfK_D z|M8Qa-(vnhEdI;Z#^!n%|E1jjm-7GH?f*;vzf%4epY;Eh@_%zX75@ch|9a{F_f0;} z{xQ{fx>h2A@2qpv=L4Zk&E8Sbgoh0lZB_I6qFHGezvZ8xlGeRSoa z_%34qTUYDqD@{?gx4cz2&$2ju-fnBV{VCR|ya07*h-c0z28BT4Zm4qfcD13@C z?R&FY{Xa)Qmg|4H{+H{2x&CL?|9;>+)j`0L^}oHbm5%=gZ+FV||97(fOYeKW;LEjM zErE7+9T=uJi;D4I6-6b0CpBTOj}-x_184&%vKv{vwQxFvKc+RBUvvll+!9OlGjae& zz~puvH~@mTR1+{oK|FSV*E;vj9bQ>?uyFfUO`G0v4M8mHV$J8pOeV{9*oy$q2Wm#6 zE+G0zi0E>T(8i99e(*)mBp`ngl)!Q(fqmgx<}d=tYVsE^u0B$u2(ueo|>0JC@^uN;nU+RCc z{>Km4e}nn|K>PpJM!U5Cm;3)x{+Ijz^7GB*|8R(Rorl9GG5{=#|Gu@encn|zZxpW$3J6F7&`tLd&1!H&)5@&j|bWE zqyglgv8N3o|BO9J5c$e*i2kfY)@5hpi;R{{d77%7|1Pww(*9H0e@gpLssAVRf4VpE zueAScZ({zZ(*9G*|5E;!^1uB2E95`<=)IxCa?AqpE#?1ud!xNm%Ky^-U&{Y)Xa6Vp zzx`^vRoeec`Ja{lrZscCJ3Y93a{0ft1M^?V|LyHI=z{AT?XA-P|6AGr(e^(wf=jac zqt%c`n$^U8?7Fy*j%MV)N1-s|kg@3W$8RH3&{MQR*9p##`=|Yb<0BSWQv*Z21|2jk z+Y|O&zQF*x{@lH4(4hct=E5eMH^x1bu{rk+&}7Ig2MUcZt??BCkx>Pekb~B(9fSd0 z1CUKb^`he5WMYpojIXe?Qy?R5w?5z9OjkfU;sdZ$$*;O}#sy(!=A^;r_>y!Oz3~;FQHM{=(ygw(FJ3;qkk-2Z#IJ=e6lo zXiaDJ3Wxj*fj)o9jF3IAVa%p_m3_yai{eqCxHB>I@04{_*A^yYLOuv2NsFeFRiV^iqU^h=z*~BB1scJobn~_$Ir&i}3M` z1=|>V(Qe10qkg|rKvQr;|}H#4A>F6Z8Dvc2M8#7 z{l_Fur*qdulz6eKaMIoNrZWs6hS63lf7I6`t$#5v>MsTnW6fI;DEF3Q8-#a`^$`!; z5>E!AOfd7p(flG1CFD zVL|1p+D+a-5n4G@1Sv?;j}tISur!>RH#4)GnuOY)V+>>r0Lukjs4&WXWt49AY= zTDg<>9WoTWMI=90pObj|T;)|8_(VX4XPhZ1bS*rO=IG}t@7INZgt3wZ|3?^l|N3eV2xtUnJV)uf8-64XyKjsbb6>l zMWmt^-Kr#q;g2uvYfI;Ai|+XsYj^_7v0QupMfCliKNzlqV%LnH>fyz&?D3%5DNxW2 zVCz2)PJc!u`(3pwl|RZg*D|{;C&SBsn!LN` zzQdY_tQi0aiFmMvj?H-KMwt(Tz;1t?uqN^ssJF9b903)G$E8n!HWE^3zyE zYsep?HDA^+96-QSyMaTIzs3uJ6|F{z*%8gTIA4g$xGAAR^%YH_vTU8>dZ6&fh_yRq z&F6|R$=*zOf) z%m_5F1d686w|EB~Sg?Ww*{TUakRx)r1`Y$n`<2UWtQOo&srgV z8F$Xabp-wSM8coGHwkahZSt)*cTMsOWVK%ZZJohSj9mz7yhzScc}6-EIRU~W5h0Kn z7qA5KY#xkZk`aV{bUZlN-DebEI`SM8?t}n`d>AP!2PGCUxeue^Fx)iyQ;vpjT^PQ} zJv^l&B&bN1A6(iq^uxn!O@OMzB;a7FP3Yv$-dyxS>8_wcgNbaXY#qelGgiYQGIC$F z{uj!NF*WWedk7k>V#wC**qVi?Q=x|G*szJQ;%-GP04;tNFuW0Ym>Sd#RPK~w%m~RQ z1s_z`L0iAMxluKB*|#+jfU!vkLyb=^=5UIWkBshQVxkdX=@>USY&14B;#Ke-wL7&wE4-HrbnA|Dcafc|RQt-lJ!o>TO9*ihBp>4@f?Q=z;g) z;ApSGESp?St=k#pK=LTWh+~r{al}3n0m|ZqDRE}j7=44GUm^nIg$LU-+N_e3a1clz zSb*Z#Q#%y-nKGTyD-|x|e9sFHy^9OW@9yot`ScFx5}N3VEZze(G4vwF0>Q^!7!5fv zyudEN_Z+Jy%H>zn9pf&y_K6XO(1SyIy|6AeB*Kr1O5L*;<~e&YV=tzx_F`Or;bxE~ zOR5p*sd7vK2 zu0?&)bE+?t6mj`Ha_2yM6uArGtVV87t>?Z=`kv!3%z#HBS}21mYUrXlhqNcY z&g$a&60{xMxnq6Lniz8@+`op6!Fy24oLJ3Dtjl?zcmNT7m;QXhfb}a^1!ZvZpu+lz z!21M#;MN1w`RDT4;1W%#E5O6iIcOb()>!}bN}&JuE8YQKJ*WNd!O`2}G}t^mT`R=B zl77lp0bFQZ%X!|#T&E3|S{o!Q(4Okr4)2x#nP3nW81e<}0y)q+?)hE3nzNs6GOw~w z?LP3{*};;!8GJMKIifi&DT^;E^ADm*s;(eL9#2LBOm-wI(ux{U>J~QuKXllR?DFME z@J4EJDrz8AH?I*=(ezjsyofuJqrGJt%BwF~hu7tMWJ)6mvV!d67Qd=#jKewX?TtA= zV^EiG`eQ+uM~lk;X^#3gF|2g)5vFtwP z0AN}Er}ifLe{St;wzju7+UWnkjmH3`{r{WT|3!GmpeI<(hAPP>6TPzHtxv^cr$OIM|>7jV^{k~$D*T_!+)KW{tkB3TO z)MsGNgb=)oA$*IU1&N35e5%J8*x?mC7{(8v*?H)RxP`h%_HMaoA;-4i-B83dglj?BRv`zYQvZ5jgtPLbc2eri!8&*Pz1bg89Hj(K*a~eIDQdeS&{G zhB>v}ih)rjbW9q4=ed?{PjuJQjj^Lo@Zg9Osg$#&{#WXMzjyr)?|-)2uS@-})c-R2 z-`9i!Tde+LfBR~i4SjQ>@}|0?5uEsg)B z_s9Yn=pP^K?J700Zv@gfrSVLU z7aIs;Kt=qu%b_P!l-9caayOTCEC~cfJEf z$F;~qe=ojc8eKyrFmvwk_Tki+gt%5%bp*r{Lt7IjgPGPaF+oG-1fXM@N%si0tPKE3 zUj!Bclj!^RGs8EgQPK+h?9P23q@{e8DGH$Fqg}F}%z3&gN`>8Efn_mJCSco~t_ek= z&k-&~bQ2-;m_Ns3*Gg2rKc69iK@d}8n7E>&-ZptD6H~LMAgTNfz~<=acmqo}zx@ib z9vvFprVDF;Z;y-eGe`);A^b+i%m^{!6S4Sb6BmT*61DHzOm>oPjR#Oaih_%S^9RTce)gGc4Dgp`ZkK3$Q_%ZQ+G_L*&UrXM!NrFtp2G z46!)0PI+3m0|Gq}5?0vr?`WtyQJ2!3ma&Ke#C;k@M!%^!(gieDoHr^i!?bg}mZ?n* z%1f1$Sox)yMgynCI8ZsUvJU0i*kzc$54 z2y2?I)z8m58|U%WOMYwNN5h$0`@Zk_Ie27|rvyz45q{lZ_`mZM9e%xnmohEW#CFXg zsCYF^Rp^vePL2uG0&$BfS`s_haZOn71N*MKuD@b8urW4V(v)CVOX^?~)j?@VV1NIR zs4byOtpCc+)*I|qgSF1HYm;xFdb3Z|g2G9Z(OeOsAw_6%wdPG-`KY6&1|P4&c8be# zTB|hCgapv&77E%@lG&_s4sX&Mr#m=dXhP^hlda7LQZ9Dy`3EbT*QK0>B(`7mpX~& zO5HZCAcTQY28$jm(9VmhjzFA5!sCd+H%PIW~>O$*~bDG== zrwPRt3k*k&H_8er{03}>QlqAbm|qMdkTv)_1rr4)DNd}^&agr0CUJvFTR(1tIzBwy zKW#889P2s2BEdxfip43#f{(VBT3s&C4XvIP8MG86hbbb4@a?W?GwzJ5`<`P3U9R3Y z;)DMb*JbOg{{8|<9c6aVjk}!!lI&~*DYtx<)3PZ1D%Vt9h`#QZdrjvTKWmjGYm6c= z>-B^VNSm#I8%*LALh7YBNN2Z^YLy0?(qD1dV6+M*Ac{|8Y}GU-%EhCKgXvfZ~%3DM(40=8=^0v1YV8Zk9}Hbx^W! zH2@i5n$$oNua!{lAEtOzkUb_I(Lbx;EusO3nw{fQV}|lBnl6!#j`W<-p+r|>e`cQ( z?Zp$nldv##q?IYAdMBrLvP5K&>O{E4sDeT~G|?jKOHq_o=3y8}!$ci6*$(bGmh2_Y zTIWPgb!_7%)5H1^$hCL8`(gi7k+WWGWMh(kOwuoZ$dT-S2D&#mu$YG0N}@9hCl*+p z6wg;Gh0;m|qc8`WoCDnb%l9>-o@^hj+|WKd@bK zI7kIz(!$8j1AYM<^<#tIq_5TW#9mtv%g||k4{)zkQx~S3t&kPsC7@Q~MOM5kMHA{{ zk6TpG@=InD6ug!(zD%yYQ3WhRBD!|lkf$1h@$3X13rLYI)?Mp1wOnx1e@e1P zHX1GWl~h~n^?Fv?$EyVy;Q@iLmK@G{1C8?C1S{0f^EN2d7;3H5RhUv|?QM!a|}hNnDFWxBPTsF_1K^!kVzSooAISJHBPMyJi)!Ir|k;Q1fZrc*`O5@PaMS z4(dD!GO_J!WCy9PDvfarC_(3llM}0AY<k+^p^uu0mabG&8kd$@qB)gN3%BNiR1J-9N}|U}-6Q8j;sKk_yxOUKU}Q*$X_DtD zoF(Xf=C>M&;A57RMg}1)h6}+?aYG}T1;QMO2wzRGBb^U-B^y3 zoQ8R{iTFdTo=78P5M>J03wP(dOdeRyAFnf*%jF%u>dO?$=Vv)Y+L8~ax8N&lkA_c7iCL5%!H6oKk;lw zOVQgLUDUKg*`E&MW1lZa@~ZZtQDA8(l|=^b{LxFxd|+5}7+^RVYZ%}Dt3@hZa?8DR zyAY$DI^2x<9oJ4&jO-E79HgbQPX-qDMcrfMZ;;7bpC6x^vDbdkEg)24*yQX{kgKQU z48M=f^-gZ~pi@^}=dpJ1Q2EBVtEA-|0$a3k6?DZnv?~O+G+2Th%n5Qq6*GZ4QdG<$GfQ2 zG`y~RaP+PN+O^uAH2;a?^cMk8Yl)1~tlOD2MjseFT31id8#-)GVIuM|uRmarGgGvt zDG?Urwe9KDbK~8S+_7T!0lF964pXcOltrmHpjTz zEN!%3e#8tqX*lG^1>>n1>6NvmsB1rb{M+#4_;k>%R$-!JG;q%gVGnA|#9;$IT|TR{ zW|RD^001oT0{2zweTJFXJ+nh$6bZVUt=HR)Xb4^e=byFJ+DAW}@UDqFenHsi8Tnvg zOP@88MYWZNG<}}3nx3{)=Uoj9uA{(64K}aWD~sjBS+jjkYs6o)O4^pBan}xQnNCok z-&_f25?$62z-ild(kZN7Su!{r z9n{bP=*nmouU4z0_Hcba#)|$HNEdR!Fa#NIU=u1LGGcV`5a~Mg$R>GZVJZS^Y|K-R zBQc;G3tE(>N(v~1p)sKcL6d_I#-RH8wx<^WK--tWX1?u??U~`QDc(_zJmlgRL3Fny zA5P>!DHO3P1B@;yS|owTLiic4JhGL_-nQ1H85&4nr_gaw& zrY)rtdXbX4a85<__0ZD^uI|tA!0`0b;eO|=%`f$y*Yub5VD10uFV{E*Ai)4t*ZYUj zVf$OHc2+uo@C3O-9I2e_G;+4UWCI7~FXd24*M(V)hPwOZzFw6Ff1LgDkN+(98aH>a zkve^|HH2OB+k1hdCBXZTq3{uQ+h2G3JrBz5XM=4vi6g>qCh~W7Aykhs$w+Z6QL6kY4?E_ zVRSF13%!y-%FwfXWQ?!$OCIS?r;9M%r8~+5JbY-G`f(4Qy`g768jfw^3Cga3F9PW; zW$35qn)=|En=6c)Va_}#t=xv|8vUjn6!(a|_Y3|~YFNlb5O=)S0(J`2t<&LecF|#{s zYr+wIDL%zIZaiyc-`NlS<0DWPN1$R|S#FVH#iJ~;05meGBg{2i@w%cB}OeJJph@@nE}G1TPY|r|Lj;tBG-|K9UT7 zNn8E1MehId+0_cq>$&#?q&7s#Tj#4b6I}m;#XIX%CokW?UI;IBvYHj@ zut{6peGY4(+Kb*YfUx*Jg#RIX;4LuD9kN_aDzW*y>Gc%gQ>H_KS``@VAfq9rnZYcHnsqY@sbKcPK{>UGRf8Df1qgZ$MQ^ z`cNsLWFI}esWr!ZKH=O2N#xw2>x&j*?2{}6c4@)5<@r9y4$`_Oe^m0tu{?1=<3{M$ zWa%Uk-(jQan1}h^9+Tn~cZl~yhw@$n3}X6zCE&1JQFyS}r3;W>4T^K`ke2#E)CFAg zY4MIdCh9sURY7Qnb0`oYP%ktSD%3qZez*Ap&zV$1&r`}Dj*+2(i-d^JUIENmbdz_y zHh)=tNYc5?RFfBc#g28!DSDAgm$%+i*H%$mI0lI;FI~j6v%0>b_BrkF5}^eMVOB`l z!RZy=D-!mnln6PU^9V$k7Wu-**^G0=1BM_M;@!}9n!A;(-I`(m@m(IB0lQw4hhspi zcSorpFW+N=CQLa6a~uKl!Zo@eAy@cZU#IA(QF?8P#h@myEHw%yX+0mBH;aIw7DXE} z-pMF-i0U^@lChKHyAoQ z(fJ?wtM_(zaI`;o==>M8`^UQ>$XTHN>uW@0eNl1?H{0y20ZxQ7Z{^>A8IND!io(xe&4=#D&kdQS)_T`>6 z$E@hG(9#=p3H5$Hx6Q@^N52BDB37VobnsxEPOzY3vP(Fe9Th|9m!d=X{MKIpYN0iy zcd7*R(NQYpS(CYtEJ%n17s-J;j9d%&@WCvmG{F?$Y`IG;;?Q8e6cO==Mn;rj?DQDq zjck0W4kZM~PN@7dSqEH zPjc*rwt+S&VklLqK}7@MDEP2i9$+reZtAojgVLZp%*UYuDvprAcTq#z{2tYo4$cZ) zy*&~&)MaPM`#fLnzh89E?Urw;8t`R-w`n9p&|ef!=j)aH3k!LMeLjHta7gy);jo_e zHdVP>=}VHMF?^!qB4H&iCjMjY`SYoM?S&YG_{%*i zCPie693B&R)pI4(GRdT~er!eLxw9!6h+MmpS88I1;ey#x&nub~l?tAskrayJDs%pA zVKKa7xlfKQd_(Il4D!S*0{Ho$2%`5g4YY0 z@CC{aM;MXJZCO0Ts{o>qpoM|~rx`3%R^%W7OD%rZzZXnfEC37NZ>zV6wic=?iaJt+EL2iN49+iek#tFm6lDsrrkq;!{L5o%j0*9+ zvOw2-P&|s2FP^BYS7IM`488Jg(-Hx!#2E5*cs?@?ikgeathC3X$iTUAAXw4;o-D;9 z##>#J{vbE%;z8M#PF;b-+l#LFID zPOZ>Dla-pgM3fy_Uf!8rp^)c-?-eA0di8~+R0qq*^G@p)ZTzap)-r2Tx zURj&1Epuz<_2$Ot^{dy$c=OfHXlJtdy4`*~YE3qc$>uZDo=lonYdoH`M_c39+neV4 zc%!v3+BUXd8?Q#MM%z2)WXsqdO-6?G$}qOto0F|qEpuaQW4t+r$FHo7?IlL<{DG8t z`;HnaIZ*30}K-z5LXt7QI;@;@v8qYG;*r~OPt@&OW7uB@R@j)v&-I>Sy{CgKA6+gAHEC`(d7PAbQ|ThG79fRy*0NDbwR zuLxw5U*;a5QDh5FhX9qbu|X*nup#mF9<#6t2YfXS23-woFbP)PaKv!NcvTROHpv%F z6Zv@b%9-g+4M2*1g08RN#5H<#+@TBlk^+hfDJUV-$8_iAP+S@x@rT+IVJ`c_R-hN%?e=qaDm-*kz{O@J{_piwR9;Z!@lA|Z1 z;T0rDmw8q{?CmC!qvIxw#Kp+sSD)yPipM86EN~ZANc%-yf?Ap$opyS^X_j=pndP5T z`i}d}hvriEX8gKkA}C1}sax&vE0=h)jNdFzmQgq0P_|0n+Fc-Jn{ID4=)pQZSf>Y7 z)>QZ53&WrcqI^75BaSnQ1M)nvolatvG;&lslq^0nzM-HF;=J6f-D1;J80c!s1dH)1M%Taa*4&DoP5&^0gT&O<{Z6nLl>rk1HmJ4!w1 zyx>=5&4d$s?gfJp&vYp-??l(O#ZjzQknJKqaLv|Dye+s9Eze=IwUu`{cNEk>wkr3o zJRGTNM7s9+rM9T46q~QyR=OTFw$Yj!R0{ot`Lej(g&u*1&5-F(j zf0qUBN{}tpDt*&)z!yZVeGn{(seo0SGgnznh@VsJq6|#>ik^BSM`hDAlwm?vrEBs| z@k&Yum!-HaLO&_$m!YX?tKg++%R;^6W{+0l_pT6FBwA#(#?wW0saIABd7O`PuAoN; zn$^1`{Jy@vN2_%RG}|F-%U)12 z0W~hs4C2ca4Pb@qdt}-P=msCK1PaKe5{vR%21Duab1Y?H5J5!-zD%f7eAOLyKBqDQ zuj3%5)o-v&5%TQLq=kKsRg%gD(WQciKjEv==C=4iJO zgOE>aRq<_2V)H%mcQvE!%l9cj@^RiS>w7+cu#6US&=!dp;v^nspNkX}CI7&%{w6gALI*|} z$iiSB;lM z6<83oUVqrgM=qx7-XzgRgiOo>^+f~zAc$3=sZ{Ha8+9bJ$(mRE$>T?yqx7UB&r!=8 z_ERpWl_{MT;WZ`KfDD08Wxfy4o;>Dp))Qh=(cy;SMHtjT_68}5j}k)Ne+}LZQaLZM z5;BWHrXUz_6UKntJp9O<)L_?!Gbf%(9cU#4QnYnLm=vBHkssf6RilY_6Bb`hYC< z|5)GNru$!8+fb;z*@ANG>zl3e{@3r#|6{#fx_*?dAEoO@>H1N+etd=N2fe}IP9jN| zeEr4AQUOR8VsND4DkZ$FE?ZVZWem7;x9K>(w~Hj@8K^uAqG^1KfDHp zQllFZqL}>6!`_*e6B+{HOV~7-WWMDk554hKMMD2200AMs8~(-2@B>T!_Bqx;rSfC% z@L+G)2i6Dn>#{Q~gy(qdiGQ1vLcM!esG6t#Q` z39~`QmnChU#-A=uj%}21VQ$iF(_&oJ`3Y*zmc!9J~*q z%c~pwR$%k(1(+9=TTWq#Yp9%{=XWrw3)HcvJYj7&81=Eal92EDISV%YUAO^ z;=-|~MxDan1+YL2-|Y_+dk8{-hrs^>41L@8(7thFp=F1b1?~r;_E6NfP6*Hxmx z%DC!GG6Ua0l95H9f;K~q*v=(whSvrxQX|LW>piw>)7>PVP2Mubmy$(e8`7IOR~vlG zv91lw5NqI#a=-we{_xX)V2v^6S`?33BvPuVemeC7ht}ItBu0GkbV0zVgW3>F|Rb3E_xtqJH`m&vgOJ&j^y1HCY` zGn&Sb>ksz0j${XzW`lPq?n0jua@-&RU_3DT9Unz>_~BE7fJAh}WVYlXU?{14m7d>t z*a{x6;6vBCQK8sV5I8n5aRLL3T!)l<_%#3h+$tf#sK!xNR+(u93*&@k2>c=F3=;mAWw|t6nYR+>(*EJZnJe5GHy}_T)H%F+YD$ zs@HN-h^f$Ja6nwRbjLHypccQ)%ReU82>3U*UnnSm=@Y@Zxo}JoLj;nhx;&ILOhyy} zwdxuoR;}lFLj;(41?Kb&_{6HMw_5drGM8klt7*HK9l{NpgFCsT&K$8f>y(^*hY8_#_w z6+yQ7H&m8DLB8Rfz5U@o6E(T0^S`yRFsV?5&Murx`kIb~HcCTrX_Z*FdyxFN&l zEf;r)W*!I`Ns_8{P?ZC5clguh#m_(e@XN=+!LP^8$6tTmIQZ4v-T#miGPj0j`rDi0 z05v-G6;bAQVdHW%7#{3(#JB$b?&G}YC^Npi2M(yZ%bMZ|rj zu=6A}%=vT{;2|J-{<%oV?q*$~M|V8@{U@Y4sKTzynH@Lf6ln;CZ~G!xN) zC;%@Yp>?8Nggy_)u6I+zkjK$}qGlR*LAQlR<~*IJBtHvmFaBt$~MtE95*S}m$^?L6Gt_3=8UqWiZV4Zu}iK{v=GR^KjC9RZY5=D>b@H7Po3{(yG|MOyT*sZQ5iB&1@m`|KUq# zvEsJsP>7^exX-|D#o4Kq=C#Ej3bCVqhve|E(s+=^#BgL?ptCeFK$q0?6-g1VW1?pDN%`0Q;y0qpzjz0T)=jfre-&rZIXp0_B&dUQuycg_616jFEWblb;Jf z7=Mw|Gsn#_`YsjfIjUcAk(9oi)se~yk5-Rnhal-l!#%Rnd4;|j2F69SQ zaXd>?O>xImR|lT4iIj1)NY|2p>q}^mV7bQ-ApziN#)Z$e2$?^@a+aBRE1^bk1WvJJ z%K}@DWMra9D<^qs$?s{yC-v>5U5IbIHtQ?x=&10jTe^{9@@0Fe4FA>?S&1YYz&9e7 z%#)J_G-8|)|DJ?twx5n7@<1eO3Cy(u$~8Gxp;qWA>_d7hYZXnG%pGk(leQ;rb*xjRHly0|Cq)WmjbTJoKdO_C2k+a2e>s%JuOr-O+al(Xz7~#P!OqO5suIg8xI|M@4U8v==%)@+JP+N6zvZoqTWb;aH7!iMF#%WYhR#V##p*40kvcsdkEK3Oler5%hH zRmy2EI)o=7xW(c9qCeQPNY*G!1H~5bTDr8JW;uka6K@o;ZgGGJDoE#xsHivSw!T5F zvDAI0MX?{AFzAE+uy_ucmP^n<4xPVxe6AygXag@ zs+b2x;0(Q(#fJRkokK@{(^je__S3d0BQ|V2s9s>fPMkBPW{k$G)#VOER7IU*a3@c^ z^<&$%HnzF3ZQC|BwrwXHYh&BCZQJ@i`QNI0pEuu{p6M4;UENbPr_cFx7+#cG1=yrb zTY6R84!Asme>20*w#PU6`UT=BFVeQFEEcD^yYYi+4>CH6a`A;uO!+_pYYhEPiL0O? zUWL<575s8?uAv7%{wY^rG(+;G=yBaYe=b5l@fYx|e5DFe-a(dPu_U#O#R9*yb_do{zX8bZ6%%$s?Mp?%de3c z0S63AFgS9_ou|F&+AvmjPwHs+%w*JHroH%$e+ZaTD_&ZsjwrO9jGgd&;1^s}$y3 zrc9^J{(}ZCI-7J3-%Puz)q&ddos5QhKyoKW$Yf*d;~h;dvd|tN(iY6?6v#+>jL)y zhz`olXUQ2xBy=s-+iCbrD91gb=9J-iGDiMT}4e?$p zsyYROMNADD7M<_1*@iXFqF`ji4aqQiEkhz!NV-luZSXi1OJow9pAu zH3{4EU>h2Zk$3gK~)KaAMr6Z4n-$o z0km_t7ZL#nM7c+kriFZ{1zMCsv#unbkx%sz-^jLH5(j(xNVy<5*K0xw3L$DyU1UPG zBys@n)#~*p`_SslY5{tEc#sL4Z79Ua>OhqRy&z>FD&(gUr}-xMkBZ8bq93t|wMJfed-@qViGuo=lPo1RS2l>esCt#w?W{{5gL z@2@4}=u|mC#P^RjDp=7?EmdbqJ6K)-lmXeAPkd>{8|f**!0jrD-zxGb52g@=j_;=F z5gYgGSm)^2)bUU*fXfP$0R$pC?i|>+LBsFm>w9GT)&;NyY|$@(DDb%Ld-^2wu|?R; z#ZS=Nusw#9;<~EtS|FvD^D^?)cU$)2*n=w{uo)P3YYwmgF7Eo`$lG>hGR|X-W(Y?s zsdFrj4T)h^FMzy|I{O00*Yr5#DT_6mIXDBy_dk91m08RWU=j>7g{pc2^}!0j4rgud9Asl}FW;NxL#{?);@(5~kU%>O~zaa)bu{qEOZ@!cE+m&?Jyi;Yk~Qb20c>Jp|yZ6 z;e`dlJdu+M@vkvZct>}kL|B|vQ=~v`a$FS%J2*Vs^*iO-_MQm=hd3-*CyTcYU94MV z=}M!8!h<7b8kA+Ro(~s2!xS;`$>{z!z!L>M<&o_`LB$9j}Vz6oUFo)h*9TEu3{N6h(NUwk9F=T#9a@3WvGUA+Of$_LIz3^ zoO#kqQN!VkiV+b)|G1wfF{9a=ia#2L)zPl-W(!B5OAUHnOlP zSqKhT9U9FVoeYLSG##xTRx^1YlKu+4@)wl*8z4#f5P*??Q2>pbus3+wzsXW<&6>o| zeJ~vbbXzA}F66AtraZaQDXZV5&D+(#s#d>XS1~hZ!l8jW!Q{E86gs8g9X{6?-`|vN z<8zFiCMnLFP?XiyTi?9#N3n?H$)H*QN#Gsy)s@N@um8pP?hC8?hG2|Z+!Y4Fw1jq0 zkUCqe(HU8?8C85gXV6`#w0&;=}6 zUtyjIkH44G`JFm*ygZm8lKM!-M-N&bO?6*)>Zc8(9odn`9VJn<@$?}zT@Y*_V8Y6T zL#CuR>!y~C8LcZbfeN#i&`pRnWd#oo$r>>tR~3<9oYkUTy;22DWe~5PD@pSm%5Do;9VOu;=DH7=B5Lk$ zlskZB`sw+)mwQqe1baY3S@XLtXE9YNX#q(>NDUm$X+=beJX{kGsl3D`%N%PtaW_KlDIj~HnG#b@PTS%ONp zHGke&%J+mSky)&h8a3{CIabePr4%ZeHZ);r)bQwuO1OSWWgmM2#)A#M2kKtYDV2a> zCoINJ!l1)!4f${LC}Kr7TMu_FQ`+>ZCp>(Se8npFa8BIJ@QqdHe$v^Vc5m@7IXrTq zCo8Q+mk7qpCI%Xt>^tx z%_2DC`D+Y;vDA>{`smU^o0p!s!bUR=^sAOVkcF@pv$65=Iv#1CLu&~DRy@f zq(p401vYcR{T3RyG15@O7WZ>cMu>i!h2k+uaIlIS6zVOJqbYi;c^<4H>i1!L{_=*KeYL*<0KIbLhev8=;ds?D zoSne-$mCI^*B7~2jlb3AAn`R-aOTZh=eZrkTLfXvxLVz#t8?U~?7o7q8;>rG4|uM6 z$Qstn!Q33*5hlROq+s9#&&=?fW2~!VVO!q~&+WfMgYWbBQlM_k*u9#!n3$^-N%Uao zQOJ}Vi44vl@kT8+qxV@y#w2k}KxMPgl9(YPpdv~B64y1Wi zhocEn4fiFl7(B)&XpDsT7V3&Q5(FHRjgVN_oK+sNe8CW=T3saMu)u77TuUY(xm&rp z*1v(d14weLrQ2qVRc*xzRtc5%tfCXP*mP%pf-8N6O{YypZN1Er3$pzm@v*Y)QRe#O z##fSxL`?-&`xm0e=^<{|ngY#}nDVvIlNCQ!G)%g*Bfyg)eG{EqCCLZg0;fGSqcQA& zNOT3+rF@?hC&{Wq0%;$0hwxY^i+_~bynIK~yBKH%>N#HmO-=OGCzVMhCPP9g z*8*7H)Md@s@9OfNZ^tFG3_JYofNk&G1<3&0&oL^GtxbJ77IOU~e zFW)7`9BoSgCJ6t(lt74_C}SC;`w6`G8uN5DfqtevpS=>U)_u1EXXEK!sXJ!{lbE1h zD&AeG%iI_?P4@(5OL61ayT^y+miw^UQcghheXskjik@OSf?t?%+afZRwdi zyj%7Js5sEJEJz)HRHzZ=FH=6&%PAUVXb{q?xe@e$dZg=A@*%~guzsg>e8zwILxCuz zV3>JwM+|(DefXy*r2Q%s<&(7fKEp;YXOukY%;N<_IxPq4X{k=AzwJ5T4QVx91GU9C za#Cuid9b!mVA&J=J*RlakS}NU6)#l}*6m|0c@cW5RzR9<) z*(UuHByDeYPe#yFi4y_n;PTTDqXc}`-}&@Q0lI^JtE-d4ei{r10zOHvh_!{Uh*#BW*5(IKu|M)zLw7@) zu2{fpZ7kr{eWs(n3VVo0=i~f8Y@I3+AYkV0`PdEjdt+^T^BZ*XoBan_XZjh&{4M_r z(4+X7#|#kv1%N#Q)ZT{?>}akCTSxY(Ro6xVKo|V1c~bt{4B4hW0Qf{9nt5(FeEkGV zU#A+sK49GQ2a$$u(cE)hQp`PAjid1hf+2=`EN9Y<O9oSvq)3je{ln>m});C@4xAHeCu#OkqTqlMwJl7|$gF&cMADVOk z6Hh%qWu`?J&_fTioA~7CZOoSgsO)N6qhA5o-~BoOq}&0dW61fIW6+x z-b?A{!3zL|B+-;I`Ky28qFYx4-&jRP+DhSjTb!`?2y*24+=nOWnrCh%3PkQTOe!tz z5jf7Gv>JFx{|s!j+fy4mn3yNSh%h;Wo50bXpyd;Y0+h_nJ6Y0s}p z2GOaM#2;tKJ^w~{A-4X3&>3laml0jg zi*6&u#2FD@b(f*xTL^1N*26_1LacAsSN0i$&0d;f8HEUU6S%LmDO0UM zkiD+A=Xp&agGX#7jx$W0wcIu1SF9y$_6!?2YPcFhG;|30Uwzrh-%i_$PX@=UxA{d& z7s;%(O_S9im=%0pr0^lK_G0xHH-y3Ha|ma#l(bNd9!>?(GKEt0%rg>}KkzZXzhAYV zK22Zp6~6j;z$Ou2_J6WI1ynxtt*yN){|mhSKuy%w0a73OmVgiUX298e`U~@RNzayz z?XJL;Ut5XlyIB0|KLIlCt;U!?Y;#@0Uccp{~G>#_?)No?Z@+hmw$ls}4`P+s@NRR}v__L;XJ-px?8PbORYm z{JfvPU#p~jhSQc8Ih*Je%r?;rjw{pgaUX7-Oag#j;t-$R8eDld@FrjsM7axht<(k{ zt6i^ulLz*=lO6Yp2>5bi=MoSi9v1cZVLd-TON`e2tVCLwp7qZA7)gyZ)9oML6;MFR zCl@c*D(*M>SFp_k&)fO`k+tOKtXl2Y(eHXEvGlPqF&>x(gb$N3n&t%XeYyG20yvB+ z25&yX2G&+nBv8eQ zEtW5EXm@mocf~Q1-hMc|yk5=q@!TB@Y+uI@_U`cmA58YYiqa>={rHWx6_d3vS{&lPaJMGX_vw;aJ~& zr2*-3hd@@g50!Z)L(9B^^GzI~CT1y;9F*;%HDOK^*Kwk5ILXvJ6oh5aPNJ&mdvWka z(Y<5q;x-3$mP@~o(Q+1>EeZb3yv^yS1-*Eu>H5nSJfM$Q!aV9gzb;zdy6(ZQVr6$> ztCv(iE|<;4J($Z0ffz>)Q;);V_atu|$3T~Q0c1)V>swS{K_FI$oki4{8{r9g!Cgin zUT1LqUn-ReJE&#^AzzCux4=vr*!w6hwEIYlLwF=t!ve3@O}iW=-(63$K&!ZGTZv`I zmlM2g%t|~`*iIKRCC%s&vg#$^&kDFvg+c*H@1CU?n#7kLApXTPraiF`=vcG_4<%h2 z%g^Nwc80Y&R99pexD9h9wVu9KR*#zrdI9~b)609=!k4(KGex83mbLb^#u&Rs%W0BO zP*Z0fE%Sgm4YB|eyc{Ld-{C>cM>K?^V$`x`F-8M+8`Q>0H`(3&sob%0gJLqqa(2jN z3#T42j@wQDl;`{YXbHY#Q?~6n!iTXHE z_ucxPlwZ%{O}mB4Uzr%$PNNt_?8t7xD*TBA&ekEUw=i9BryL0K1hxunM(m=LL2E)} zDGgiDOyar6AP1EXEQ;8~_smV2{vIs#2$G-ms>n=VYC}JYwx5ljG4sX0|9Ev$?LVqZ zMZkq@w|k^`qiz#B1S6EGA&wvhnd?_zQJ{|{%Px237FMDr`Hg^U2;56j=}lG(R=hRj z7<%+bdBK^xV|ms3VtM;L#eaDymvlXoj89UEGO3C7%T?wqr!2=dvGkQcr=t~N{;}Lr zgk8VJb&L;9xtFaGy+5@fLJaJ+>d#o;<99;|T?`J*sNCG?{WVQ*J5@7`AVNVc1MY>z z+a$gPoZc$BmZg!+(hulj_E>OU!;jUBwuFM7I-K8ZuYTB>upzsD9n;HrMsf=L&WlRF zC{ex0HFbt>mRMm03xv})#pAw^{=`i)11+WRdk3O207-JG@LAnfYcXm)HmgfOJERmz zjJHBERqIzee@`j0uz!J4YQNVPZvrS~X77JV&Uel1@3k%mT9>zxC)gNLC+M<_I<(N= zo~;f2*&f3^`?_eZ=ITU>Fd8uxsHNnJsM^x7W#Ai-u@F=lu0KrLA2@56(i}w7a1`C_ zJPBw6893Yyghx{hb+uy@u&di)K=+ucnTZ8aADSM7y`vgC;s zPnIN-7hoTCEz*hlSTMTr4(N8hbwMohWtIT+SK=z1TZ> z!sL07Iix5;A7C`AB;9 zsQeR;y6Y4FfS9k>`Us2yiWWs}-{)LCpCfpQFLOcKG*fWvi*Wgu#+Hx0H_Sx;7jWyG zWSaIt4a%<0tXewDT)zTblZd&98`nn2fMJG$?B5N1t5=$(pF;YAOUZ217}1QsC5`o^ z{$QJmOPVII%&DIf)e{18D&6|-v5oW+GZg;4z3JkCY&wP}<|O=x9Ku~Ll(JildtnN! zZ9MaENhBS{WmixTcAmR{@{WwB(|$eDKg8_69jK6l&lX2x%KqTmb0>L#Hf)`<)`F;D zTfeny=)#x78^;LM;7dQU28rQQ3h~b2TskvHOr&`~t3v=ZT3E0wCsPtiTvWwAr|T92 z)lido%sH}HvG_F7o-yrmKn^FPbz4(=z$06qq`m&BxaI!ytoO}JBku3wUwAbXp$!Gi zQ_ua7yAIv{C*JKz9@}oy{*YH0TqYe@KOkGdv@dn)5#TC_i79olc0|{L!(>!lqg5v} zfecYK8KYw7{``r+{IW`5c%L$xMeec?OdQFvH#Hm0(>&OwvWaoSfW%(g)(u{91b60Y z+a{Rs5B$;@-vl>5AYvvhS^7}g&QMxe6lwnGC2;medhrP>2Nj&tuz4IpV7L&+~kEkmI6A0i56;r#=t7r zw+?mHZH%95vL=^fHgwry0%G2#7~)+bDqJ2_pyQdW>@Y|@z9x}}rM%I0aK8CelDZ_q z;_c14IheK2`-gQd7c5^IC~{PKpr*J*39)WJb~Jfec1cC1A>uCxY+lp!83Nx+N+IQy zv&Qy%_1_#6k>nT|e;%f?2)A4z18g#5GjRhE7)E|9v~*=@TCi;E(?tsX=M#IvR7A!8 z=#pr85E*iHzv;DdBxj3LtLL`*%}9xc=8`$!WHGZoGamW3Z!`@XqYq$BXdg5@5vb39 zVsp^U!+e53h;3WoACf8?D&*Bkm5?*IHV~{CGbUp-se~3K;mxa9nOzZ$50I)c^&ZAc zns!UbR8Z#376gjK0-epln== zimH+6=y&M#eGWFx5?+fiTpDVYX+2A}a}m$)q0dcqg3oWM>F2{1pB^5-Hf~(Tb6i&k zz(Z$;Q0piABF5|8v@qG8M;@V`{?NaK65VAl)}7oq*!7<aM=u$uFi(&!O1=4QtPBWbQZK?-Fsno!U-*^_$yrGP_GL@QeOBWcjx}-8P8GS$t&_s*ae* zms1{A2!z3qxY1Vui8648K*jhpMg1>Kd+WsJYpQRfw2wl-s#6_rAF=(NhuptM!e%D(mSxe1o|3?Gt%Q$3s&&b-pxxGB)25DtPc zykd9@NAI18FX7L_S{u>`%=PE##)Yr#y%B~N?j1hyEr6h9;3MD^$aaJ(bg8R z4-)r4)lw&(s!d5EsfT^gClp=_Arw5D9g2=Vx%MjD_#LX!4_eEc1JGX~X}$^`~}VtG+W z7@9Sv8HZCbTm@|*P)*F)^32@jyneCCO+?@_t^Q%d;oiey-L*2=TBI0d7~$41y#9YR zHc(Jy>%>eXW%<&ABDo{ErUl9mY*x^9+?ToKv)m*|(%J)UIHWPcWmNF|b zQina8Zbs2MrJ`mLmE)e`)vM(0U7V{HxwUx;eeb;oRl7u@IPSQ=@M-D}@ zzrMd#>6Mr*h#V)A&(d~5o5L99y!sG-rNCN{voLciq8?KG$S^=D*}WqdcPFoqTN;7U z6=?d}uT1(ww<^gkB^RTOV|zgGjd~@0=HH@3wDV71nyKjb#EBb_-5^s*94*YXWHNAS z5BNOtt(G92GR3R;0o?9&v2$x!)7H`>?w|ki#l1JxfeX=Cuw}7thHer1_onoVkjteE z0f!8Z)ZY`h*0XxkeMdtTSyVB#EF z1e;1fZN7g#W9WXU+Gm)@9@HEV*k~>Bl2V73EwcC0LU)%q>VJyZG)PjVR07zVm**>5ObeuL7(sJSTSVO5o<&11c*0KyE_8*H_LB$ zP^;uv;(k|@;f_L#Q)!Gs6b$~XDQHN3COPef;_pZv1vOl{z`nUq$;0fxA@djbxTL5# zp(vZ1JAVpUQobSqsMtKJ9loGgrm^73!E027YK6rSZS{#a$#0LWcz=TrJ#iIqur0t0 zcgy02)UfZWGc0L0^z#nx#SaZ30a4D^ zLWO6sSzdb9&DBaj5GJA*#Gn-m)&G@pHLmYj_64}o^>*G6OYG@+`-N1hLSzh6sHYa& zv>8P@(a}zoIN|hMBUfsq2u4Lr0!$C2b=>y3W#4|X#De7?PuIcTVW7ke4R;)pn|qym zt#5xBjT|Wx^ru#&Yh}sx7WuX}2OK6svZ5&7qW2?D9Otsm}d@!lB#qiPP_VufLrz}!ylM^_Q zmYAfrdnV)B>3PX3o9(7AWTjT1L2+GQGcJfcFsgfxw)oi149gHtrz4+Dv;(U5XL@pG^y&2J4+OlHyvq zZuL&;j+hKIj_#--kbn;*Vi!;gl8Xtl;3LI)Uz=5$xfIfw60e@YTS4EyTsdrKuVav6 zcm^HuCy7v%Wh@N!z`!6TSH41GI31Kf9erI`#H^2UxUOUm27yn(N9B@{{<$mG{AZ)@c?{+s5$?u; zu}(xm)HJp}`fpWX=m|O-p;E-3ct-&?+Rry`#NO1|DWc}qAwT;cAa>vF&c>%7-FHU} z;G2~Th~o&rvnfz_54a-0!*6`O0ffKa*nan50J{FCyY}bH)PFZh)xPbI0M8F$NdR!) zPfW^hJ2C(vv!>rlGQd3!;n`jLttSQW{|lh(EpMDld4I!%`NUGqV6AS5>bf%uV}eWD z>~q@+J?tLAM_@P2p2G8SUvO*m_{{_(#$FFkS^H@Sl^ibee)-L2C_v=ysFr7XhH&wnI4w-I`lQV zO!g+RT&B34RtwPCBAF>Ja(;FtB5Qkd;7jP>8IUKBz8}Mg|Ts_*k(w^0u-6q3uK&e?{9Op6zqY5vxtZm=3CmnZjI=Fu8m8O|^&L z6J#-Vp4UdMJ;wn#4lNJPgVprEAMRxewgn0EM~4OrZ0XgyH*qu;DkXqe?RwpHldi(l z>XWa8Lrk})rsobgM7JbwGwlX(#U(9;@nLpn(^E*jWigJ9p*G^v45JV1inT7J&GVI! z_YP3+;zHua3s7)y4fB=-@^yQ89f|Ebz_mhEm2AmjBl6J#(|Spvu5XNo5zNfYr9_h`=qElFY+PIn4bYI#1#?_ukWnGiFBDwq zB23SF;BrGDp2HxwAv0lt?HKAiK8NC~gG)$bEs(&Dm(i#efw-1MRA(FTgvPNZGU-HH zDlZ8mt4w|CT{O=LoBSHkU*i~mCi~l$M+M#Cu#pLMpn16ZXV~@DVCM*DIt+9U8CBvp zg*q!U_9~Pqxt{uEbnn^nM49=pMb5MdY47ko)PXLA9%ktaUvZsbr*%_ykx#6|2TH*< z2NN>bV8Ccn%ZSfWq&@8AzdVfFLs0GQ8_MK{e2FJnCMet7NFk- zV7Qd#1RDc=Hh)fQ;X=SJiWtZ8-hDtmahfi`eE-XpeW)t>j5Ru`?%fxKT-NJ+H@mc} z6)zAiqPN-QJE#FBDNKUn-FHStoXr1#5->c5MG{hE1rq{JxkM#Z-WH|86@W8H^!l*$ zbTE2Qu9tvE^g~zy?dREZ@5U`xlBIEL0C~tc41RD@FoRV7IHQQt1P-`_`!Z{3$Cx=yLZkLSL*5_mV(qr&5Y;AE>Agxkl0h&mHGg+XeVOjNfj2(aYt7V z+Q4*)q4G+BSV`m%_a%@u=EjE|!lQ{^KDsJK1vPNlgKOEk&x{^XNO23=$5I!e z3_i*M7u__^#pwqV2> zG2ZBL!(j|8Z)*odVsI23Goc!V;_+z6qa8{&-sb#c13etn zdYRoLSsZ)yVV8>l8d07mzAK_&T!&fJlz^Ezujm|f&JX!B8wL&rhu<@+9vn>3!&(X{ zuY=wIWAbi|69tV{m?iH^3JzLSnfW=%tqhALbb=kV@X}+(^e?y3_0N~oGDF~wa;lr< zJiaUFYbVOC4ui*ov;1ZE)p$H3BI|;F&A5c?!)t!Iv^@7Dq^=vvE@mEKi z{>rrwxf-T69f&i0qj)A^01_Bp#2*{5K!gbe!f3whOwDQ6pd=4DTV2UTLK+0znH;g z>n>>457QyAFrpjYiJ&}`nFV@+_xvBXDoGrIW;u{OL5*rjTqrHmkc9(uSMJO?So5do z@}dPe&74R}1rty0ji40Hqgn$dM@czu*dPFm@QGow4x)iH9_U_BsEHX6FukJ|Z7Ir| z>ig^h=g>L{+vJFpeW#{3nf*<(i`Z;#um;9bQw$-VQPVhT)C-|Y z9JYQYMd-cwK3)wL)6%txuQxQ{v!aT;Iv_bnSS^J6@h^6xLb?IU0W!da2-OcYNZwq*yL;J5YgQe9kD8ipQ|=Rsgy*zAu1?-iqCTKAIgl(LYP55#HlKF=lYHNnS$*J!@LSMa`nTsImlUSRLEPW_lde zHC8I!sek%v_?Xn~_GA&MrkcR9tfasMd8LR1)#$lC4vZxlZazKDO+MWo&6|h=0(qp&)EAzcctgH;79p)UQ7v zuX6iBWs*m%LogIAHgr#1cy#IJii#|l#M){?yURUTnw4v1%}hqZq`LL7?)Qa3u?71> zIDK9%h`nj-=eacKHov~wX~#~ucL6R#wQ02@r6-g(@Aub7Lw5`Z6Ylo%df!?Y0r$HQ zi4!iw9E6Xn#JLdxyOtm-&^k369qV0gDiAlE^<;Dj`?_>)?!QwYB2==QIQ-m)J)+T@ z=3Xl`Qd#=qbn4Crp-qWiDPpFZ{#0DYH85OL)b285g97XZqZ98KQQ^JcPB8A&d2Mr?jNJ!kB|7Lc}l!-P2jt zVq-!0dRr@w%oI<&W99U}sM4ZbXy$+ykm)^u-dQ~=IcYiM-XA!}1jY@6^AhdS2pWq^ zNycZVX049z3pQ?6_3zMs6sW+dVWeu<65fQ}z!$T?*g?efly0n3e!!1!{*g{Vk#Po6 z=Sq2(t@bkiwKpy(RZr8&nS&A1+bhp5>D3YGh+2hj5FQ0hlBjq()=~-9fabqHp}J~E zK|huPt`XPPWHg+j$AsXa!js*@MCPB#s;;K9b->vCuzGK!e#xB#xWpWtu@(o4KuOBB zTxSn`3c~76q0x>aGg{9&M&Q^OKN-gHBt^uNHPqk1g#_bm37NX`42+86ap_G&5pRF3 z`W~}R2(xT(NeK=Zurd98p{$I{1(~E`nk7jst8Z*YMO|N$E6bgb4&7%62<$nHGl(Tf zlp;Fo%QvqCK^X|5J2Ir{~EaZ8aCQr0k{&w29 zHUTFk1XJH(J=O$GH0|1hA_Y#T9m_of{eug-YQqV-{{>UR^1XhjU9TtC6%gYPgmXcm z+X=R!AyfSY1t>KqM4Rx80Q&c9#7@hL@h~TSQ@o00-|f#%9~ct^gpPfBlit%wbcl*l zuKKJ?@F;cIsF**D^O-2mu@Ju=3uZ-lGqs>GPW_@^>)ixBGl+Q8V?BpFws9*e@v{0% zZkZlAj;j3u%W=Fce4}PHi?km*mFm?*sB83Xo**aCN0JZDh#J2*v`=Ky8E}dXysI=s zC^y8$eJfJUW0Ui`fp*$cej#Z|o^$^{Ut;+4F?9}4;`vCG9)8}W-FXlYHD?yCg>MrM ztObm2bKmI~=J;RI(+Qnycx2x0@0PA=sg;MWmoQjPqzi(j`yu(5yB4|hLSaVzYw+H_ z$gNJ=SjwjJLd463iosQ~ZRzdsvA?rj6&){hn^T#fspor9BWw5>-EHBB$UzazLQ*^S z_#R^U%(DV7aS9yLCH|p^>?lM9NbBZ}Xo0}#M{t#&Sh-{_z~;vE35Z$usnvvarC1|F z@ReOP)6Tf%<7$6O{eB$BF27B#xlDH2!n|n1Gyr$}hXTv6JFC0IkhG|$YB}N}ok(?5a1Bg)kZ*eZ!-1_4APpmct z3O?AAI}-Xu}0egVZjY&t|HRNibTHkXwyvX$IKa9GWza~+l&+9*4-J19T5i@cQ7D86zSy?T=BstIH9m~qfFgB_LD z1a*&kgQqrs+1JrcKf8)ysWJYh;E)!x2{La?7=h=cE7lWIZjn1*eDQr1`qwbUcDuq! zeFZ_vcOZKno|KSH#`5kQO3-W+T9sCbHl27mcs%al;QroKEz=gwLa?~Fh(Xz`uWY@L zNPrjNsJY_puc7Dr9r!$kZRJvCf953WLGb>Z1p`py}p7dcM`Y}yZ40i7fPv73b4h`8IyFQFk@(+4p-%6 z!E`4PIW`~~ojjP8obgFd$x*Y$&M5W;BhAqR6$!+sl?UUXrB0A>;HMF*RA}WC)3Sfm zfvJ_OX_W9^^F4>8>vtl?5|)RSP^O{R)XZtTRyj?`vk|*iUUi#@aRjS@5}h?1)0+0u zbLQP52`H+lbN@LvXD&dzmlZ$QPEUPwGuA=4PSAS_ocU~DjkR1L2bm-#nN|go<)bcX zuu27@LG%%)7#}W&KLNSA;IE&uz-fQIN;2U*ciGP-W`NdGL=}^bbyMRn{2A z4@CCHcLFJ8O&+`?=u2&es;D*KzbOFWYp}__#(q&EUTe}!Xs`ZCe{btVuT+mgLg+C}E5Hl(XHGFFjWzuvNW+6!Fxp z-W1-qO=vU_{C(!|K%R7?4oMJosung~GcV+lmR{OwZicB^n|P|wO#8F+{mGgK->2LU zq9!W-sQ1{-Ng$H88dj$9sE>YshplXlXZ)Vdl~h4F@LXF3%g-k zptLUBlvQtjo2yw@6(M_oP`1dZ!>ffysjw58O}f2DUff<{dt7p(EAX#SNu4(={8iau z5Bs+tEDcF9JSGHn6Ds;@3+j*i71Fs&5e2}8P8ySM9PgLh)}|;V8^W(ROlHDa4Kwk< zB}RH5Z+(R)h;3n0LRwQ!Vl1Xs(-zz__jigZyV8jPqXg+XfnLllOeKVjPX=-RC+ zq0nLD=+f|K@_i+)f*19u^S_Onk$L!TI3oe$UeTP}dke#V<~b(}>VD1F<5-B0M8{*x zb6u+Y1BLs8N5qFw)=mnest`;s?5ijTa2lhl777$~pZOtMfxgIV165Iic?SAHJAu2H zsMy8eO-{Mqe`B>m?z2PvH5AaHBQ8$O6Mg0dx$JGmHs_>^6IH7*#FC*gvSh;$nPF|S z6a@Df#@=r2+@CgGq1VZts*!q1!10=-2}av(ig4d_|f2l%0W z0tm^!0dv=T1`dsY8}YuqsG_c-AWT~fe?P@f08x9_R&0`H`-95`;PkeR8h}~zwiSfg zaoJOZ3CREi@UyoE>~s!b0D7hX^Y;MzWPs=cfE^$`0pJ4oJxu5~%g^ZR`%HZ@*fb%! zK?BS`2Dc6T92SrL-Z1j?i;O_Fr#Ac!{-p`qGpt@^2VwS%lou3TJCK~|{LBhyWz8=4 zOJ81KgQ1;Npi|OKn0f5%lPxpU(c{G068C&oIWULLXc&NXdw6qNlJ@eXt|INj-t@Gs z&Ste;pk@1NgoqWkP<&2WFU8FYlbmzlKML)(J{SUYiJ zcpaw~5uarq*?uUzmcruG@R4SH7Te4wM=u{IU(fjj&FdEZAx~hnrJmZt&e5xj9s!w# zr+SI23*M=j*O#zD(V@@vyK$nRXM%dDhA2`F6gNWPt2;UuJN~n6njWKjB&m#AOWm9jjP~%9y!a zJ>e6p?!hmaDAsLUntrKrhL?k5M7x!AKkveqJ2Lj?nl>4lc+|5&BJIHQkv}u3g9mA% zIcVUMOZuiLdd$TH?$a)LOGGmhfFg(Q&OjmayDNf1Gg&vgo7*k9HG7-s?upLp+00r! zz~{R4UBD7%L}r?vt7L;I9^cmm=g{3XgAcFu z-$1PY-Cp-w|LWNSZ2VUyJ_MH1dosVw8CMzB{PZrHJ^&nVWl5y2DvP=BO&@+8fR+!x z56c(70jlq(XFy*<7l07|y7>dp;tjY8v=_K4{c6m6UhrL)|FHbO#&?DQ0FmwSfEI`d H6zKl}VPD7A literal 0 HcmV?d00001 diff --git a/win32/tests/tests.vcxproj b/win32/tests/tests.vcxproj index b11bfdab75..738227e057 100644 --- a/win32/tests/tests.vcxproj +++ b/win32/tests/tests.vcxproj @@ -227,6 +227,7 @@ + From 5f9fabc8b5ca753f95074eba49f58f66e211c8be Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 18 Oct 2024 10:26:55 +0200 Subject: [PATCH 02/19] trivup 0.12.7 with fixes for AK 3.8.0 and Py 3.12 --- tests/trivup/trivup-0.12.6.tar.gz | Bin 32862 -> 0 bytes tests/trivup/trivup-0.12.7.tar.gz | Bin 0 -> 33672 bytes 2 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 tests/trivup/trivup-0.12.6.tar.gz create mode 100644 tests/trivup/trivup-0.12.7.tar.gz diff --git a/tests/trivup/trivup-0.12.6.tar.gz b/tests/trivup/trivup-0.12.6.tar.gz deleted file mode 100644 index 5417120a69a5cb6cf73cb0d5ecdc8f6ace6fd4d0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 32862 zcmV)bK&ihUiwFp%?>J@x|8#O`c6D$qFfK7NE;cT7VR8WMy?cM#xUn$Yf2&V{D#@Xg zO0?u_qE`D`#dZ>J9NVv@>~`z;(V;}j=9(f^lJaF!pU?iz3;+_mNwVWjw=K18ERn!q zFc=I5gTaixrmuZ}V%)yB4Ab)e@SA*E{8RL|)!NvMzvJ_EYrVbx2X_01Z}6E1q2WXG zfB5hGtiNK@(4Jb|_D*}N)!Ny3y`yh!@2u}^Zdd+Le*WV>p>JQ$XU&%0Ue~wRe!Ka< zv$chP+dEq=^=~xi+nd{M_-|)>a~tOW<~lrLTiRgJCh~t>%Kz=H_1BwQ>szn%Rw?=aOXdIJ!S4Q1fB!d_|J$3JMe=`rbA1Qo z|N3ToW2d#XwGQ)ttG%=N2iE$Q^Z&n8{_lFTJKw&z3|VblXX~x@b`$=2-C!S$3wIu{ zebaI;ZO5tf9EVZCfcaKn`PY`IS1PBLX$QbVqj_k1E;C${%>#?sE(^T5Kep(}$aW3? zj!is&8Z_9A9bPier@!7jtV})Ap4ejp0W_H5TWn_eQ#%YTlg)hZ+BV@!cxi<2j|D&+ z$GfrJ3pVy#)5b~xRhn91r&4*zk{AWdn@Ci~o(aWZyrBc30VpCoW8_`qD@g^{3+=Jh zfOmFKVax$o2tsYlHB*>CgJZ`4A?oXS7(h!31tbR0HFFMFEkFX=M>HOa1QT5|z43f% zxgk*&U_qTVpt}d}d^R;g%eM_Dh-pj%LlqVDp+|%F2YuE*emnT7ce>Bu`^o9?j|Y4E zd+g26Z18@c?H-@}e0uQi{eZncKHS?s?X%v|9y~o7oF2USG&qI_)m|TJRx9|Tcl0yc z|J%vwe!tI-Puao8lfwf52GCD?M}vd?euEtx?H+#GJ2-mRU;uy}9S_*y!N-FE6dN2j zu+d7k8asZ=KJK6HzK5T^HwT9YgP#eJw+DkGZ0jvF)?+8V)4{>+r^DVUJNa~aa@^l% z(5=eeL4WtKckpq4Plwi_MYjLr{?UN--}ep=b9;%nC;IqiACT<5Ioz*MOVG=`gVX)p z0d^<;whNR4{0I0GH}pWgHb2ZK+8efIA7cn=9(>F=NZ2=cxE7j}5uCu)7_?>E?9Z_uNL0R)f= zUc>h{pZW(xw1cC;{^{wblfl99Q5|~z6OiS!vfG2IdqmyiBkC$p==k(!1d22u;y2h& z@Au*PDH4<@)k6~WfnvJ@rC0^p0|E||KC+|zcZUb>_K$Y=5u0O#^3y?ozYb${(8mG? zyxE_6(B>!VGL8{oRiUp6$2Mpr*}+@Z+xzhV(H4ba?E43TPl#l@?-?hz{@fG-S>Z zYsPFAu$R5rZ0%(wE@a)Rh1kpgD@8@Q{(trQU!MQ`>(>AB{HI+1qxJvf!@K6e(c9y1 zYX57uX#H2uf40`Qx61YZTYNrRp^epe z*c01=$|oGDDO$@84fkRWt+Nh1zXKpHe%p9p`SnEkLwjtw0Tns!A23jzeDB&a@%Nkl zo_w9Cay+wqBg6*!cR-dYugeZx)0$cEj~iA*wdPyf0a^(IedW~p$K3XOo4EcL%{@OdUpIn4#?oH3g@;)FJ+R4RWubL^3CUXel{Tdu^d`NqxK z)IgYPdJJGl0Q8xAQLntL^)3Gz)uCX1aRJYv!TQTeMWAS-8aZ*STYKbK4K{P2kU?J^ zJT`6KSojBC7=8%7a19$e1VF|OHUUM>y7Bz0%Ea@j%eHC;UPWtI`x&>D6X3Re{REv8&tuFgQ^H0{a6@=+@s z`Q8=4zUHbqV))iF!nqIXCK~QKl?r$wwW4tQt*>%-fr(LA*-@YMJYUw^Rtk>z^<+hQN~c8LJ* z`u*O?!CLgj0L2~Z3kqYUF!P zV06}IFpfrOv9aZc+=swo7nVypH9)OYUcT(_?|wSP6TY8%r$+}z?_R!?`x>AGx-`sP zHg`c&53O-%ndqKiT1^kMZ{Q=(^PRiq7da zPj6t~z>f9@KOLWb=mWDgl(K_HXXo@S+@qVF1#6dEST9YXXqT(iN-!X^2l@W zc`h~=8>~4Ml-31+Kqq*+0$j5x*q3eSZM_A*1HV!^f!z`N-)8gDa%TTaB$#@>B{&ut zy$9SmY6kG$;X^pG7|QTax-b~yF%l7`CQR`Nd6-u-&jq|;Q>tS7{6p{Uhu-j{H+a7` zvfVYBZOzb|+2cSDE&;1wJ(z3A`t14NKMZ?&d*r$BS&+Wj9QzhBUXxd%Ne(|y2nlD6 zP59|s5pOgn1`Y3qz<^qWSFAZ=&)=LLe?Sk4<{|VnKo)|rvAOR{+;999j|!&&h=e&h zkC@&Kcf>I*#{!~6Jw-Tj=uKtF`KJSXo04Z^Qq^T0a=q2K(W3AHN@2A&?2b zy@ggm6m%F))F5!=vy_T?4?MZ+;QuB;edb55Y> z-81I-4bH!T7f~qc3XLtH)L8_oIhi|-7;hBeIM5**oF4r6>123%d_3s>pIDh;HA4|! zL#M1uSYJ?%prl~qSPOlyKdf?ifs>3L~f!!j5KL`d$1vd|ML8&JpU;_zn}eoV1=lzO)kFW{=e03@1)Luwp;6^|Hn7^oK3|6(0RqQ zg0XMUxQc4SKDNsudr+lvW?ftiZFl0GS3=`rIGWpz2_@)ne^|G%mI zU#$PHHcI>dfBE`9yLj0s0zxo)gHI6WcY1(yqY^T{zxIzQ*sB3hEW07nt@qtko;VYz99x&T$&OJcDQF z@DOh}L5XMVEr3POiGXaJWUM#wJ`=*LX^Ly=8oH)xb#4KPLH)VQ&Th|cbwsbGQ9&ql zPG+jx7|;u-udx@*tmha|?FqX*Yqp`*p8(ze)6Odu>VM5OKu${hr7c+4FuNL`LeX-Y`|1dgif2-9N;0^M8>1rvDn5cRFen_4%T`lVw6BV5M@P7H}O;q0D z{wPD#5QFZSCOlHKb6G$#MK{4BwR-A3Dvd@L|B)|*p=&72W&m6n98J2oXsQujlMXJ) zr?GpBCLTxFjU(@yB>d_M-a*yI!z!#NE3D=pl&rJdK`1JJUVqH8UD|(2`%h{ADfK_D z|M8Qa-(vnhEdI;Z#^!n%|E1jjm-7GH?f*;vzf%4epY;Eh@_%zX75@ch|9a{F_f0;} z{xQ{fx>h2A@2qpv=L4Zk&E8Sbgoh0lZB_I6qFHGezvZ8xlGeRSoa z_%34qTUYDqD@{?gx4cz2&$2ju-fnBV{VCR|ya07*h-c0z28BT4Zm4qfcD13@C z?R&FY{Xa)Qmg|4H{+H{2x&CL?|9;>+)j`0L^}oHbm5%=gZ+FV||97(fOYeKW;LEjM zErE7+9T=uJi;D4I6-6b0CpBTOj}-x_184&%vKv{vwQxFvKc+RBUvvll+!9OlGjae& zz~puvH~@mTR1+{oK|FSV*E;vj9bQ>?uyFfUO`G0v4M8mHV$J8pOeV{9*oy$q2Wm#6 zE+G0zi0E>T(8i99e(*)mBp`ngl)!Q(fqmgx<}d=tYVsE^u0B$u2(ueo|>0JC@^uN;nU+RCc z{>Km4e}nn|K>PpJM!U5Cm;3)x{+Ijz^7GB*|8R(Rorl9GG5{=#|Gu@encn|zZxpW$3J6F7&`tLd&1!H&)5@&j|bWE zqyglgv8N3o|BO9J5c$e*i2kfY)@5hpi;R{{d77%7|1Pww(*9H0e@gpLssAVRf4VpE zueAScZ({zZ(*9G*|5E;!^1uB2E95`<=)IxCa?AqpE#?1ud!xNm%Ky^-U&{Y)Xa6Vp zzx`^vRoeec`Ja{lrZscCJ3Y93a{0ft1M^?V|LyHI=z{AT?XA-P|6AGr(e^(wf=jac zqt%c`n$^U8?7Fy*j%MV)N1-s|kg@3W$8RH3&{MQR*9p##`=|Yb<0BSWQv*Z21|2jk z+Y|O&zQF*x{@lH4(4hct=E5eMH^x1bu{rk+&}7Ig2MUcZt??BCkx>Pekb~B(9fSd0 z1CUKb^`he5WMYpojIXe?Qy?R5w?5z9OjkfU;sdZ$$*;O}#sy(!=A^;r_>y!Oz3~;FQHM{=(ygw(FJ3;qkk-2Z#IJ=e6lo zXiaDJ3Wxj*fj)o9jF3IAVa%p_m3_yai{eqCxHB>I@04{_*A^yYLOuv2NsFeFRiV^iqU^h=z*~BB1scJobn~_$Ir&i}3M` z1=|>V(Qe10qkg|rKvQr;|}H#4A>F6Z8Dvc2M8#7 z{l_Fur*qdulz6eKaMIoNrZWs6hS63lf7I6`t$#5v>MsTnW6fI;DEF3Q8-#a`^$`!; z5>E!AOfd7p(flG1CFD zVL|1p+D+a-5n4G@1Sv?;j}tISur!>RH#4)GnuOY)V+>>r0Lukjs4&WXWt49AY= zTDg<>9WoTWMI=90pObj|T;)|8_(VX4XPhZ1bS*rO=IG}t@7INZgt3wZ|3?^l|N3eV2xtUnJV)uf8-64XyKjsbb6>l zMWmt^-Kr#q;g2uvYfI;Ai|+XsYj^_7v0QupMfCliKNzlqV%LnH>fyz&?D3%5DNxW2 zVCz2)PJc!u`(3pwl|RZg*D|{;C&SBsn!LN` zzQdY_tQi0aiFmMvj?H-KMwt(Tz;1t?uqN^ssJF9b903)G$E8n!HWE^3zyE zYsep?HDA^+96-QSyMaTIzs3uJ6|F{z*%8gTIA4g$xGAAR^%YH_vTU8>dZ6&fh_yRq z&F6|R$=*zOf) z%m_5F1d686w|EB~Sg?Ww*{TUakRx)r1`Y$n`<2UWtQOo&srgV z8F$Xabp-wSM8coGHwkahZSt)*cTMsOWVK%ZZJohSj9mz7yhzScc}6-EIRU~W5h0Kn z7qA5KY#xkZk`aV{bUZlN-DebEI`SM8?t}n`d>AP!2PGCUxeue^Fx)iyQ;vpjT^PQ} zJv^l&B&bN1A6(iq^uxn!O@OMzB;a7FP3Yv$-dyxS>8_wcgNbaXY#qelGgiYQGIC$F z{uj!NF*WWedk7k>V#wC**qVi?Q=x|G*szJQ;%-GP04;tNFuW0Ym>Sd#RPK~w%m~RQ z1s_z`L0iAMxluKB*|#+jfU!vkLyb=^=5UIWkBshQVxkdX=@>USY&14B;#Ke-wL7&wE4-HrbnA|Dcafc|RQt-lJ!o>TO9*ihBp>4@f?Q=z;g) z;ApSGESp?St=k#pK=LTWh+~r{al}3n0m|ZqDRE}j7=44GUm^nIg$LU-+N_e3a1clz zSb*Z#Q#%y-nKGTyD-|x|e9sFHy^9OW@9yot`ScFx5}N3VEZze(G4vwF0>Q^!7!5fv zyudEN_Z+Jy%H>zn9pf&y_K6XO(1SyIy|6AeB*Kr1O5L*;<~e&YV=tzx_F`Or;bxE~ zOR5p*sd7vK2 zu0?&)bE+?t6mj`Ha_2yM6uArGtVV87t>?Z=`kv!3%z#HBS}21mYUrXlhqNcY z&g$a&60{xMxnq6Lniz8@+`op6!Fy24oLJ3Dtjl?zcmNT7m;QXhfb}a^1!ZvZpu+lz z!21M#;MN1w`RDT4;1W%#E5O6iIcOb()>!}bN}&JuE8YQKJ*WNd!O`2}G}t^mT`R=B zl77lp0bFQZ%X!|#T&E3|S{o!Q(4Okr4)2x#nP3nW81e<}0y)q+?)hE3nzNs6GOw~w z?LP3{*};;!8GJMKIifi&DT^;E^ADm*s;(eL9#2LBOm-wI(ux{U>J~QuKXllR?DFME z@J4EJDrz8AH?I*=(ezjsyofuJqrGJt%BwF~hu7tMWJ)6mvV!d67Qd=#jKewX?TtA= zV^EiG`eQ+uM~lk;X^#3gF|2g)5vFtwP z0AN}Er}ifLe{St;wzju7+UWnkjmH3`{r{WT|3!GmpeI<(hAPP>6TPzHtxv^cr$OIM|>7jV^{k~$D*T_!+)KW{tkB3TO z)MsGNgb=)oA$*IU1&N35e5%J8*x?mC7{(8v*?H)RxP`h%_HMaoA;-4i-B83dglj?BRv`zYQvZ5jgtPLbc2eri!8&*Pz1bg89Hj(K*a~eIDQdeS&{G zhB>v}ih)rjbW9q4=ed?{PjuJQjj^Lo@Zg9Osg$#&{#WXMzjyr)?|-)2uS@-})c-R2 z-`9i!Tde+LfBR~i4SjQ>@}|0?5uEsg)B z_s9Yn=pP^K?J700Zv@gfrSVLU z7aIs;Kt=qu%b_P!l-9caayOTCEC~cfJEf z$F;~qe=ojc8eKyrFmvwk_Tki+gt%5%bp*r{Lt7IjgPGPaF+oG-1fXM@N%si0tPKE3 zUj!Bclj!^RGs8EgQPK+h?9P23q@{e8DGH$Fqg}F}%z3&gN`>8Efn_mJCSco~t_ek= z&k-&~bQ2-;m_Ns3*Gg2rKc69iK@d}8n7E>&-ZptD6H~LMAgTNfz~<=acmqo}zx@ib z9vvFprVDF;Z;y-eGe`);A^b+i%m^{!6S4Sb6BmT*61DHzOm>oPjR#Oaih_%S^9RTce)gGc4Dgp`ZkK3$Q_%ZQ+G_L*&UrXM!NrFtp2G z46!)0PI+3m0|Gq}5?0vr?`WtyQJ2!3ma&Ke#C;k@M!%^!(gieDoHr^i!?bg}mZ?n* z%1f1$Sox)yMgynCI8ZsUvJU0i*kzc$54 z2y2?I)z8m58|U%WOMYwNN5h$0`@Zk_Ie27|rvyz45q{lZ_`mZM9e%xnmohEW#CFXg zsCYF^Rp^vePL2uG0&$BfS`s_haZOn71N*MKuD@b8urW4V(v)CVOX^?~)j?@VV1NIR zs4byOtpCc+)*I|qgSF1HYm;xFdb3Z|g2G9Z(OeOsAw_6%wdPG-`KY6&1|P4&c8be# zTB|hCgapv&77E%@lG&_s4sX&Mr#m=dXhP^hlda7LQZ9Dy`3EbT*QK0>B(`7mpX~& zO5HZCAcTQY28$jm(9VmhjzFA5!sCd+H%PIW~>O$*~bDG== zrwPRt3k*k&H_8er{03}>QlqAbm|qMdkTv)_1rr4)DNd}^&agr0CUJvFTR(1tIzBwy zKW#889P2s2BEdxfip43#f{(VBT3s&C4XvIP8MG86hbbb4@a?W?GwzJ5`<`P3U9R3Y z;)DMb*JbOg{{8|<9c6aVjk}!!lI&~*DYtx<)3PZ1D%Vt9h`#QZdrjvTKWmjGYm6c= z>-B^VNSm#I8%*LALh7YBNN2Z^YLy0?(qD1dV6+M*Ac{|8Y}GU-%EhCKgXvfZ~%3DM(40=8=^0v1YV8Zk9}Hbx^W! zH2@i5n$$oNua!{lAEtOzkUb_I(Lbx;EusO3nw{fQV}|lBnl6!#j`W<-p+r|>e`cQ( z?Zp$nldv##q?IYAdMBrLvP5K&>O{E4sDeT~G|?jKOHq_o=3y8}!$ci6*$(bGmh2_Y zTIWPgb!_7%)5H1^$hCL8`(gi7k+WWGWMh(kOwuoZ$dT-S2D&#mu$YG0N}@9hCl*+p z6wg;Gh0;m|qc8`WoCDnb%l9>-o@^hj+|WKd@bK zI7kIz(!$8j1AYM<^<#tIq_5TW#9mtv%g||k4{)zkQx~S3t&kPsC7@Q~MOM5kMHA{{ zk6TpG@=InD6ug!(zD%yYQ3WhRBD!|lkf$1h@$3X13rLYI)?Mp1wOnx1e@e1P zHX1GWl~h~n^?Fv?$EyVy;Q@iLmK@G{1C8?C1S{0f^EN2d7;3H5RhUv|?QM!a|}hNnDFWxBPTsF_1K^!kVzSooAISJHBPMyJi)!Ir|k;Q1fZrc*`O5@PaMS z4(dD!GO_J!WCy9PDvfarC_(3llM}0AY<k+^p^uu0mabG&8kd$@qB)gN3%BNiR1J-9N}|U}-6Q8j;sKk_yxOUKU}Q*$X_DtD zoF(Xf=C>M&;A57RMg}1)h6}+?aYG}T1;QMO2wzRGBb^U-B^y3 zoQ8R{iTFdTo=78P5M>J03wP(dOdeRyAFnf*%jF%u>dO?$=Vv)Y+L8~ax8N&lkA_c7iCL5%!H6oKk;lw zOVQgLUDUKg*`E&MW1lZa@~ZZtQDA8(l|=^b{LxFxd|+5}7+^RVYZ%}Dt3@hZa?8DR zyAY$DI^2x<9oJ4&jO-E79HgbQPX-qDMcrfMZ;;7bpC6x^vDbdkEg)24*yQX{kgKQU z48M=f^-gZ~pi@^}=dpJ1Q2EBVtEA-|0$a3k6?DZnv?~O+G+2Th%n5Qq6*GZ4QdG<$GfQ2 zG`y~RaP+PN+O^uAH2;a?^cMk8Yl)1~tlOD2MjseFT31id8#-)GVIuM|uRmarGgGvt zDG?Urwe9KDbK~8S+_7T!0lF964pXcOltrmHpjTz zEN!%3e#8tqX*lG^1>>n1>6NvmsB1rb{M+#4_;k>%R$-!JG;q%gVGnA|#9;$IT|TR{ zW|RD^001oT0{2zweTJFXJ+nh$6bZVUt=HR)Xb4^e=byFJ+DAW}@UDqFenHsi8Tnvg zOP@88MYWZNG<}}3nx3{)=Uoj9uA{(64K}aWD~sjBS+jjkYs6o)O4^pBan}xQnNCok z-&_f25?$62z-ild(kZN7Su!{r z9n{bP=*nmouU4z0_Hcba#)|$HNEdR!Fa#NIU=u1LGGcV`5a~Mg$R>GZVJZS^Y|K-R zBQc;G3tE(>N(v~1p)sKcL6d_I#-RH8wx<^WK--tWX1?u??U~`QDc(_zJmlgRL3Fny zA5P>!DHO3P1B@;yS|owTLiic4JhGL_-nQ1H85&4nr_gaw& zrY)rtdXbX4a85<__0ZD^uI|tA!0`0b;eO|=%`f$y*Yub5VD10uFV{E*Ai)4t*ZYUj zVf$OHc2+uo@C3O-9I2e_G;+4UWCI7~FXd24*M(V)hPwOZzFw6Ff1LgDkN+(98aH>a zkve^|HH2OB+k1hdCBXZTq3{uQ+h2G3JrBz5XM=4vi6g>qCh~W7Aykhs$w+Z6QL6kY4?E_ zVRSF13%!y-%FwfXWQ?!$OCIS?r;9M%r8~+5JbY-G`f(4Qy`g768jfw^3Cga3F9PW; zW$35qn)=|En=6c)Va_}#t=xv|8vUjn6!(a|_Y3|~YFNlb5O=)S0(J`2t<&LecF|#{s zYr+wIDL%zIZaiyc-`NlS<0DWPN1$R|S#FVH#iJ~;05meGBg{2i@w%cB}OeJJph@@nE}G1TPY|r|Lj;tBG-|K9UT7 zNn8E1MehId+0_cq>$&#?q&7s#Tj#4b6I}m;#XIX%CokW?UI;IBvYHj@ zut{6peGY4(+Kb*YfUx*Jg#RIX;4LuD9kN_aDzW*y>Gc%gQ>H_KS``@VAfq9rnZYcHnsqY@sbKcPK{>UGRf8Df1qgZ$MQ^ z`cNsLWFI}esWr!ZKH=O2N#xw2>x&j*?2{}6c4@)5<@r9y4$`_Oe^m0tu{?1=<3{M$ zWa%Uk-(jQan1}h^9+Tn~cZl~yhw@$n3}X6zCE&1JQFyS}r3;W>4T^K`ke2#E)CFAg zY4MIdCh9sURY7Qnb0`oYP%ktSD%3qZez*Ap&zV$1&r`}Dj*+2(i-d^JUIENmbdz_y zHh)=tNYc5?RFfBc#g28!DSDAgm$%+i*H%$mI0lI;FI~j6v%0>b_BrkF5}^eMVOB`l z!RZy=D-!mnln6PU^9V$k7Wu-**^G0=1BM_M;@!}9n!A;(-I`(m@m(IB0lQw4hhspi zcSorpFW+N=CQLa6a~uKl!Zo@eAy@cZU#IA(QF?8P#h@myEHw%yX+0mBH;aIw7DXE} z-pMF-i0U^@lChKHyAoQ z(fJ?wtM_(zaI`;o==>M8`^UQ>$XTHN>uW@0eNl1?H{0y20ZxQ7Z{^>A8IND!io(xe&4=#D&kdQS)_T`>6 z$E@hG(9#=p3H5$Hx6Q@^N52BDB37VobnsxEPOzY3vP(Fe9Th|9m!d=X{MKIpYN0iy zcd7*R(NQYpS(CYtEJ%n17s-J;j9d%&@WCvmG{F?$Y`IG;;?Q8e6cO==Mn;rj?DQDq zjck0W4kZM~PN@7dSqEH zPjc*rwt+S&VklLqK}7@MDEP2i9$+reZtAojgVLZp%*UYuDvprAcTq#z{2tYo4$cZ) zy*&~&)MaPM`#fLnzh89E?Urw;8t`R-w`n9p&|ef!=j)aH3k!LMeLjHta7gy);jo_e zHdVP>=}VHMF?^!qB4H&iCjMjY`SYoM?S&YG_{%*i zCPie693B&R)pI4(GRdT~er!eLxw9!6h+MmpS88I1;ey#x&nub~l?tAskrayJDs%pA zVKKa7xlfKQd_(Il4D!S*0{Ho$2%`5g4YY0 z@CC{aM;MXJZCO0Ts{o>qpoM|~rx`3%R^%W7OD%rZzZXnfEC37NZ>zV6wic=?iaJt+EL2iN49+iek#tFm6lDsrrkq;!{L5o%j0*9+ zvOw2-P&|s2FP^BYS7IM`488Jg(-Hx!#2E5*cs?@?ikgeathC3X$iTUAAXw4;o-D;9 z##>#J{vbE%;z8M#PF;b-+l#LFID zPOZ>Dla-pgM3fy_Uf!8rp^)c-?-eA0di8~+R0qq*^G@p)ZTzap)-r2Tx zURj&1Epuz<_2$Ot^{dy$c=OfHXlJtdy4`*~YE3qc$>uZDo=lonYdoH`M_c39+neV4 zc%!v3+BUXd8?Q#MM%z2)WXsqdO-6?G$}qOto0F|qEpuaQW4t+r$FHo7?IlL<{DG8t z`;HnaIZ*30}K-z5LXt7QI;@;@v8qYG;*r~OPt@&OW7uB@R@j)v&-I>Sy{CgKA6+gAHEC`(d7PAbQ|ThG79fRy*0NDbwR zuLxw5U*;a5QDh5FhX9qbu|X*nup#mF9<#6t2YfXS23-woFbP)PaKv!NcvTROHpv%F z6Zv@b%9-g+4M2*1g08RN#5H<#+@TBlk^+hfDJUV-$8_iAP+S@x@rT+IVJ`c_R-hN%?e=qaDm-*kz{O@J{_piwR9;Z!@lA|Z1 z;T0rDmw8q{?CmC!qvIxw#Kp+sSD)yPipM86EN~ZANc%-yf?Ap$opyS^X_j=pndP5T z`i}d}hvriEX8gKkA}C1}sax&vE0=h)jNdFzmQgq0P_|0n+Fc-Jn{ID4=)pQZSf>Y7 z)>QZ53&WrcqI^75BaSnQ1M)nvolatvG;&lslq^0nzM-HF;=J6f-D1;J80c!s1dH)1M%Taa*4&DoP5&^0gT&O<{Z6nLl>rk1HmJ4!w1 zyx>=5&4d$s?gfJp&vYp-??l(O#ZjzQknJKqaLv|Dye+s9Eze=IwUu`{cNEk>wkr3o zJRGTNM7s9+rM9T46q~QyR=OTFw$Yj!R0{ot`Lej(g&u*1&5-F(j zf0qUBN{}tpDt*&)z!yZVeGn{(seo0SGgnznh@VsJq6|#>ik^BSM`hDAlwm?vrEBs| z@k&Yum!-HaLO&_$m!YX?tKg++%R;^6W{+0l_pT6FBwA#(#?wW0saIABd7O`PuAoN; zn$^1`{Jy@vN2_%RG}|F-%U)12 z0W~hs4C2ca4Pb@qdt}-P=msCK1PaKe5{vR%21Duab1Y?H5J5!-zD%f7eAOLyKBqDQ zuj3%5)o-v&5%TQLq=kKsRg%gD(WQciKjEv==C=4iJO zgOE>aRq<_2V)H%mcQvE!%l9cj@^RiS>w7+cu#6US&=!dp;v^nspNkX}CI7&%{w6gALI*|} z$iiSB;lM z6<83oUVqrgM=qx7-XzgRgiOo>^+f~zAc$3=sZ{Ha8+9bJ$(mRE$>T?yqx7UB&r!=8 z_ERpWl_{MT;WZ`KfDD08Wxfy4o;>Dp))Qh=(cy;SMHtjT_68}5j}k)Ne+}LZQaLZM z5;BWHrXUz_6UKntJp9O<)L_?!Gbf%(9cU#4QnYnLm=vBHkssf6RilY_6Bb`hYC< z|5)GNru$!8+fb;z*@ANG>zl3e{@3r#|6{#fx_*?dAEoO@>H1N+etd=N2fe}IP9jN| zeEr4AQUOR8VsND4DkZ$FE?ZVZWem7;x9K>(w~Hj@8K^uAqG^1KfDHp zQllFZqL}>6!`_*e6B+{HOV~7-WWMDk554hKMMD2200AMs8~(-2@B>T!_Bqx;rSfC% z@L+G)2i6Dn>#{Q~gy(qdiGQ1vLcM!esG6t#Q` z39~`QmnChU#-A=uj%}21VQ$iF(_&oJ`3Y*zmc!9J~*q z%c~pwR$%k(1(+9=TTWq#Yp9%{=XWrw3)HcvJYj7&81=Eal92EDISV%YUAO^ z;=-|~MxDan1+YL2-|Y_+dk8{-hrs^>41L@8(7thFp=F1b1?~r;_E6NfP6*Hxmx z%DC!GG6Ua0l95H9f;K~q*v=(whSvrxQX|LW>piw>)7>PVP2Mubmy$(e8`7IOR~vlG zv91lw5NqI#a=-we{_xX)V2v^6S`?33BvPuVemeC7ht}ItBu0GkbV0zVgW3>F|Rb3E_xtqJH`m&vgOJ&j^y1HCY` zGn&Sb>ksz0j${XzW`lPq?n0jua@-&RU_3DT9Unz>_~BE7fJAh}WVYlXU?{14m7d>t z*a{x6;6vBCQK8sV5I8n5aRLL3T!)l<_%#3h+$tf#sK!xNR+(u93*&@k2>c=F3=;mAWw|t6nYR+>(*EJZnJe5GHy}_T)H%F+YD$ zs@HN-h^f$Ja6nwRbjLHypccQ)%ReU82>3U*UnnSm=@Y@Zxo}JoLj;nhx;&ILOhyy} zwdxuoR;}lFLj;(41?Kb&_{6HMw_5drGM8klt7*HK9l{NpgFCsT&K$8f>y(^*hY8_#_w z6+yQ7H&m8DLB8Rfz5U@o6E(T0^S`yRFsV?5&Murx`kIb~HcCTrX_Z*FdyxFN&l zEf;r)W*!I`Ns_8{P?ZC5clguh#m_(e@XN=+!LP^8$6tTmIQZ4v-T#miGPj0j`rDi0 z05v-G6;bAQVdHW%7#{3(#JB$b?&G}YC^Npi2M(yZ%bMZ|rj zu=6A}%=vT{;2|J-{<%oV?q*$~M|V8@{U@Y4sKTzynH@Lf6ln;CZ~G!xN) zC;%@Yp>?8Nggy_)u6I+zkjK$}qGlR*LAQlR<~*IJBtHvmFaBt$~MtE95*S}m$^?L6Gt_3=8UqWiZV4Zu}iK{v=GR^KjC9RZY5=D>b@H7Po3{(yG|MOyT*sZQ5iB&1@m`|KUq# zvEsJsP>7^exX-|D#o4Kq=C#Ej3bCVqhve|E(s+=^#BgL?ptCeFK$q0?6-g1VW1?pDN%`0Q;y0qpzjz0T)=jfre-&rZIXp0_B&dUQuycg_616jFEWblb;Jf z7=Mw|Gsn#_`YsjfIjUcAk(9oi)se~yk5-Rnhal-l!#%Rnd4;|j2F69SQ zaXd>?O>xImR|lT4iIj1)NY|2p>q}^mV7bQ-ApziN#)Z$e2$?^@a+aBRE1^bk1WvJJ z%K}@DWMra9D<^qs$?s{yC-v>5U5IbIHtQ?x=&10jTe^{9@@0Fe4FA>?S&1YYz&9e7 z%#)J_G-8|)|DJ?twx5n7@<1eO3Cy(u$~8Gxp;qWA>_d7hYZXnG%pGk(leQ;rb*xjRHly0|Cq)WmjbTJoKdO_C2k+a2e>s%JuOr-O+al(Xz7~#P!OqO5suIg8xI|M@4U8v==%)@+JP+N6zvZoqTWb;aH7!iMF#%WYhR#V##p*40kvcsdkEK3Oler5%hH zRmy2EI)o=7xW(c9qCeQPNY*G!1H~5bTDr8JW;uka6K@o;ZgGGJDoE#xsHivSw!T5F zvDAI0MX?{AFzAE+uy_ucmP^n<4xPVxe6AygXag@ zs+b2x;0(Q(#fJRkokK@{(^je__S3d0BQ|V2s9s>fPMkBPW{k$G)#VOER7IU*a3@c^ z^<&$%HnzF3ZQC|BwrwXHYh&BCZQJ@i`QNI0pEuu{p6M4;UENbPr_cFx7+#cG1=yrb zTY6R84!Asme>20*w#PU6`UT=BFVeQFEEcD^yYYi+4>CH6a`A;uO!+_pYYhEPiL0O? zUWL<575s8?uAv7%{wY^rG(+;G=yBaYe=b5l@fYx|e5DFe-a(dPu_U#O#R9*yb_do{zX8bZ6%%$s?Mp?%de3c z0S63AFgS9_ou|F&+AvmjPwHs+%w*JHroH%$e+ZaTD_&ZsjwrO9jGgd&;1^s}$y3 zrc9^J{(}ZCI-7J3-%Puz)q&ddos5QhKyoKW$Yf*d;~h;dvd|tN(iY6?6v#+>jL)y zhz`olXUQ2xBy=s-+iCbrD91gb=9J-iGDiMT}4e?$p zsyYROMNADD7M<_1*@iXFqF`ji4aqQiEkhz!NV-luZSXi1OJow9pAu zH3{4EU>h2Zk$3gK~)KaAMr6Z4n-$o z0km_t7ZL#nM7c+kriFZ{1zMCsv#unbkx%sz-^jLH5(j(xNVy<5*K0xw3L$DyU1UPG zBys@n)#~*p`_SslY5{tEc#sL4Z79Ua>OhqRy&z>FD&(gUr}-xMkBZ8bq93t|wMJfed-@qViGuo=lPo1RS2l>esCt#w?W{{5gL z@2@4}=u|mC#P^RjDp=7?EmdbqJ6K)-lmXeAPkd>{8|f**!0jrD-zxGb52g@=j_;=F z5gYgGSm)^2)bUU*fXfP$0R$pC?i|>+LBsFm>w9GT)&;NyY|$@(DDb%Ld-^2wu|?R; z#ZS=Nusw#9;<~EtS|FvD^D^?)cU$)2*n=w{uo)P3YYwmgF7Eo`$lG>hGR|X-W(Y?s zsdFrj4T)h^FMzy|I{O00*Yr5#DT_6mIXDBy_dk91m08RWU=j>7g{pc2^}!0j4rgud9Asl}FW;NxL#{?);@(5~kU%>O~zaa)bu{qEOZ@!cE+m&?Jyi;Yk~Qb20c>Jp|yZ6 z;e`dlJdu+M@vkvZct>}kL|B|vQ=~v`a$FS%J2*Vs^*iO-_MQm=hd3-*CyTcYU94MV z=}M!8!h<7b8kA+Ro(~s2!xS;`$>{z!z!L>M<&o_`LB$9j}Vz6oUFo)h*9TEu3{N6h(NUwk9F=T#9a@3WvGUA+Of$_LIz3^ zoO#kqQN!VkiV+b)|G1wfF{9a=ia#2L)zPl-W(!B5OAUHnOlP zSqKhT9U9FVoeYLSG##xTRx^1YlKu+4@)wl*8z4#f5P*??Q2>pbus3+wzsXW<&6>o| zeJ~vbbXzA}F66AtraZaQDXZV5&D+(#s#d>XS1~hZ!l8jW!Q{E86gs8g9X{6?-`|vN z<8zFiCMnLFP?XiyTi?9#N3n?H$)H*QN#Gsy)s@N@um8pP?hC8?hG2|Z+!Y4Fw1jq0 zkUCqe(HU8?8C85gXV6`#w0&;=}6 zUtyjIkH44G`JFm*ygZm8lKM!-M-N&bO?6*)>Zc8(9odn`9VJn<@$?}zT@Y*_V8Y6T zL#CuR>!y~C8LcZbfeN#i&`pRnWd#oo$r>>tR~3<9oYkUTy;22DWe~5PD@pSm%5Do;9VOu;=DH7=B5Lk$ zlskZB`sw+)mwQqe1baY3S@XLtXE9YNX#q(>NDUm$X+=beJX{kGsl3D`%N%PtaW_KlDIj~HnG#b@PTS%ONp zHGke&%J+mSky)&h8a3{CIabePr4%ZeHZ);r)bQwuO1OSWWgmM2#)A#M2kKtYDV2a> zCoINJ!l1)!4f${LC}Kr7TMu_FQ`+>ZCp>(Se8npFa8BIJ@QqdHe$v^Vc5m@7IXrTq zCo8Q+mk7qpCI%Xt>^tx z%_2DC`D+Y;vDA>{`smU^o0p!s!bUR=^sAOVkcF@pv$65=Iv#1CLu&~DRy@f zq(p401vYcR{T3RyG15@O7WZ>cMu>i!h2k+uaIlIS6zVOJqbYi;c^<4H>i1!L{_=*KeYL*<0KIbLhev8=;ds?D zoSne-$mCI^*B7~2jlb3AAn`R-aOTZh=eZrkTLfXvxLVz#t8?U~?7o7q8;>rG4|uM6 z$Qstn!Q33*5hlROq+s9#&&=?fW2~!VVO!q~&+WfMgYWbBQlM_k*u9#!n3$^-N%Uao zQOJ}Vi44vl@kT8+qxV@y#w2k}KxMPgl9(YPpdv~B64y1Wi zhocEn4fiFl7(B)&XpDsT7V3&Q5(FHRjgVN_oK+sNe8CW=T3saMu)u77TuUY(xm&rp z*1v(d14weLrQ2qVRc*xzRtc5%tfCXP*mP%pf-8N6O{YypZN1Er3$pzm@v*Y)QRe#O z##fSxL`?-&`xm0e=^<{|ngY#}nDVvIlNCQ!G)%g*Bfyg)eG{EqCCLZg0;fGSqcQA& zNOT3+rF@?hC&{Wq0%;$0hwxY^i+_~bynIK~yBKH%>N#HmO-=OGCzVMhCPP9g z*8*7H)Md@s@9OfNZ^tFG3_JYofNk&G1<3&0&oL^GtxbJ77IOU~e zFW)7`9BoSgCJ6t(lt74_C}SC;`w6`G8uN5DfqtevpS=>U)_u1EXXEK!sXJ!{lbE1h zD&AeG%iI_?P4@(5OL61ayT^y+miw^UQcghheXskjik@OSf?t?%+afZRwdi zyj%7Js5sEJEJz)HRHzZ=FH=6&%PAUVXb{q?xe@e$dZg=A@*%~guzsg>e8zwILxCuz zV3>JwM+|(DefXy*r2Q%s<&(7fKEp;YXOukY%;N<_IxPq4X{k=AzwJ5T4QVx91GU9C za#Cuid9b!mVA&J=J*RlakS}NU6)#l}*6m|0c@cW5RzR9<) z*(UuHByDeYPe#yFi4y_n;PTTDqXc}`-}&@Q0lI^JtE-d4ei{r10zOHvh_!{Uh*#BW*5(IKu|M)zLw7@) zu2{fpZ7kr{eWs(n3VVo0=i~f8Y@I3+AYkV0`PdEjdt+^T^BZ*XoBan_XZjh&{4M_r z(4+X7#|#kv1%N#Q)ZT{?>}akCTSxY(Ro6xVKo|V1c~bt{4B4hW0Qf{9nt5(FeEkGV zU#A+sK49GQ2a$$u(cE)hQp`PAjid1hf+2=`EN9Y<O9oSvq)3je{ln>m});C@4xAHeCu#OkqTqlMwJl7|$gF&cMADVOk z6Hh%qWu`?J&_fTioA~7CZOoSgsO)N6qhA5o-~BoOq}&0dW61fIW6+x z-b?A{!3zL|B+-;I`Ky28qFYx4-&jRP+DhSjTb!`?2y*24+=nOWnrCh%3PkQTOe!tz z5jf7Gv>JFx{|s!j+fy4mn3yNSh%h;Wo50bXpyd;Y0+h_nJ6Y0s}p z2GOaM#2;tKJ^w~{A-4X3&>3laml0jg zi*6&u#2FD@b(f*xTL^1N*26_1LacAsSN0i$&0d;f8HEUU6S%LmDO0UM zkiD+A=Xp&agGX#7jx$W0wcIu1SF9y$_6!?2YPcFhG;|30Uwzrh-%i_$PX@=UxA{d& z7s;%(O_S9im=%0pr0^lK_G0xHH-y3Ha|ma#l(bNd9!>?(GKEt0%rg>}KkzZXzhAYV zK22Zp6~6j;z$Ou2_J6WI1ynxtt*yN){|mhSKuy%w0a73OmVgiUX298e`U~@RNzayz z?XJL;Ut5XlyIB0|KLIlCt;U!?Y;#@0Uccp{~G>#_?)No?Z@+hmw$ls}4`P+s@NRR}v__L;XJ-px?8PbORYm z{JfvPU#p~jhSQc8Ih*Je%r?;rjw{pgaUX7-Oag#j;t-$R8eDld@FrjsM7axht<(k{ zt6i^ulLz*=lO6Yp2>5bi=MoSi9v1cZVLd-TON`e2tVCLwp7qZA7)gyZ)9oML6;MFR zCl@c*D(*M>SFp_k&)fO`k+tOKtXl2Y(eHXEvGlPqF&>x(gb$N3n&t%XeYyG20yvB+ z25&yX2G&+nBv8eQ zEtW5EXm@mocf~Q1-hMc|yk5=q@!TB@Y+uI@_U`cmA58YYiqa>={rHWx6_d3vS{&lPaJMGX_vw;aJ~& zr2*-3hd@@g50!Z)L(9B^^GzI~CT1y;9F*;%HDOK^*Kwk5ILXvJ6oh5aPNJ&mdvWka z(Y<5q;x-3$mP@~o(Q+1>EeZb3yv^yS1-*Eu>H5nSJfM$Q!aV9gzb;zdy6(ZQVr6$> ztCv(iE|<;4J($Z0ffz>)Q;);V_atu|$3T~Q0c1)V>swS{K_FI$oki4{8{r9g!Cgin zUT1LqUn-ReJE&#^AzzCux4=vr*!w6hwEIYlLwF=t!ve3@O}iW=-(63$K&!ZGTZv`I zmlM2g%t|~`*iIKRCC%s&vg#$^&kDFvg+c*H@1CU?n#7kLApXTPraiF`=vcG_4<%h2 z%g^Nwc80Y&R99pexD9h9wVu9KR*#zrdI9~b)609=!k4(KGex83mbLb^#u&Rs%W0BO zP*Z0fE%Sgm4YB|eyc{Ld-{C>cM>K?^V$`x`F-8M+8`Q>0H`(3&sob%0gJLqqa(2jN z3#T42j@wQDl;`{YXbHY#Q?~6n!iTXHE z_ucxPlwZ%{O}mB4Uzr%$PNNt_?8t7xD*TBA&ekEUw=i9BryL0K1hxunM(m=LL2E)} zDGgiDOyar6AP1EXEQ;8~_smV2{vIs#2$G-ms>n=VYC}JYwx5ljG4sX0|9Ev$?LVqZ zMZkq@w|k^`qiz#B1S6EGA&wvhnd?_zQJ{|{%Px237FMDr`Hg^U2;56j=}lG(R=hRj z7<%+bdBK^xV|ms3VtM;L#eaDymvlXoj89UEGO3C7%T?wqr!2=dvGkQcr=t~N{;}Lr zgk8VJb&L;9xtFaGy+5@fLJaJ+>d#o;<99;|T?`J*sNCG?{WVQ*J5@7`AVNVc1MY>z z+a$gPoZc$BmZg!+(hulj_E>OU!;jUBwuFM7I-K8ZuYTB>upzsD9n;HrMsf=L&WlRF zC{ex0HFbt>mRMm03xv})#pAw^{=`i)11+WRdk3O207-JG@LAnfYcXm)HmgfOJERmz zjJHBERqIzee@`j0uz!J4YQNVPZvrS~X77JV&Uel1@3k%mT9>zxC)gNLC+M<_I<(N= zo~;f2*&f3^`?_eZ=ITU>Fd8uxsHNnJsM^x7W#Ai-u@F=lu0KrLA2@56(i}w7a1`C_ zJPBw6893Yyghx{hb+uy@u&di)K=+ucnTZ8aADSM7y`vgC;s zPnIN-7hoTCEz*hlSTMTr4(N8hbwMohWtIT+SK=z1TZ> z!sL07Iix5;A7C`AB;9 zsQeR;y6Y4FfS9k>`Us2yiWWs}-{)LCpCfpQFLOcKG*fWvi*Wgu#+Hx0H_Sx;7jWyG zWSaIt4a%<0tXewDT)zTblZd&98`nn2fMJG$?B5N1t5=$(pF;YAOUZ217}1QsC5`o^ z{$QJmOPVII%&DIf)e{18D&6|-v5oW+GZg;4z3JkCY&wP}<|O=x9Ku~Ll(JildtnN! zZ9MaENhBS{WmixTcAmR{@{WwB(|$eDKg8_69jK6l&lX2x%KqTmb0>L#Hf)`<)`F;D zTfeny=)#x78^;LM;7dQU28rQQ3h~b2TskvHOr&`~t3v=ZT3E0wCsPtiTvWwAr|T92 z)lido%sH}HvG_F7o-yrmKn^FPbz4(=z$06qq`m&BxaI!ytoO}JBku3wUwAbXp$!Gi zQ_ua7yAIv{C*JKz9@}oy{*YH0TqYe@KOkGdv@dn)5#TC_i79olc0|{L!(>!lqg5v} zfecYK8KYw7{``r+{IW`5c%L$xMeec?OdQFvH#Hm0(>&OwvWaoSfW%(g)(u{91b60Y z+a{Rs5B$;@-vl>5AYvvhS^7}g&QMxe6lwnGC2;medhrP>2Nj&tuz4IpV7L&+~kEkmI6A0i56;r#=t7r zw+?mHZH%95vL=^fHgwry0%G2#7~)+bDqJ2_pyQdW>@Y|@z9x}}rM%I0aK8CelDZ_q z;_c14IheK2`-gQd7c5^IC~{PKpr*J*39)WJb~Jfec1cC1A>uCxY+lp!83Nx+N+IQy zv&Qy%_1_#6k>nT|e;%f?2)A4z18g#5GjRhE7)E|9v~*=@TCi;E(?tsX=M#IvR7A!8 z=#pr85E*iHzv;DdBxj3LtLL`*%}9xc=8`$!WHGZoGamW3Z!`@XqYq$BXdg5@5vb39 zVsp^U!+e53h;3WoACf8?D&*Bkm5?*IHV~{CGbUp-se~3K;mxa9nOzZ$50I)c^&ZAc zns!UbR8Z#376gjK0-epln== zimH+6=y&M#eGWFx5?+fiTpDVYX+2A}a}m$)q0dcqg3oWM>F2{1pB^5-Hf~(Tb6i&k zz(Z$;Q0piABF5|8v@qG8M;@V`{?NaK65VAl)}7oq*!7<aM=u$uFi(&!O1=4QtPBWbQZK?-Fsno!U-*^_$yrGP_GL@QeOBWcjx}-8P8GS$t&_s*ae* zms1{A2!z3qxY1Vui8648K*jhpMg1>Kd+WsJYpQRfw2wl-s#6_rAF=(NhuptM!e%D(mSxe1o|3?Gt%Q$3s&&b-pxxGB)25DtPc zykd9@NAI18FX7L_S{u>`%=PE##)Yr#y%B~N?j1hyEr6h9;3MD^$aaJ(bg8R z4-)r4)lw&(s!d5EsfT^gClp=_Arw5D9g2=Vx%MjD_#LX!4_eEc1JGX~X}$^`~}VtG+W z7@9Sv8HZCbTm@|*P)*F)^32@jyneCCO+?@_t^Q%d;oiey-L*2=TBI0d7~$41y#9YR zHc(Jy>%>eXW%<&ABDo{ErUl9mY*x^9+?ToKv)m*|(%J)UIHWPcWmNF|b zQina8Zbs2MrJ`mLmE)e`)vM(0U7V{HxwUx;eeb;oRl7u@IPSQ=@M-D}@ zzrMd#>6Mr*h#V)A&(d~5o5L99y!sG-rNCN{voLciq8?KG$S^=D*}WqdcPFoqTN;7U z6=?d}uT1(ww<^gkB^RTOV|zgGjd~@0=HH@3wDV71nyKjb#EBb_-5^s*94*YXWHNAS z5BNOtt(G92GR3R;0o?9&v2$x!)7H`>?w|ki#l1JxfeX=Cuw}7thHer1_onoVkjteE z0f!8Z)ZY`h*0XxkeMdtTSyVB#EF z1e;1fZN7g#W9WXU+Gm)@9@HEV*k~>Bl2V73EwcC0LU)%q>VJyZG)PjVR07zVm**>5ObeuL7(sJSTSVO5o<&11c*0KyE_8*H_LB$ zP^;uv;(k|@;f_L#Q)!Gs6b$~XDQHN3COPef;_pZv1vOl{z`nUq$;0fxA@djbxTL5# zp(vZ1JAVpUQobSqsMtKJ9loGgrm^73!E027YK6rSZS{#a$#0LWcz=TrJ#iIqur0t0 zcgy02)UfZWGc0L0^z#nx#SaZ30a4D^ zLWO6sSzdb9&DBaj5GJA*#Gn-m)&G@pHLmYj_64}o^>*G6OYG@+`-N1hLSzh6sHYa& zv>8P@(a}zoIN|hMBUfsq2u4Lr0!$C2b=>y3W#4|X#De7?PuIcTVW7ke4R;)pn|qym zt#5xBjT|Wx^ru#&Yh}sx7WuX}2OK6svZ5&7qW2?D9Otsm}d@!lB#qiPP_VufLrz}!ylM^_Q zmYAfrdnV)B>3PX3o9(7AWTjT1L2+GQGcJfcFsgfxw)oi149gHtrz4+Dv;(U5XL@pG^y&2J4+OlHyvq zZuL&;j+hKIj_#--kbn;*Vi!;gl8Xtl;3LI)Uz=5$xfIfw60e@YTS4EyTsdrKuVav6 zcm^HuCy7v%Wh@N!z`!6TSH41GI31Kf9erI`#H^2UxUOUm27yn(N9B@{{<$mG{AZ)@c?{+s5$?u; zu}(xm)HJp}`fpWX=m|O-p;E-3ct-&?+Rry`#NO1|DWc}qAwT;cAa>vF&c>%7-FHU} z;G2~Th~o&rvnfz_54a-0!*6`O0ffKa*nan50J{FCyY}bH)PFZh)xPbI0M8F$NdR!) zPfW^hJ2C(vv!>rlGQd3!;n`jLttSQW{|lh(EpMDld4I!%`NUGqV6AS5>bf%uV}eWD z>~q@+J?tLAM_@P2p2G8SUvO*m_{{_(#$FFkS^H@Sl^ibee)-L2C_v=ysFr7XhH&wnI4w-I`lQV zO!g+RT&B34RtwPCBAF>Ja(;FtB5Qkd;7jP>8IUKBz8}Mg|Ts_*k(w^0u-6q3uK&e?{9Op6zqY5vxtZm=3CmnZjI=Fu8m8O|^&L z6J#-Vp4UdMJ;wn#4lNJPgVprEAMRxewgn0EM~4OrZ0XgyH*qu;DkXqe?RwpHldi(l z>XWa8Lrk})rsobgM7JbwGwlX(#U(9;@nLpn(^E*jWigJ9p*G^v45JV1inT7J&GVI! z_YP3+;zHua3s7)y4fB=-@^yQ89f|Ebz_mhEm2AmjBl6J#(|Spvu5XNo5zNfYr9_h`=qElFY+PIn4bYI#1#?_ukWnGiFBDwq zB23SF;BrGDp2HxwAv0lt?HKAiK8NC~gG)$bEs(&Dm(i#efw-1MRA(FTgvPNZGU-HH zDlZ8mt4w|CT{O=LoBSHkU*i~mCi~l$M+M#Cu#pLMpn16ZXV~@DVCM*DIt+9U8CBvp zg*q!U_9~Pqxt{uEbnn^nM49=pMb5MdY47ko)PXLA9%ktaUvZsbr*%_ykx#6|2TH*< z2NN>bV8Ccn%ZSfWq&@8AzdVfFLs0GQ8_MK{e2FJnCMet7NFk- zV7Qd#1RDc=Hh)fQ;X=SJiWtZ8-hDtmahfi`eE-XpeW)t>j5Ru`?%fxKT-NJ+H@mc} z6)zAiqPN-QJE#FBDNKUn-FHStoXr1#5->c5MG{hE1rq{JxkM#Z-WH|86@W8H^!l*$ zbTE2Qu9tvE^g~zy?dREZ@5U`xlBIEL0C~tc41RD@FoRV7IHQQt1P-`_`!Z{3$Cx=yLZkLSL*5_mV(qr&5Y;AE>Agxkl0h&mHGg+XeVOjNfj2(aYt7V z+Q4*)q4G+BSV`m%_a%@u=EjE|!lQ{^KDsJK1vPNlgKOEk&x{^XNO23=$5I!e z3_i*M7u__^#pwqV2> zG2ZBL!(j|8Z)*odVsI23Goc!V;_+z6qa8{&-sb#c13etn zdYRoLSsZ)yVV8>l8d07mzAK_&T!&fJlz^Ezujm|f&JX!B8wL&rhu<@+9vn>3!&(X{ zuY=wIWAbi|69tV{m?iH^3JzLSnfW=%tqhALbb=kV@X}+(^e?y3_0N~oGDF~wa;lr< zJiaUFYbVOC4ui*ov;1ZE)p$H3BI|;F&A5c?!)t!Iv^@7Dq^=vvE@mEKi z{>rrwxf-T69f&i0qj)A^01_Bp#2*{5K!gbe!f3whOwDQ6pd=4DTV2UTLK+0znH;g z>n>>457QyAFrpjYiJ&}`nFV@+_xvBXDoGrIW;u{OL5*rjTqrHmkc9(uSMJO?So5do z@}dPe&74R}1rty0ji40Hqgn$dM@czu*dPFm@QGow4x)iH9_U_BsEHX6FukJ|Z7Ir| z>ig^h=g>L{+vJFpeW#{3nf*<(i`Z;#um;9bQw$-VQPVhT)C-|Y z9JYQYMd-cwK3)wL)6%txuQxQ{v!aT;Iv_bnSS^J6@h^6xLb?IU0W!da2-OcYNZwq*yL;J5YgQe9kD8ipQ|=Rsgy*zAu1?-iqCTKAIgl(LYP55#HlKF=lYHNnS$*J!@LSMa`nTsImlUSRLEPW_lde zHC8I!sek%v_?Xn~_GA&MrkcR9tfasMd8LR1)#$lC4vZxlZazKDO+MWo&6|h=0(qp&)EAzcctgH;79p)UQ7v zuX6iBWs*m%LogIAHgr#1cy#IJii#|l#M){?yURUTnw4v1%}hqZq`LL7?)Qa3u?71> zIDK9%h`nj-=eacKHov~wX~#~ucL6R#wQ02@r6-g(@Aub7Lw5`Z6Ylo%df!?Y0r$HQ zi4!iw9E6Xn#JLdxyOtm-&^k369qV0gDiAlE^<;Dj`?_>)?!QwYB2==QIQ-m)J)+T@ z=3Xl`Qd#=qbn4Crp-qWiDPpFZ{#0DYH85OL)b285g97XZqZ98KQQ^JcPB8A&d2Mr?jNJ!kB|7Lc}l!-P2jt zVq-!0dRr@w%oI<&W99U}sM4ZbXy$+ykm)^u-dQ~=IcYiM-XA!}1jY@6^AhdS2pWq^ zNycZVX049z3pQ?6_3zMs6sW+dVWeu<65fQ}z!$T?*g?efly0n3e!!1!{*g{Vk#Po6 z=Sq2(t@bkiwKpy(RZr8&nS&A1+bhp5>D3YGh+2hj5FQ0hlBjq()=~-9fabqHp}J~E zK|huPt`XPPWHg+j$AsXa!js*@MCPB#s;;K9b->vCuzGK!e#xB#xWpWtu@(o4KuOBB zTxSn`3c~76q0x>aGg{9&M&Q^OKN-gHBt^uNHPqk1g#_bm37NX`42+86ap_G&5pRF3 z`W~}R2(xT(NeK=Zurd98p{$I{1(~E`nk7jst8Z*YMO|N$E6bgb4&7%62<$nHGl(Tf zlp;Fo%QvqCK^X|5J2Ir{~EaZ8aCQr0k{&w29 zHUTFk1XJH(J=O$GH0|1hA_Y#T9m_of{eug-YQqV-{{>UR^1XhjU9TtC6%gYPgmXcm z+X=R!AyfSY1t>KqM4Rx80Q&c9#7@hL@h~TSQ@o00-|f#%9~ct^gpPfBlit%wbcl*l zuKKJ?@F;cIsF**D^O-2mu@Ju=3uZ-lGqs>GPW_@^>)ixBGl+Q8V?BpFws9*e@v{0% zZkZlAj;j3u%W=Fce4}PHi?km*mFm?*sB83Xo**aCN0JZDh#J2*v`=Ky8E}dXysI=s zC^y8$eJfJUW0Ui`fp*$cej#Z|o^$^{Ut;+4F?9}4;`vCG9)8}W-FXlYHD?yCg>MrM ztObm2bKmI~=J;RI(+Qnycx2x0@0PA=sg;MWmoQjPqzi(j`yu(5yB4|hLSaVzYw+H_ z$gNJ=SjwjJLd463iosQ~ZRzdsvA?rj6&){hn^T#fspor9BWw5>-EHBB$UzazLQ*^S z_#R^U%(DV7aS9yLCH|p^>?lM9NbBZ}Xo0}#M{t#&Sh-{_z~;vE35Z$usnvvarC1|F z@ReOP)6Tf%<7$6O{eB$BF27B#xlDH2!n|n1Gyr$}hXTv6JFC0IkhG|$YB}N}ok(?5a1Bg)kZ*eZ!-1_4APpmct z3O?AAI}-Xu}0egVZjY&t|HRNibTHkXwyvX$IKa9GWza~+l&+9*4-J19T5i@cQ7D86zSy?T=BstIH9m~qfFgB_LD z1a*&kgQqrs+1JrcKf8)ysWJYh;E)!x2{La?7=h=cE7lWIZjn1*eDQr1`qwbUcDuq! zeFZ_vcOZKno|KSH#`5kQO3-W+T9sCbHl27mcs%al;QroKEz=gwLa?~Fh(Xz`uWY@L zNPrjNsJY_puc7Dr9r!$kZRJvCf953WLGb>Z1p`py}p7dcM`Y}yZ40i7fPv73b4h`8IyFQFk@(+4p-%6 z!E`4PIW`~~ojjP8obgFd$x*Y$&M5W;BhAqR6$!+sl?UUXrB0A>;HMF*RA}WC)3Sfm zfvJ_OX_W9^^F4>8>vtl?5|)RSP^O{R)XZtTRyj?`vk|*iUUi#@aRjS@5}h?1)0+0u zbLQP52`H+lbN@LvXD&dzmlZ$QPEUPwGuA=4PSAS_ocU~DjkR1L2bm-#nN|go<)bcX zuu27@LG%%)7#}W&KLNSA;IE&uz-fQIN;2U*ciGP-W`NdGL=}^bbyMRn{2A z4@CCHcLFJ8O&+`?=u2&es;D*KzbOFWYp}__#(q&EUTe}!Xs`ZCe{btVuT+mgLg+C}E5Hl(XHGFFjWzuvNW+6!Fxp z-W1-qO=vU_{C(!|K%R7?4oMJosung~GcV+lmR{OwZicB^n|P|wO#8F+{mGgK->2LU zq9!W-sQ1{-Ng$H88dj$9sE>YshplXlXZ)Vdl~h4F@LXF3%g-k zptLUBlvQtjo2yw@6(M_oP`1dZ!>ffysjw58O}f2DUff<{dt7p(EAX#SNu4(={8iau z5Bs+tEDcF9JSGHn6Ds;@3+j*i71Fs&5e2}8P8ySM9PgLh)}|;V8^W(ROlHDa4Kwk< zB}RH5Z+(R)h;3n0LRwQ!Vl1Xs(-zz__jigZyV8jPqXg+XfnLllOeKVjPX=-RC+ zq0nLD=+f|K@_i+)f*19u^S_Onk$L!TI3oe$UeTP}dke#V<~b(}>VD1F<5-B0M8{*x zb6u+Y1BLs8N5qFw)=mnest`;s?5ijTa2lhl777$~pZOtMfxgIV165Iic?SAHJAu2H zsMy8eO-{Mqe`B>m?z2PvH5AaHBQ8$O6Mg0dx$JGmHs_>^6IH7*#FC*gvSh;$nPF|S z6a@Df#@=r2+@CgGq1VZts*!q1!10=-2}av(ig4d_|f2l%0W z0tm^!0dv=T1`dsY8}YuqsG_c-AWT~fe?P@f08x9_R&0`H`-95`;PkeR8h}~zwiSfg zaoJOZ3CREi@UyoE>~s!b0D7hX^Y;MzWPs=cfE^$`0pJ4oJxu5~%g^ZR`%HZ@*fb%! zK?BS`2Dc6T92SrL-Z1j?i;O_Fr#Ac!{-p`qGpt@^2VwS%lou3TJCK~|{LBhyWz8=4 zOJ81KgQ1;Npi|OKn0f5%lPxpU(c{G068C&oIWULLXc&NXdw6qNlJ@eXt|INj-t@Gs z&Ste;pk@1NgoqWkP<&2WFU8FYlbmzlKML)(J{SUYiJ zcpaw~5uarq*?uUzmcruG@R4SH7Te4wM=u{IU(fjj&FdEZAx~hnrJmZt&e5xj9s!w# zr+SI23*M=j*O#zD(V@@vyK$nRXM%dDhA2`F6gNWPt2;UuJN~n6njWKjB&m#AOWm9jjP~%9y!a zJ>e6p?!hmaDAsLUntrKrhL?k5M7x!AKkveqJ2Lj?nl>4lc+|5&BJIHQkv}u3g9mA% zIcVUMOZuiLdd$TH?$a)LOGGmhfFg(Q&OjmayDNf1Gg&vgo7*k9HG7-s?upLp+00r! zz~{R4UBD7%L}r?vt7L;I9^cmm=g{3XgAcFu z-$1PY-Cp-w|LWNSZ2VUyJ_MH1dosVw8CMzB{PZrHJ^&nVWl5y2DvP=BO&@+8fR+!x z56c(70jlq(XFy*<7l07|y7>dp;tjY8v=_K4{c6m6UhrL)|FHbO#&?DQ0FmwSfEI`d H6zKl}VPD7A diff --git a/tests/trivup/trivup-0.12.7.tar.gz b/tests/trivup/trivup-0.12.7.tar.gz new file mode 100644 index 0000000000000000000000000000000000000000..ed9d8d62e57ffd10eb083cf78a6b484d12c97288 GIT binary patch literal 33672 zcmV)wK$O29iwFpY8xm&%|8#O`c6D$qFfK7NE;lZ8VR8WMegA*j$kA|rEq}$9B#$5k z8T>V&2Wc1GXQxBhl&+`c!BiRu2~H~G}~C+lymwzC_4$LIChc76K~?Dh}e;Is66!-eMm z@SpkFe#Pd#H8-2}gYDPbJFj2wZtHsoI|sGxSA{?1pMU$0?^>Uii)u};Z|etJzuo#j z*xSRu^@F{d`Zrkf_1%No?rwc=zjgrYf46>6`vcqimh0d2jFCB;Sz{-?+`9Mwq96VD z!2h@Rll*@t=l|b?|8GnFzrVNrdUtPo@0DK5Isbns|3B)sJIDRbZ?OLNcXzY+|Mu?o z0r3Cr-8!fpdwbik{`cVhA6V^MuK)i~{@->Mcdm7L<+IYb%(iRw{VM$Py23shm-fCP?RHz#c2I-jixeDSXgAO0}` z$ZY1^SoS3wJNCrFN*-03n|`BEc*&v|dCZwgRL0H(iotk82YdriM0m!?`HZh56>P`1 z#%2ZHSzdv$8NfmiYGd{!h6yw{o*BSIT|Es0Xi1@f!~nWBSprrokbw3PjfWz^MAs(H zcsVz1pC}8kpw1T1-GO&5n;X9ATE@%^X-oq{6&3WMM}zm>KI@;n9eir_It;#__D+82 z9(E4do1fU=eTTJAPJilk-@PBO_a{e(onD`{jt}AK@u1gzb2d1E2gOz&Y8DIlqILWe z>->1y>-78Vq{q4+Pmj6)450T~$AfOCUt!(j_R-m4_xN3f0RVPvt;bur+8=!vF$E1+U@zo3nnG zh}JzGbb7tB(?R#-xC}l21Z3$I+AXMhNYp(!rmg~oPI^BfP^1A7zrsGf@4)jO5|k*_ zLK5_WV(o!atN`r+0S8JS*>UIHQTJWvxZOc)P7uncZogB8vFi4*K$kcBsReDGQI~Oy z0ILFhRc363Mv`^kvew}bT|`?HhOzH=#XKRBwcj&NZv8(;0m}9NT>sDY|6Kp)`v1q) zarbSfKhQ1v+v)$X-0N}u|Daya_5W}2an144^{mfk)w0L4C1}WeMr;|2g~wjD7K^Qy zg|Lu$s}^D}|1W11x%~fX`JbQv{PX00e*Tlo|3Lnqet1{y9=|>LruM(u0lbTz|Lkoa zZ0GX-TYNs6zA-U;qxyph65g>JY+J7vj*Yq55JJ8Xyy26i&|l8yhI`jw19)#4v)LUx zTQGxxX5jgzis-19TKl}^_=*`Gy#8QJuY*E`_s-m`E{scP=F0aMUSn(P((hBa%jIlcXxtgL%x;m8`Bwg-)z9e+4J`E*=3 zof$rw;ld~Fnc;cX)G`77DaU(`2DKx@zFY!=tO3vOpdA~(?L4sjcBH&O2Vb1@y9~4| z*ZFKt@b{bkp?n>wa5Z`i*o2|KlFt_v&z>omw3;VKMcv zmxY2r(Lyz1Ix}yr(afx{#S97=^mT^EX4M-L|G*2w^`RHGVL^ug$hg9$Af?S4$Gt9q z@TV^Go{n5-4I5mp=rW#xUE!go{Vf3a{}UjY)7z-Mj8p3atlhvu<9yqz1> zo_UF#p*}l8<=x7`$24jUM zzof9w91q4l9nj$k9T4`;j`4i%^z02Nh3vR9_;k|y(1*!Vc=iki5!85lQk?;I%)B*l z>cX3W!@PxM^9h|Yb%L>vk37yf_&jF|Qhs+>buK8a3jm%@@Y(`gwJN4B+tKTK4SsuW zp>PVSFZ-Wr^~#(r{)b2~cU)7uTZHfH!PPRBWGrXubW=8lJWW@3%&ly+y07>N^W-?CIVWVD+m5YYnG9d;a$i!`9&; zom~GcNMEgvT@xo>l~lq-}oyW6;11MzPP(6Wxp%a02$o1LI4!gt7rc2(sE67K=6ngi9HdplWW)|c@6eI3k!I<4Pyr# zsJhHmYUuv0|36skd*idgzVCJL`?2W*nef|NXcbsNgW*E;yqSC!V>1nQ?T+@U*OM_T z!8Bjq!m6CzRXE9P$7d)JaHTrlZCMWh$L2LUQUJ51r7?0GA08SDzPQguOWR*=9hxJ{ zu$jjGzK7TzYpis~i8e2lCu?kdmx`Z7v zA8%ygIM6;D^twNsoep~^Cxhny3CRr73|V{$opP=~z928di8r&Z%`)OKF{j8R=w(6W zo>#_a5QDx)gydc*F7d8A!jdY`gq%6Xg!}1GTj)BJZ!&WGe{TQJ?f?1b_p|@^Odr*? z>E$=I|JUmWG5h~Muz-C3|1Cb}b8!H0QJ9$C*tHg1MYUib+hl<~s8BdJFE59dJ#{V$ zzHvDmEv?xEO3;r9>c34^%Vl~l|8x8Qzo`A6%m1BM`?>x9KV1GV?w)@AuaN)sgZiG7 z|GRrT-2bnZpZ|Z8&(wA1bm3#^TQg4|qpi)Fqw$x~WGGMp`szE*%#$ytmOU9ty9O_G zIdewxHGVG?P_F>JU}BB^Qn_F(7VtykGftzI=kV+T9^wrbDDjNF1+aK5?vag?jP(<| z&w}tun&O(cg|3NOnOi_YP=9W+^V{=V9nmXkR1gYXkeTW>1oQ&xYwQJ^lv50-)|B0z zSL;ygPk`?KX%~e8^}l2rz$YdC(iSXim`#l@q2RfqO*OTz@RZOhXb`fU|1}z{vsbGN z@CJE4G_{yVM9@5cKP1nHrk3=K2nsG;c)$MiCMZAS&IUu&5WVKPCfp6Q3t2!;ie_Yr zl*+O9s5BZ){71eJhOVJ7n*nfTa5Q|1c1TS%!fU_WjNLn!nbD`*0zLfY#@EE-0K0ME zS`md`+`#ptSb125?P!I~oIj#<)_Z>hG>_z|HI;c z?CpRWn8*Lf_y0Nn|7QFD-2X4<|Jf(~za{_Q-H*lpfYra9`~Q8D&$B;mEj^b!UrhV6 z@ZBl!4gE^*9kJ0@pTUOToU+91DV2rKgWUEuS@bo3yKpVrFKK;r<)Qc+0m` z6RYycu&4%Z^0{LidH=vJ)?~1AjRLrV!b4{ve8h}dX{Q#2COas6iW2QRi&FXDhe77@ zKbQZx{LkfoLjL!?*;5?^tdalqoxOPcuYqA9A>HyjRinIfZx8~0m@MmIqVE+x){{!v+ zd)xKg{-5vvbN-+2|MSl`=l{bY-gO=hpU42PF8=r4!ESv2zrT~m|NbZL|DP2KcuRD{ z(BRKlYH;vpg=g#u;)6e9PZ=Q`k$T!7;m_D3!i1-z@%TvLh{V&y3P)&} z&)Cz241dO+E@(I+2AxyFh9mf|4IKWAJz?nZXY2`shd*Oa7(N_$%aaBWf5x6Rg!nV| zBtgUr!y)>!4q20(lP@w_HsxumcJaH=vU2-RZvV;cKe_%N(f{e*#6Q#iv%kB)o7;bK z{-5*zod4&ae-8gAAH6qpSdLi$z9s+PuJ6E~od4(c|D6ASGyDHReXmyAf3;uB?f*Id zPxAkXxtKY3x_9;D{D1EN)_;9>zYhQH@7F;W+}^40<^KQQ%KneG|B>Nck<}lqhBVTw zB<^FE#eH-%BmX@Jg&Bs7MW;W08<~Qhq7}MKaDM3Y`rVUb=9zN?L%n(pG%QFO$7CV z;?8txjWLX`u(VSkBW}0u@($DGf#-*v{(!ek*Z)}+gQE(l&FA$(fLinUc0m?yKHn*v zd^+yHmoK}GszI@H?+fjdj~}~(=JP!OaemJJ#HzotV$ed7U0kq#{DYOu@s-1hXC7rm zcwP$#$eViCrP9Nb-mvrWbnw%#Kj^U##$R}x(0aL0I68UvwtLiRJ}=F$eRIAj7dYf+ z2=w_&X87!R31c>ui|jl0Toexqg`EkZf2WiyOqceUj&pbbXC4ab3whiL7|*b&275zq zcyI_gtl^jzxsx6cgvil8>a>oBZ_c_$heVP8`Y)zDYTDwG#?rf*C_o~Z32-VX)@-sn z0SLZ{^q?aNTL@-=@5COHn;IT&-!Z+IGHp0g<7Hnhe%;(Ui ztQu5_AX1FH#>u(~=#gvKJiTja?=l<^jh zKU!XTqJ;cOue1903?|&vHN7i=od}%D$QzW6%qhk?aZQ@O7?pb!J+k`WMp#g}sCJV# zkcC#t5Z$B4#)e1flkl`7hloYxao(F66=OXXdrGSL7lEwTDY8Ns5Gc++X<*SsK zB)oARqFG01MU|UC$Hef-S>OCWc3!W(zBsSJKQDjQ>F=NA-*8aFk%vh?RamnU5t_&N zuF|bA(1MD}lmM0~r3+3m8L$^hj^JXtd}Vz$buL?U$GzOb6IhPr*7Gl-@Av${a2pi6YWP(5FMnl^2UU-Of^NXH{^Owc z6C&Afs$D7kQOaC%(yT#25X4|eL7)_ER&pBxF?q0exRy|@> z4@gMF0~tD=grXaHJ`4i8{dLN!IKM!>T~xzB;LUm&8Qs_c?ZjT1X^8NJBvz6bSaTBM zq-qKREv47>h^PC=HH~Yeifzi#e;yV_K2wVhe;|-U-9@2>0UUP1+vJj;#v+g*e+*>4 ztYORmfr;7;9Ex-qFBn!JjUuZfSaV^%5Ow0lmBR#3BwHy;HZ65O7U^C0U<`Xjj4O0SH zlkb|mgEJ6VUW9DL2qQ>gxm*K>0pk5ioo%e--OZ_aUsbBg#R8z2+kbNV&qM7$x&LQw z|5;)GnK_rczvv4Wug?3H>FKz$oBp8xsV*nec)xe3Y< z^yd=^fBMeUzd^Ujx6aa@kY6Axwfb+%41QwlLQvyHa+bm~(xJ!+5FUvLfy}rB5y-Q7 zFosD+5c=`Spxf>+iZ2~_4hnZdfI~ivl$C=Li%hr=qcLN+X>{it4cELh+zI#al#Y;~ zBAKptWi8MT53@A^suB~AgQYg1lRr62(Fdiwf(jKTvR$%mV1v(C35&?ceZ}%$C@;p` zxTEYLXtW9;o3~?g;iFE48s^M`O^g|KD`){|@sohzjL5^(pl+aYrxas`Pc|v|pt=s) z`pwOas;SGqZHWMkRYDkQoY|PeDNH^xx|4~CMu6qaxWQqgv7sRc1a4f4pHTswi;p=# zQO71EmP8KvtQH{8Ug@&Q-n9DNK04)n>JHwcXPxS8NluD~UGxVeA42rN`_Mf;tT59e zS5xzLK{=2d3NbRX$dfpjJ`w@S;)N-37UmdzgP~s{0^_9v+ces&l9RCKNgr5%GPCBE zFY+@bI;9s1oX5G2;~zPfm!{i1?7TU92Xyf#=!q=e12r-9BE|y2$4wXwIWW9H8{j)L zvn9%5L`(zmC|ZK(@XhuWOg;1KDVhT8L3B;4pG93PS{+l_G#aOYdgOG?0`w?) zix{w9PwN<<2hllRH-pC9Y9tD$5jhU$Q3Os0@+jhx)sl3*uOw~^(1VB@ubV+!_*NA~ z(?}bK@+i`#19=>2Q9A;{JqdLaupUO>Wc3Uhhb2=$CXpJ8M09@Mf~sAfgvavCTeKtylTpEC?tzj0Mi2B!}yte*~3X91GITdJ6_1W_8)&QAe5EdBn73~5!&^qq8O}v`ZNj8~QS*Ug&c<(H4 zP2B{(iTWJTl$I3Xi^}|is1mCyh>VlVHM^d!6WJ787<<#MI`5u|lNCK}Q`?$%kY8qpI3442E3D6kS<(vMH z5vGwN$|u8dO>(v@q9h>*eE$>ef4TiXxBpB3&)W9h-cCLD|IF?GDfWLMb{}&9urB{o zeHZ;d_YQVz`};ffeYF3>M9l5~-^Ttg!aGKXo5{gNZDwj+l8L*(LnrFi1a>hnN36+x z0fRllFZ^>~2x32of^*{*3*%_+7ceq`iAH)T9(=zq7?aO9pFF6gmIxmYmBOgcz@7;q zIG02C7C!SL5AEe#4>PdCD|j#rA3(E9-w|;Ob&>4dw9!J2ZNs~vh-t{*-4}Q=bzW4K z^xdV2@hx#bqbmvCb)MiIASn-r{Na7UnNB?uGn^BsjjMU)cp=b@bsvK7w&%?6g2nj+GRvDQie^}Y3dakk0~bY?u=4e#hXZs5|2B-t+_DP> zMwQSpY51LEo4PgCZAUl8GkuB&N1RAGpUw5ZT>tx>>wkIv&s_h@)c?LF6xeG0Z|`7x z`yiVCbGyEOkmvvWHu~QK<9}_}^Y~wR{I5LzS04W>kN>qc{+E&?LFPr^zmnM#KA7&v zbRF-*WUORSh(6sg=R%^?30k~ICh$P_qJ31^VXkP zBl*Pf^3Iy&Bf=;pUyf!Cu$qg}N@2DsKZD?fvT zKr@8j=vWy3RfP+Q8_VM|N*2RIALClNV-foRA|t#*L8q&rKbSMq01!7<*7!BhryFA&=;>Qj~zhvr{dWC5(wQ{sk#YTppsc@AAU->`_N7|&3 zZFTefQxeuKI=<5!TO+h$XrKvc7?_aav_^U{=Cj5+1m5-gtksYY`M1%2DKhe139aV392w?=R=p`F=l4h*MW+2XvM`ANzAd;Q%FM= zvoxwu&$g&%_zcT67sx09-U@7xXPbDT-Vk~6(3!xCH4N?Y7eg!#ty7*B?tnl~1cw#& z{5u-zM$o1BOiNfq0b+-SkLf3KL%Ac%$E`-GAO}W^lQ@FN?w(QnC`)f@JpJml`qa4I~U>AOMYwN z2V=H0JFe@vDR?B2rvyza5q@1^_`i!79e%xrmohEW)Uqc-Q1MEds?aG?PL2uG0(Ofk zS`<6jaZOq41N*MIt-oS7urW4l(v)CVOX^@1)j?@VV1NG*sV$*PtpCc+w=3*bh1D*S zvdJ}2z3C9Opl}prG*<*@NERBE*1V}}7j@Lw;Nw-;Ms`_FYn3LN-~bxkLP1-K9P!K< zDr?XT1)iYHCs3~T9^Cd`<~>lL}XGUgx$f?5OWBwD##NWm`J2I3fbl3dSGr8;C? zsldgdAi?F&xgdjwC4B2eXPg!lR1Y9>tRl9zQ^=Ui2sAmN6=1(!58;b#j{OkvU#~ei zL4CnHGFpW|l%ta)d3gAME~He(*6Yb3Kq2O(a!-z9DC#7d8+ChPdOi${GFbFjfp(Eq zbp-50)O0)@%j&;4mR(AN4(c=%k1tV__(Sh%1pwK`X6ryp4IL3{m3NKHT<!Jz zGwgN@NVKy7q}1}snU+N1S1D60QpGK1#< z!Fa`}_=Y!w%GyonknNR~!yf#9)e-xXe zg6tvji2g|hZxszV)Z`kU8w=!j!EyQ|k&_;o zdMr>9$fOL)&G^ck8$I(9PjK(bal67Z0jMfWHfY7flL#6toNu7M(Vmeu1W}R5YYu6E zn-3W_GV-K>61!9Fh1Ym7q1@b3(7=cdl|m(;ywZHRkVbR4gSv=lnxuIOCrQ0_$0UYg9Xeu!)`E+U zAQA`Rvp)e174`UHvqjyxT{s}2#qy=ARD6jmE7S1UjX9e|mth)hg83n&C(;Nfh_Zyr znY(jdCJii~AEDkN!Ty{emC{gQ3f7qsx+ZB>U4~eC*Tt zNLtlaFbXUVrLfAtoj!VTo(~gtGW0N2D-4HJ9iGdwf>*)u!thESA|{ z)3&Pdc}g`iwzVlJ(w0XnY*31$LH}xn4KTnErs*gUQD)9%sifg`-R|+b258r6d(!+T zj?-UwK&>@AN;7X4<`{io@Mv8Sh=ufa5y@sp##v3(ae@so1^w{ zeLuvC{$_9&a=|bJ8E;?{Dk3ssba5Z)I(Nt>d2M1U0&{FEV~!&spc@lfl%`4wD1@Og zp$A@-gZIXu`nr~*X8=Ikm%$dUWsj|eF=KPQqa1k1g)h9|Zb>?vz=KjKVnqfRT~V}1 z40B1(w$O$(!;-};!l2oWQ88i-Sy&J7+J-m`!N{S|d339z`xS|{luqbbO6tlr71Y;# zM<=+tyTk*--q}&7abD+_dM`@)%d)ri-}=ifjsb8mK-G5s*J!ZLUag+w4!}Ht@6e1? zPIf9OTVS$*gYp-1sHE${tVBcI{BmC|%7Z^X{nC&BO#3r#?qDO;d$Tu$UGv-SJD9O@ zb)9S1?r4wLsdljz1jSH4IUV#-Q?DNWVth72B@Y1+%uP01Ptv}(v}4Vyiefr2InKji zoGY;x*XEsXj06Y5eQ{7y_-ttFjbQP~uwLDPiLFQagWGU@PKvPp0s5Av7;6u22Nv1~ z^KM{_h)T@!9Ty+)UlZrp@lRb?W4_e39}54P1D6_O8fnWLfLS?^$>PBfQNw>pwj2~F zCxx-%ExlGEJJw_jjb_S}-9`VMQM#K&8yCyr|sjxcw;Z;}>+mSQElhK;F z3(}5W$5TsmA90IohLZEzGjT5UiPgh;rCb*{xfY_ye8PPG** zZ80C{ZJ)fd+t|OMw*}gT6UAjuucjsXCdQE_Q&WyaAAg1FSTh4! zhVe{b$m8QhLE_Z|2YC~!u9UkF1M|BUMyjK`9~r<*^xoDtwCX(NoZM)xrCLBDbNR@c zns{BXZ{po$uUXvLuhkx6r&_Z#9&Gmt;05fqrw&xJ7#X+XBgycWxYa*d?{zZWHQDzc`VXteuZ7WN7q0WjHxGg$ZH@$mtsMXwzVK)X#4|t|A`q>- zVY|^LfAbc@IrvaYD>W0e4&OZx(zorrdf#%h24iQ3(OaCzNkX9;;T+;F=Jy|e44vQS zw_ojl^^Cpk{`j%eVBgPg2aYp?Ewst`4&~^f3x3chW!@w64X7$nA1Vcu?4yG>wU(IA z$6wmOiDq}``l9(5`y>g0U70X$X}%A#gCzIlk3u>-mM0Eq-0qPDDO4EAja=k0uJjHg$J`;x&Zmrp!n<^lBgd9T|k*n!aMYsDC?k9dA{W@ zp+JB@Inzw2Q1|HM-R}21XHp40Pbq&mMuvu2Bv^d%3Sf%Rjo$Iv{blnZiRUs?O z90BvfCAuIXC48!{Q*hKMy*7m~D9I~Jm5fDNPKV|!0$`{`(T0q7GKd|b`b`sOY^3-u z1$nuxNCid6p|@#;31&i<=!75>BwPj=t0%R5M{4% zX~s!Nj_~{png?$Yj*z>$1?=8*~x%ez~+J zl@*SD8MBI}0(GN-2kUf#1s#)J!r^SF7(%}k9m3_e{yb0%%{jeOIiQP?@$c2nl=_HMAywL$#%Wt3p?Aj|2@h*?H(+8oPED zGq`iY>cb(~jfcZ>;s#yZ?KIWt^mdh-YQTKZ(G-vM*EJ!v%9K|MBRgLY2w+UL;B2$; zC9AD_Edx^M%lzg3Z_3)b<(tq-TBGH-tW+1S0y?@_!{_A~iw2G#xudIa2vFAzjTmao z=Y6W9{4*nWF8_DwxXZcz+3_*H@|Szmxw39P2ELpTyvnJ}cOBE!SwA$8^0eR-=tPR; z=xw3U7dnFxl+%hvrGSE`Xe5TBxTT$b5n8Mep~*y-2Yf@rHLR1hk}!JKN#a;ua4YM< zHO?*Eat#wv3uwylB;B}emnT;*C^*>*;$<0n97hnR-bnUd{@CaqReAbfU(4+2(}8g$NgB7BtSh zv1=`Srm;UiKpW90Qzv)&5Z4hDd?J8^&lzrOt+}b=Kc#4iFbqlB;0hMC6*EFm0$He% zyc&mIuA(|R*{1j_%etb}dF>)mB*aM;it)$v1CV4j^&&uO3E9EFC9EuSj2iT~9tM>& zu+PNxDY__BKvMX*N)!Dj{hEn7qbm}2CF3?hGGv8Zc)_;{ALfdnMLFY2fM5$)2y})O zD&HElZ-wfYs{V;~S89BUR<{aysHm_sT#=LPU1=$mjVVi$RppY6I~8Vx{Se=uG|We; zK|keAPnzH=#i~DJC(u*ZnwTO+m>6r$nU{-+L4k=8v8+BMwz3p}IW^n{%eqFDJ{rMf zUAJk84*Z$GIzTLtVHs$bQ|_H|{GoO14c+W+4c?|(r+~nn8TlZ`pqPedkvVWS ziv@Fx*)owd*^C-_Uj%nDy%9vHcS!O#uXjQ^0ok|f+Nr*;@6nB-#B4G0D zcy9UzS`5|9YJJ#!O$>L^3La^RDKnmhZ<$TIz#ytt1#QAI$-3niB0l&k3Gh}WP+=!d zUm5sq?tF#`*Qa5Nw#b9h!c6_VgSTO0^)c%@ULd`MMZFy1`S=#^G+(A$TncQxKG-)8 zUYWbKy~*Ce>)oBv>sPOh@$RdG(ZO{0b-n(2RGaP^)7@tiYdWo(wefgbAMK4_@9$2w z$2+y1(Y~?&+ITg3HQGOzO!tia(R5^(uMA_azB}D}Rh#VW?TmNF@c5Ovv%ludA$=fa z+C=4yR1}KP1JAG1t(}{MZrQkLJ!L48P3MUj{*&^*?Cd7)pNWy;phuublrU`G0U>apSbF8|MF_)eiiBd$$g1$KD?1|DrTtdH%0& z@Hwa3bt1$RW*&uK_pHgj6jZlgjFLu2EAZy|m&pi?C-;hb!`!mNi)9#$zJVEB!>3_c zbf5!@dsk$TB5>hPrPyRVl#$g)Z6 z&2yc=>-zSq%-7V?c*LIvb=>jPS-QN!e6lB=tAPj62PeD>Hk{6kOGM_+3m8^;ap1mC zILEa2mVJ3q_@i{(X|n#{u+!@ma@L>k|8xB>*Z=ZQuKz{#KY9QC;R@h7{jYvdi|c>; zyY*cE`zHF|%o@ogUl%f&{ZK|zkqTkKo!F>JOeq%$d>@Vt+p#fomi4O{W*<{a%pB)> zxft>gC5gwf1^!~{xHpD7p*swCU14b!NJ%3F4cU{xMSXN%@GW#-jh2Z$yQdtq;_mF| zj^QlEUA(>ENMKHZ~}w1~w$V-eVSCaREb(gF;tBD@=lwH(W8CHC`3Pb5`<2 z)I>fey>LEp<^~`|KS9@5aN-)hI_}T~eN6#Hg%p$!>SMg~@^M1X%sFqa|K<8$uK(qq ze+~UlU)+7w`p?$?>U%p0{ja`%u%GLH-{SM^Pg_fm9JaPh`!ieI`B#_#9yO14kY&C! zF0<`geZLC-ysog1#-+XVSO*k@D{D3@P)-L%MN#7*;XmQ2;G-Gt=@tb!^kCuybX}$= zO3r()BE0I|AV%9W%;{`RMOLy3MXp?!?i??WO~QQifsa|3<{slnh-{-+X>R%rLh5A< zEsqjha(u{Hat@OMohS`)3B?#WpYfHXhwXrxi5KjzEU)lP1m8y>YKwT6HG&Q_Ns(wU z6kIwQ(40aGiN=C^hONSckU(r6ib-(I#u+cstD9&{(4f{9i~^;02G!CxT?-@U1@xw2 zqN++4c*i1{(;0j}?VbG4J?tE^HUH0}AF%f)N1zV&DG@xyJnz0a8=Sy{ zVyh1|i#WHA4=GQ3=f~3?rj$SFvF^vyqb`60_`TNgpxfzJ*qRjg&@8}8R>Y+3A3MGF zd-y5x$y1AOy8}vK{}!5SvC~#>&~2X`wR-IItao}sIpVQ%huwbrsMY=0In)vSF|n%zIAkz+FR^Fq_1x}l+6AJfl_x`?_c z4s)U374wEj*M2X^t``b)k%r4uX3go^6#rEazZ|bnC~!Z-STw|vxU-KT*0}E-Ufc=t z?0z_GqnoOFCc@dp$mCbw=pv897dHsFOVg+QqAo#&Uac~puJB`(S#@crCz9*(NREz8 zU!r%3;tA|UeR|Yhx-Ldw63^mA`DT`Xdh{Lln+brJu2_7cX+I~{*axf@{JL$#TeyhS zEjIX-TfAAwBMU^!s2gx7TcmIGCXljDxA!XaU>hH7(}NaPJ|?OW#+k(d zc^=tLN3n_;If{6h)MSD24H>l;=4EH~8k@($fEQyXkaQ3Mg#8!eE2*1J*EM%3a5!IGE?SlO4S6lp^9IW}FCfr;PAQ?CiB(=-lc7~xg%n!HoIlG4F-DbBOd zkFxr8XlmLfcxl?YP(P{J!@!8 zSmPe6PFV3piOb=#2QR%TJ_yAiKIBw`cM*YJ)^7}0Y*|cY({LVdDTD`VT%sAomnj;+ z1|f3DwBym$LYNZBAgf9&(r+0Ir@_y$6u}^Z$_!kYP$@ee0PcKxG6J;h36B>N?sfbt z*I8a(NrQ!H*dVJ-{Zfy~8E-jpVC&>paZ4ns6VNI`r7t8sWF`_-!NtgLA+pin>3sVH zzvqT~R}t|?#^%Z`)5x^|4qlxMlNMn3SKczt% ze-nYA64bAxY$aBuB#(hkz)gd;ip9`O;%OE*OF>rhznIoxLQMhFfl&r>v2|I0N->>M zY!S+p5a+VwJP+nI*FGW20DRR$N<0VlA76dJqA2#xV zi>bOdiL?49_Q1_l8&?R zl9Fpera+)FfAi3uJmzusBWzRA;fDUD@0Eb}dNGcV975fH4c-l6IZv<>P8NeqK``JG z7z1+ia07Evg?%<=OPW)$1FZ-{3bt+tlfttq#Ket}u;17$ZAZj+PO^lW;#0uI{GK=* z-2WoE^TAfRqshgJdGzHz!99djluj(Hy0{5aiJ>q9;%-rFq+$?-LTxK;c11TZYqXi) zV+W$Uri>2cC%?J>NACZj`hOhM4z^$K?Y`d5{XcU5kA=6D%=PnFACT4lAKUx;bpLB_ z9}3lXYfx@`d$*R~|N6c8e{9!t*N@!wBX|ACT|aWykFRk3pf?y@N+gMsuiqG+`F=1s zMN6mY`68uTh2gV4l@w5axf#263*V6yf4RAq{v@1z4}ZDwF#;(3;WaR}8{Lo)#pG`u zr*C4;d_y371)HV`nQwW?BWHYFkkEhefI*1whI_d%T+fuhU5>R^DE!bm>K+dJF!f>j zHQBl5!*e|L#J^QaqJ_VBda(NT&TiEhjVD{W%oKJZJz_qYBi?U8ic&g-gxMhB%Mv$F zS*seu)lO|+~F^%z+oT8x(>CKX>4X!yeKO2}K*uWd* zfB`=J{?mY9jWOt35RY6W(kiHaK8_6|LhCCmK|vvxctZ-NeJC+0mD(2S(+Ul)OUiJnwMLDd<~Q(XjuM0v1Bi@ipeyn1ag3;}&l1y?R}Hz5RM; zh1=Gunvg(x*&MgUk%d!drFtnPoT&<3 zMkhqUrOzB=;+4X;X+dy>jDUYr`-O}GCVeD=IcJVBVu%piRF?PlK0vxC@vb#Ny|>S8ALN?X`vEj7cQtnbCBH#a>G zlPok@5oM4(g~Vk~T_s_EKZVQN?DbR(Hghao12>|Zpg5C)5aa*Bn^ZN zWl7Z_*G*rZjDXud&=-wa_ z6o|3XdDlzv=+$)o6LUmZF^j)y2CtaC^h#w-W%Y29+^dST>KaKFabL*nJaLW5a=!5J z5D-28Y$RlJx2(`3U<3hsqWLd$Lg{2kr%=|UiQW!zo|%sL0yYc8?991jfLH|c?1H)t zjo_P-ECp`}b|pAF;eH1RaGbxvjzRza(+8bgeN;H`T{-+)74skLGWYRg;-F^Ano&f$kfiDgR%WcMQM&f;nn1@E`iOy=-`uY4zN*0X;Ig?X zbwQ17YMylo5MYR#!2qiZAV7;N1#JGu< z70%4WM=9rXX}4Be-CJ#?RN%Wp%00a%%KX$aXA{1|3KEjTMgTF@(nJHJ5b6b0(H8w` zhTmc1Pt3Hq^rzKV#i)C^Qlq&>xD`{AmWBRh%A60|rcIX6%vM7GFD^Q>6}L@?LL@E2 zeFkkrB!`EM#)CX2h9mP5ouz34G)Yb0kQAXD6LkZ+o+ltTE+Ve{ zQ~`$qrjKec`daFCIl};tKybg}4OB!sA&t?q5h%Yt6RV2TCX`I|W(?RHU;LZ_!uX4v zt|@Mg!FQ=pFH!vpi^TNhq^`WFp1i4!{FVCg*XqWP)QbxnYsCMR(1=rqL*70SvXpzA zNV*)E_9dl^S$F>=Hj`Nl0%xySKTPtwQr}*Ql~Ac4jT6>?Fh*luc{t-8YEphk702^9 z)f9J3RXXteR-}xhMY-O`PW2^ZUIdHA=c$VMdDfq5g6**rOE zKqJN#@vljkYW?XbA`e88mcUdipp?m}3gtS*p`4NqC#!+2VDheRYQ-FXP;-1jtZplF z(v;WT7>lsOU6>VTy(8%_&UdnE)+&AhL7peZ9Gc$y{(>Jfm6RhycWDpNk}fU}2KzME zS9)~C&!u2iCC(_-2_k6HqG>TD&Gd6h!Kngd_CB!b3-%;S%>l+Hgow z-G<^OC>_XRi}G+*Z@HJ`+i^;7Fw+$!XT)sd+JGNUD^y`fmQ6E_MLvqbSS(A9i;m9T zhCrgERjcBXHfdv^8}Q_MO>uWSvtcQBxs5Bc*p)e3G__MYp7zC&PtuERZ3jc4I>IdI zqC-e@Z8ti?)r4ab?6VP9dIyiJ*{ehiD% zvJJQ}pszTNOZM&|krbXN;z5G{a-KEBU0HKQ_=QvixGaNxBEOr6bF#6u z6>e2uOadX2On8{+NLmY_2-~QYP%>(<)?RGZo)lx7wZ=<7vTMKGTzjkA!Se%cQ_MZX zn+=_CiVgY6JBN<*P1~rJ*iTy%8L?sKLG?TncH(?eYQ|{1SzYcxM0LoqXroFnnN1tB z1@gP#(E-pp^6ib|UZcIfOXdOpt~r5$o3(d#U-c_NA+bc7v+FT+IN+wi<=eOpD2ziz z?wz&6s?swI&!;^-g^_VV$zCK58#5;7_nCd(L#dIKTgs-~Y+uzviER`u(3iu)Vp_GcVD}|L$u-z^;k^ zx?SJfjm3Z6-rdgQzvl5@^GiVaC7}EgP<{#M-}DlYx(TFo^>4_59yfrZbo6O(5ofuy zCW(tT;A`(IXxa6x=nx*W4eKjjxYS7=MKd=neUd6o#^<_ZmC>8uQd)ro8dJsaE$6G ztoeAWXb9Y(6kYiduTowo)!QOrvx%`i#la$|H*y>wur%cP2u10^YHIKonmj{e0`Glj zVlqTM2#*`RSiIUKv?oI}h(##dE<_G7zSi=dVsEF#8yeliRG@+!{K!g)kBL!O#YiqB zi@-TOv+U(>&xS;VSdUj&7lm3rkFQ{{TmmvH@J!>>W2SSX-N1=d?7zi3uk+%$(l>U^2 zcvX8#(nCLY1^S?@#)J;c@F@rx`AR)PhIwr*pgl(qZS#*}EoY?30G^6_LV} zXVBjkuDPmw{IWD@(k(uB3okb8$NqfJKxdi4%s52FaFAQ{v}U+p5VMk?t#+hV#?s2E@v~c z-zd7WM2&tZm4a=cP933G1tqPdR`n|*7x4XhzG`tF~Z&?Aq3qaa@CtbfGDXeFPD*kl;92YK@7PAqmhk5mGjabp|WMGZ7a*%TKG zT7o6dJiujkC%J-0S1>633c3?}YG|tA-`HRnL2+;Y;8j_L09A%xPR=SPm7_|((jQdX z_W(#%@*3>;STS{_oTbrR`nueWx@Y>L!IGwS3D43b)@hjt6BSR+unPRUQDG{(QPCqT zdbCl|{s7CuzZ(^%vKtj`W6|T|jml!&Jt*#wi!~@~B@K90PXb8K)#Becz-wvEc@?e& zP*PG1gdlIYEJ2xri7C|t9%)7ark1@yEer3Xfu4wb0I%sGckrxBTPOv3+)|I{$2T7E z7z+1@vGnE`$+8M~opwHYF>^S_W;ixt>14*n5PKm)$PJ0-4TpMgAMVerIOKJUabi9Y zRP;#Cv5`|@&_Ff<{zxcOs6K`V2s6BSJ(~eYuZXWG>6m)Vuvq%O4C5yV4gKbWAapOf za~efifW#O(+QOV`jaV*E{#wUZ7)s3Z(@E% zGwGu6j;Sv=bLtw%W}XB`Q+(w)z9{qd$OON8-m3d_NK2PvCp|SXMbMQ*fwB z%t9P=GG%clI~#l#N9}^QC9{&rb))Mr0VlvX#a9qQ@~h|0GyqVg8-P8BL@&! zj)3wDN*aX22N)+#Uqw?XmGDJYrIdxwE8&Z*O1!HnmGDJoOJVn8XAu#(UA`}*97>4k zhZ9YV2Crk1N=#>X7pF{9oO7&ISV!7d*d?&E~ct9o21rkr{o`$1uG&&Gyjegu_6HC_jK2> zppjU^6j)HIf;2X%j!ky`% z)CEblh8xPD*)P0?$_M4V%I1Q&*>jv3u{~pU$$I@3ur91aJ3_v2;a&6>pN{;eF|&Z+ zp>Ic(?kHk>Yr!pHH;(94VUSqAbsQx+`|DpjCvShYF>N+{*_cT&DD4Bt>TvG-le7nJrJYpEyt)An+7zndXl@=!z5Ojta_4_5X%2T6zfOL{~Zn@b? z;TI`1CyxA^P+OVw*a&G70mXmkQ&~_Yg{pnv!YMRONQ_kq0FeSz2TrYa3mE|ap3nxO zs#UwirMX9E0BNe06giKeamo&pt6N#*iVOB99u3iz^kL}oC5$`xP3Zrh%QeC|y$R0{ zWEHB$x1g@VB@zVwy%RK4Y0HTKq#G;4iEX#K=dMV%$Dw8WrYkabG|8!rAJD=!Js!+F zMV(g|)|L#R&W(Q}S-{-zL4k+8Oa*r`6?Xce-G5f!R>%-h{*$K?{@ubt&$c^8rZq~$ zOTk6FXo?mDEd78z8I7A!oaQh&Nrh~nag)NSL5|MpBtaU`O&R3R2G*^9$J7$|I|6Qe zg{R7x(Ze(*cvAZdp7QdFSWr(iEa^?;j~t~KG?}U?DtZV%0EcE=IwU6yJT@}M*D?W5 zyQQa^P=HRLNT=FT?$f5KwxCe#52noPuBXQT(m-gfdaYP4uuRl+6Y)NzW0U6U z)mKPZdtwxbZK$gFdX+h5rCmnW4=pqIDW;hSK@;_tqfMI6Y8`5?Zk!KJd>gr#-e#JT z_;|%@;L{C^Px9-1>RE|Jql9PGRikX$vQ0!amAglNA`rwWRO^u1Dwm@8 zn=Z}PZvnYQ1*g`<#wIa9YC&5z*bzFgalV@KeIb{0p(>^rbT*S>lIR&|t?r=Aj+E6& z9QaM`eH&TuLZy|=gQ3FuaMNDc@!;EPvz|&BwSSdW*1L_SU{ukM&Kz!1Nz8mf=d>(F zD>ndvuAjOSXt#utUwqVLlGZ{&7W%GYj+qnUB#gd~(73Sr#snypXg?3EtP1RMEM}=# zun(oW&;=e#>*S37t*O6uCMT!u!*^g0B)|9I$&Mv{1Dd0r6cYP4r*0uyz0*UP zF9!6e`gSR*YfpB# zgJMZF49L$pGy~H;+AyIZ9o>DMYgJjA6~EjUgHT=LGCxGZgO?SkT%@liJodu1B+ju`IYOdoeFIHM z40x5#K~}{SE~q3Z8jocsPx2{N%mxP42X6Wa>4#WJR~J>;*Q(?G=%8j6O&WB|n}ttF z&EA+EvqMdRwn@-`yr=YgzE1zUi2ty&n~eXkoyULpCh;F$1@RxYUvKZce!aV`?;Y$M z)V5#cQ6T=K<3Ie}ajs2sLGcc^zIy#1?CoX5f8cArzI#yH-L3EK*AD&wtG{+o`vcp{ z<3Hr_e{%l+jpG00`9E_0pUMA!l|S_G&bscKa}2g>eqH<@SpIwavHTyPe$?~$Ki}em zS)LCqxB2`Few$eCeSterVu)gRio4GH!ZSRYF-9{Uga*dHYAx8znp=32oSxas`N(ud z^UyP}{iMJ(+&{~JeWqRI;cVy{F1PR(=EiNC+HBjmw%4o`jLF=xb<@TcdD_h8 zlmaT|>ou4Ex%|)NfByNl^8f3i0Ire$`#an5{9pCm-R)fde~V8Z|1XdKm&gCh261^Lqu9p|1l0N=)8^mueW_1qaVw^%N} zFNYk-VS%lp$~#=8GdkV70#@b1&eQ?^T5=ZrD6pJ*nE2M1&B7aUGzy|2rTDMcLmCcf z8UKLzxw*2&S1Q2+Pp%Or{#e&)D@d)yrmku76f-jN+W5-xOq=uL8%rh^zz^QI_e_sz zA_a=ZN|c@itMk+Bw78DRpnvLIdPwNM%tZEIo$@Hiany{tObF74sgF`{5cUj0Lyi6h z&q~3|)S8lxl+fcv$%YAu^RetTUX+Na(Y!Bdch!XNA|704Wgbq8MblKWf$&F?qLGwYyl`rstm^>%O%sI4Bw;e9XcA@w!Ru7) z3gH@rH#kh0i(X51CEA<>;dnYSHtsCK3s(5cq<;`baT6XLRHBkGxL=dKMa|oih_wXB zdUrtZd8wq8OyCX7b3eQ6AIu)3>qnTwBUseB%oNA870&0Hk_kjBXU2rWlG3jxoI-Nl zlyz@0v*1BPE0-y0UTvLQ1q!M|dLs>mRcArG>+c_g718qPjwc8N5}uqX<0MfzS)OMk zxBun#zuf+pf4JK-uj-n{Y)<*`@C)IyE%6y&nRBCOULw!9%R+thBUeCZ=DlUc1KBFUvA-dM6)%VMTuqPrHZ1zoMUzHTh1y`p|m&p*1{h z4c=3<8t$8RO>=`0?SeJtQ`8)FEJevM_#;Yr@B;Z*uP_O0A0-3YTMl=?3|ABc72_*U7eYK?~f- zfUUe4+43l6DS zHBhN0e5WwWz_$KMuW1$G@YjN+3_P5(c2O#>^Ga;Co?h z$?MYEY2F8iZC5q)KQ4 z!p&Rp6&2p7BB!}hH<()R9$=H}qP)AS&y8E%n!wtadreH&t>N_HQOE8xeaBzEyuQJd zyD`^3kq-vN5cy?FZAh9HB5sB#AtFnWZ+a+CF?x%eXqwPd9GF#*2?u7yiBqg^j%m_8 z7hXxO$DJEhWFnriR65g-HS}2)*(G@WXoGUuC!)eh#-S#iyh~|7lpEU1IfjL^Y`=8N zy}oa0o2`_QZkf7PzBq5}TqqeMF+DN+!2q7rab3q<-_Gj_!~b2xx(J&v z;#hw)7A&MD3Kc{jvv8T?rHffkLH7C1*qO!Ff_OcYuTjY>k(}jg4|E9BHHH9A+a#X6 zsLN-&tW1i^p!4HkGY}%FTG0NcO^bR#+hMc!=bOwagb|KvbV_X$OrPJ=2qC~wDWb4k zzKG<&2^KJkg6L2`3)4T_mopPx{!xCRaYTxM(gZ}IFO-Kn(@%+K{43r?QFnG393^9_ zQFWSWQPjlN(gbS3_JN}TN@f{Un8|WJzeBEQfm-8)7HDaafmIR%hHpK*+D4VOxW#}v zJ90qZ2)eOZi+)zN5QDKNJY-Pg{zgndzjFcbEH9@7| zmigh-Kr;sDk#ox~?NpRz*32gjbc!LUc0M;)1BBf*6~bMWs++8-6rJNiVzcPuO<>3k zN17<*@78r#DBKxz$Cnj$ez6V;8I$NXW)hb2PF$LbBvtFvLXy@A_TjKioL?AhO7cY+ zm$aNjQHhW$XhBwT5L(I#8+xX{Tnw)T>{7}aRw!`J_ov|nIq{C377E$b@B{?K+K}>x z(;dDrop@3K4a}#R97buvODbTvcrh^@r~&q+S*k_meUu6#v21W>p+~nEN)HmcPeT3v z4)pKs3M>*4jI`fYg5m#8RiMnA$qx)9kg{AD(|94tSp->m*;%CB{1G<;`?$Npsvw-5^Vh zPV-(zdk|>+5=sx6ZNYBO?C}SE0aa6ERLB@%grdg2nEEnGj*>b>&P89TgE;;8?^Lx5 z#HWvWNftuI=I3`|2=Li#D5o<{y^>Z@FWpH~H!S!e6pikVG-E=M-7(a500Xx?@c-D| z%yMvHuM)oy8z4B82*Przq(;=f4L6-pykWB4>a?yrR2fU@x>#8asSRU|L9S<}=xG9t zGA@E}p;g$=(IhTk?%7`%{A1oOV5BiuMx4?`8Sy(YrB<{WS==eR*{H$j!Me1&a_i*b z%uLvYOX*vnV77jV3rl6^8JZ`{_h}g-?9b;KKa2?3u;=p1OLvyKp`TeJcXBO`x-7c$ zpQ?nYB0<^VM%5T`C3L*PI-rXSrRAh}ORYu|u~b$qSQB$53|NU9sod`DxDA}#V{3%T zSA1s?(M=zrrBvuv~m{7l>^5Y6oz0F3L2nbX@qv)u?|m?(@tGx+0$>$ z0#43G74Eqf45coHa^q2s!4XoFPa}P5%&plSSLZ(OYDrBrvA0LVO}Hse(?=uiNdUS8 zps>6+>_U7i^#CjGh)ikCSWU$=8xa*)_`G&Ocj&pV6g{b5fOxCd)^mV5mD*z)>fbqegz?@n{5b#+;r#9)lKYjbKeuyGwG-Z4uU<#@FbIMMICWnTu zW49dlXp4=Z*_%kad{EUOi#JwXq*fh)f^c2f$ofj-v)bw^ zE0Se%Wh{|ZXgi@D8fKP%zQLrx|om>IC*noel}%tKy12@lAoA=-SdIQDf@A z+;Ebk`uJ3HgE;XTl4&$bO>uBB>Eka<3}%n5g)wVVT;^0>zn;}c&BL)MLZDAIgK{zy z5}rrk@ffzCmDdUftZ3+DAVrCL5rHU^MQsEHx6R~lY(~KWn$e_1n?Qp~X;9QGNt=R+ z@K_Eecv6U)tx<`^%>%I@^e#lt7>NjZJmVflrU;(!)1N|IFkM&J=Rh>mxs#_-F1kXP%vj7WbjtZyMm54LGTL4~#LMeCuJVr+Or8UuB=we431YmF;?GR9DKtj*;%#P#8X7&_wyWK6!(r2>7qUkZ6$q>M8fVhny zKW76B`_a)abyg%tK$wiM33i*f7uxcjt#Atvq(d&QPH-6z^~-%^6e@rfABpXUvP=bJ z`lu>9o4}@=y{(Dbx`X(K5!(g5LQ>N9k1Wl)QtBzQ zC0wr34_-o#tO~0uM=#HSXYz%NqkCn1#`QdRTx=dDnSbMO$X?ul7Pj}D*gu3%{>pZu zc%g0YTwNbdQ2v(v8BhvMe1Ww)0H_}Lq2IGv4UW#p!W^9G5tY?#v};aH zmzz6)?sVfofh2AyT)U$^?w%77G$EC*C*vV+sL8(EHWIF(Db6F&1GFNY=M~jX5?P`L zCww;1n)tm|iGfpmquePid6U*c?5OrZ)XKo}oNuxeRjr_vnLN*k{sd#4c zKTRba)KhCA;?x?kK(#9`YOC_s zm^C1}Y|xH1RHpCHzx2HlVvsOB3ZomT7&vGv%;(%L8o1=ER!-SZeSS?APMmXl<8i7v zRygt>zwN$D1;MpY1i;UxFS1}L{{O20!Rn8}31)3Y%+Zv}NWq{S+Lv)JvdURbY=We2 zpmWGld4{i>R}I?b<_-Bv+FAalmDM?n*B+;?>0L3t$k~3)gQYt~fld5{)*9&J`sNs5 zLL986b2z$d&YVljPT9}RjoVVavN3;SRY0%IUb1?vR*Qv>6;8bZ+h#4US*fQiNxQI= zTago&h!hPI>?RI9QqPI-+@u_FD-x)odXr*4yfG|)s0LmY#mj0xokbA$nmcK7r!I6O zLe<-Ao@+pA%p=W@bcEwoHc^~KKoeTO2o`(0fas)*@GJ(#d z%2_&L_Ls2dA={y7Lc5@!!HKR4N$Fo=oF(+Nwf4qrq-i_&t)ma=eR_Z%+@z+ee744%r{Op}N+Rrv z-dT_{V}}qqMj+omY}84lR>GmOsCEhB+VEhlyZmu}5a%28Hl*7)dlhU?Pa@lDV{U^? z`BG=cn*rcS8z~^KXgA% z76Sl5V$MyNCm{F|B{VK|v6gc)^wWe|-NK1KCMAUh@HTPL7mG1Ow)uf>MF1{4i|n%p z5(C+QpE-w0x52?v6<}zgwP$2U1_lE1 zW!Rf6A=-*dwCR!Slz`L#orTe=m7Ke02k(AC`K1!00XMpids7mbM0YazHBXlNn7~v4 z{BT5!Ji z#)Loob0dJu`S`yaM9u%;zCGO4gm3m2nJZ_)kD zU{8xt51C09ZMlvM9L05P7S%NfVle%SK!`*p#PgBdWs97$r{F`nW{3lEiM(o{#NY;F z@D5}%(*@!jkpH{1ih&+y*!+o+0R!_KxQH18lc}!Is!dnE5c0Qf{wQN79IvmECVSL> ziV^B+1Fz$ofhkriojS#B1!kVZolqeapS*wQJ56E|YoY7NL71{GD4`j@8{~%*l z`%3|U=0s+vt*+g!`QS$}={Fx9sokzW0Ac*w17|6By%?+)A5u)@0d@I)#rxVAbF-^~ zR{66JsTQ8A%|_q#{l>?Qc7c9633w~gxJ1-)hQnD%HmjlAr*9Br@Imb3@%X{3{1S0} z-r^lEii>fcprSBv&uHkS8a2zzL$n^WxP+_%&+YYih+{aW6%bC9$HS7WE^-l^3I{w2 zvX-5&qY|z}lNB}}xCbdsElm4^Ra_EY(A*gsQp(%qzieGw+8HerSq=6*XZ2px*Fi@> z19xU9B&L~+LQ(_X|1yA>_wC@j=5c}9M+}SofNf(yh~l$`Vnli_9TU1ONsk;)0~0v? z+pAuhrdv6t%hcy@R z=Dsu4Pz*o`{bFQCw!_cry;9@S0UV1Y;ws{jzxQitpg8cm?%Z69CwDUG7kmjC` z=Vsp(kNhXmVR(c zX@&(m7ZDN5F==!HdtN=?lDbTRea^Ex@Xc;~r?W_Xb^N{JRo-pPq`355%w`DOmocFm zKodz{!U&c}#rm;iuS8dlA|K2B62k-OmDTBNm5v{LB*X!>3w|z%u!=JCA{h`MKsN3d z%1EQ^HU-bU7+e>~Da^Rr^V6`X+x7K`*cXr3YlTtH`s{-|x^sTn*rw<>p6kP|34 z!+-yJ#$uY5P$o5Q=zN}Ddjtu%?y`Q`LXWwM2!JbafJS?W|3DuLZb1@p#T+mPZU*>6 zZ&rW>VFCPM6p&F*Si~A4tOGP*IeEn>S^-UuuFeFd>Bz1wsv3c zwfOJ9NBs9T|6hy$-r~Qv`0p+L`*QsEHvdyA|NnvVKVkmAo$p?>^8cS*{%82{pPl%7 zCHa4k$oktn|KIN3&Nk%#+udp9|5pBQ<^NXxZ{`11{{O?~f2!t8NiNfHO4F>svLTC+ z-8qYA^C?+o^xk5TMEnkX-=&vFVG+OB6*ux}5l-Ua=*4b~UWpCqTiBiHl%;o&1@K%C zyb9}+w-i@obx-*(5EuFP`SqoGi~ciBMgiFdkfF*tG6;p^zxZwb8}cKl^+IkLo*{W92Wo5qvq+GCja%Z# z4IE;P_6{G=D>N+D# zo6=X|{D=hAqvR7g9@})CIk+FDm*}TSnE&`ni>P?U4)=4iNo2G4tV5)n5zvUO461bM zAxV3FFuE(_!Y31kSKUQ9>3s)4S7c?(b@(c*j~}rytUEaB!i)xUm^bG>GT@0hP1423Z5zs*EXGQ?%aF*GC)9R~NLLJo z8{mFlN%|m`01d8yZxnKX{|H9~Mm~swK3zS2wUHzm0y+YzX{%F3MUN^l3yuyajMbFm zxqxO+vsAzuo@SS9`Ob<h+Bbd7!DSk6LkHnOSs~&#A`%1RJLD_QUcY0j0BH#$TX;bpdnIZV&<+g|mLayI zc}gQ0!^K2xs(vVps>$CA**5x#%y|$j`KGcxPHFHbP9qO)8xx^DiyXU4xDFog+Z@E{ za3Dq^gj8gFMI)1e?T}aEcx-*5h(VI&RdXv_sCQ-!GXSyTu#Nrw}00lptU%||Vi!x(*+Y}1IVEb7if%Z$qqyQ5b1j!;cg)Y^IS(Ms_O(yu{{#KwO1~$-$e325NuPBG(S!4rd z6W`x0KpeOuZG&HVoP$*lo}=;b@U(#13pG{dl&C7xh>-adRUe1}jN#{)AyKG7l$RM< zKoE;}rxL`ct_zq2eS9XGgbJARrpN@d*K6~wu! zWxl6R>U}5F!Fvb$lL@xLKsHoBjc-&y4U<0rwuVwAN;I2&TRbG}51>|8H6D=AT{Y%? zJ<=wFN&j~4=b1gIMeHK0g(0tj_NZn_R35)DV*qhNiz%US$iQuk2YJNGKqAerpCfy5 z)}oRxVt%P) zP=H59`z@d1t`N8BflpwI_dfrKYM0WizsgDm-9hRfAGKck7b<9rnn0t$ZUJmesQHy( zZ`N7ON~PPdy1MO8*=QMj2c6zLp3>5bH2KFuz;vlt9yWoF35>nlB%;0RQ)pMc9%v!6HNhq4B*y8zUHW4&O3@Mm;l3(l~b+R5fT&H%WY}n|n@D z=j8N%oLe+-W5riwKe@wt1KI`+3xRDBE>bfa(SVM?=Bm|2Dj~GIn8j)bML$4pE$HcH zub0LpK*tZDMs|+6(3cDX4j~>8Fe7o$K&_w^nlbFVWagtVzfDrtYO`}~qf^bt)*7ji z*l zxQ+m$Ax26M0p*E32sMe!rdUCEN!C)Ag3^L{78Nyp5t3yU1a|$4khPeNdK)7)G>|2m zcqc8o+0j(IMB-R)bpig;#NM+dgsmiD=OyV}2lOA0{d)CgczAqp*5!9kPxgNvo}Rrs zc=bzF1S*0Nq?qv_Lm%|mBA&5<;AsL>30%(e&$litmBBlX4M~e7dKZh`o>parbSl=` zSATcfc(fI@)c7k{mNm@}U$ElD31uMo#*NO6eX8$|wJ-K2&*3ke9IE-!)xDJ5u4ljA z+CEw}FiKMePwu7WULb)nGo#b?fe73L8S|6&D8 z!3Ly_{u;s;#sjsgij_SKqsViQ_J^;IjzD|dchw{fM{rzp?}8!J?K?q03b4rRAb6ms zk?3kj^V>uyR2N-#K45VL!Ta)z-|Ax`vtX*eCeNI=WP*D*f0$7c!6b}Rc~gn6aVB78 z&?vK6y~XV{o0@^&+*-u(v(jaGjIM(C)hrU@S(yu=B3{!2@p=qX<+R{9qsQkG3m=UO zctAi`fJBhwlAv;0t|Kh71JP=;0)@Nkhf*_GUYH}c-Cy@_G`uqv;tU$UQMa`R^9}M0 zesh)>OmYBQIgtQPta@uH=E|UyBv)v0by@@t13lSPlHApdTS43bhBylmmrMnqe78t3 zR{J&qDHn=Q?GMXnS1&wPRO zymkPEg5c`v#yn#{ysXn7U_1bkCCM{gpl@ve%ME8hykEd{Mqe9$OUBD- zHn1t&puxJSE?x!Qm`$zu2>r)54~qG`#DLpWU^fjt7_v`<*D>h~yWQM(FhLA~0XG|F zjaU6CV^J!p&MRm-Ol0LlBqKnDx$>ed^S~c(m4y-#Di$Z0o!pRCL(%Q%*RPFJy&!ip8iXS#pv3*N0~kR9ZmT zYHB7Y+8^%*VcN|1%q{3%1~O&EIg+-9gCy@xJ}u753X#H&TmeIFtQ+V=)DZLn9L=*i z=ukvB8=&rcgLcBwGorZ>YtyAyX;V0l$Ucp0AIP8H70lrPby z(p=w(6AVQ~Ur~=nK9G7TuqYAWLuV4LsHFo<@aQ|QME{XgdC@$5h;JPmHPlLrhp^s0 z*Ag5^-CW6SKnVoV3N7U%dGeA%`6J|_^65!5O=d|cPBBG>H+F;Y_Dy{c;SZoi#XKFh zpJ`>2vP<4j9$^MX$^oF29INugrFR%fIy}a-e-=!8(O0Oe@iYqDS>l>(&(!zoO$b{A zy8XnZqqW?}lnhP2S1y&Dio1gB&fM7R`utQo_M1AtX?bIBUORZ3ye`~YS!wIcS3JwC zlP)H;STfIDvIJ<&UD2%dsPFHdD&dx3 zyC=eY)u_!i39V(j+6dvJ*W_Eaf4`d45v(n5ss^lHvHjRn3CzSOp?+Gnu?+Ly4 zsGd~yNp~ZPZ#Gi0`@^Y2uEs@{QIgg+&{R1vVLge;TWpK6eVH1D6pWF{P0|68l?iPW z$qJkna z24p5v=;tM(oL#r5cr_{(X}QGcDp0I7F1-s}#Z`4(`0X33gx4mPiM2LzclFvhVe_+9 zlxX-!R}c}T$fFN+wMUtBq>rm&yTEhH&kvO-Oi$cT3yvUff2E)Zq?FWpdbo;2!8gvy z8W{(A4b8EPCi4tcfdcT9rF^EI?<(m7E-LACo3r!2&y1}> z-O^LuPd1%;K7YH8Vv}6ua2cymGSRVW3>sD96gGoBRgU3XBh~}d;aRlNHH=)ne-Mm@ zeD7+sq$k5`NGHijvgGq_T$X$b|yv!zyXQ zwd_Ada7|j7U&2m7)X(`PTfJh0a0ZU}KEdhXPd^_X9n~;#+sxZ-MLHAai-QNZidATf zYqeIRGOcOsb{a!}I(3iIfU9~}AeGlO$~{WREs_=th$$U*zNGX0`q*O@;A+v?MP3MR zjtPr>J!3^A|dy%?`GnaI0(AD&63udIn`H*IO>2U}uSVZqExOrcf6vb;9Pthx?y zPt4JWQM)H==!)%{qXE^aES2*_T~@c>W=e&QFTny4R~AliBha>YE&AQILx=;8hD5ZD zES>czx8ypta~B!c*jITxllg$gax*BdfpOv6`;4lW@E%q6!BUzZf)<8b^}OM(r*E?& z!Bkq&Vm7<$vv38FT$0&*8iPfa@U_t$$k2G6@e~2SinvtG?ES?8dsW)3gwXaoD~E#e zdbCj9{?mqjs-c#_=CWYRdaMx+GNR0QMv=b)L_!2mNW?xocrpz?5>m4}iF*fA@s z5?vbQP4}~a8>oxg4DdZ6&J$crkSpMsoA7L;cy6}C_s1YUD4v5*8ur7*9HMIJais4k z*bZcyu}0L=aAe^fhVis4oL0g?_`1nWVk+P>)(FS>IrT5QutDQ2_f~2S){tXfAgTa` z?!m2vj?>|eTIbZM?Sg!*CIuD)tnhI09E?;HTd|8)vk7=pvr#7nc!bpqn%WQ0uOOe5 zd2ISQH8X1_*yKonmM0%`R)dqwNSRDA7hk}mR25e|>ERgrzK$G%JgjD4m7?bN&X!d- zR@q?B(8U{F5#(kf8>v*FgkO-pv*?TJ?&99B7eA0OuNNSxJxFmyagh~1(<|AERIggO z<1I-HO;;A#vwp|;|2w-?{QvD1|NpDt|9_|B|NrOicChzy=jG=1cWoAcKPmoy5MN&Q zl5~=7Jl*<#xwluB{|nbV!~frVvH244|1tl^UW@P36hTUC-{}W!Yy}7fs+vb1y8lPX{G6cWo-rs4|W#8EjwmQe)zwc0L z=!iR%NIIvB*(?E$0&)E$nyPa~?k6EYD!!isln?VJ$R==%cbSKf&81k3uRvD}8Du<< z2mf0(i+l6%k~DKwmh+;&v2mG{SBp_F&So1iOD)3)vh!+DUS)Z|l6i(+@5Myhnf9F- z-u;0;929ZqC>h5o(es@5$3GvR{CeDZD*$cy6?Bm|iYO2c15- zj%jh=+s-4)Z)@dQci_g!=^`zjy4Tm16s`0&la=~l*>#+g4O Date: Tue, 22 Oct 2024 10:21:20 +0200 Subject: [PATCH 03/19] New version of trivup 0.12.7 to fix an issue with https://github.com/apache/kafka/pull/16464 on AK > 3.8.0 --- tests/trivup/trivup-0.12.7.tar.gz | Bin 33672 -> 33994 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/tests/trivup/trivup-0.12.7.tar.gz b/tests/trivup/trivup-0.12.7.tar.gz index ed9d8d62e57ffd10eb083cf78a6b484d12c97288..657d58aab1e0d329d417b963289411ab9a95e861 100644 GIT binary patch delta 33719 zcmZ6SQ*dWNxAtS(=1k0qCbm7v#I|ki*qGS1&53PmVoYp1|1}o3rO+x5;vgk_kIw0lJ!pFJQl^rtHmr>FNb(cc1ymODN(L zXjkyA@gs7|$J&hZthQzQylcx>WqVnFsfofI0RL8 z@xN;W&FU&380YTpw~Ch5o+m5!-zaxL>!;Nbs?C-5G_b%XQDUoYOx?B7@dPR${dgh+ z0j*Dow*Yl2Xj|;lUiPJ@r}e|k-QN4ES#Lrd1eCoCp11HyeoWi5fS%ml8~=hHFCXcD zf!2RN5Nkj_wite1^}Pw!t+i%>K>4>l7Yvs=XM6MJ#g6H=E2nQipO552e7`Xr%Eq}; z9lp;uT!>7x;A7vXS$mau+bZ0PQuJuJ>t%ZmhraC!YmD<+5w*_z$<3akCDHP!GwH_XVSpRC~F zw!B=+?}0JdFN6IuBtJ9K6Su$#f%akb1O(@i)?qW~Els@uRzd@ExaRtoFQM@4-&>H; z3Tkpf!JCSdgp9y+Ga0}uv89CG!iPoU4GOF=%{#NwopWgr>++^?W;X-enKJRaPqacW zLfA_J@*@#mXAAqn>~bICwvh!|y@+A96}?6TT~K*o&o2~huJZ@hMyLfJfQ~Ccq}8zf zfgpw287)cJ>wAmYvu=K3qJbczN>q+0XrDQr*9ccXz7yA1U$+h8O-1T3T`-tg@9IL~ z)`i--OXaug`#kUa?tU;ZULKOVfy>fMut?-s`c(^Z%^2by<6NGuN7&~-UhDklD1H=T zTduH@tINC=RYM9|UBa!tfY<72ed5B_m}AdCo_0u(U61eTsoyEl@hnf%D372^EKC(B z-_;{+O?Hrfex#LHsNvzUP9-lTFBNuEuav*{GI)DUh`aZyuK$%f+YPtJ#qV6mT&pXW z4#KSG4y2He*2zHBB+$Gix-DZZ1z|idaN#OIJ}ngXd9wbQ$NQZaa9MrZ--rC-lD~cO z0e0I3!gcf5ptj0>DoFO%k%b84b1^y8txG; za8&upPDx0W)dNIEx^mHI=KaI<4RZxuc05txWs*f{yzh{>4>!R<4rc*t6Z-|ap!PC z4A0+m`>4el9$|;aszFCILhYZqXQ2NE9S~@d=d}Cm9=3x6jC~$oA}^ zKc1D1&?uf6%Km#I*6YNdNrx)}8?YXLkQ+-_UTG9@1j#k;Bm}SLV|npON2uKAWb`kng(ef8-8k61(k2qT$E;<8ziE z(>X4Z=;=3-D)mY`C-w82El8+4R+U>X4XD{QhWy|qn}}X9-Fz9;NxPoeU=Fuha!)~S z!hm@dnR(wXKFH8}=)HZ1ao#OGdC_}+VdgS^vfJgoIYK<`zfL6-OucRYOkD?@-W2UX zV-G#*+Fc4~J1Ua7^4-OslG6gLZ!?#dl{%m|oLuwyBo*EKY5b@VMg@y5k^JjYfNh8a ztuhG(+q-3gKV~6}HZ&%)>V$Ffm0VPx0l1-OqRfY((8LgA`v{qV6y{ssFMk@Iwk?`& zEyN>M8wO2a>khNc%JMC?_>_gf0vk*90i_!xcf=@xD^FE_`Z)_Tm`Gl~EnG8LApb@P zK}7gaQ(+IIc#jI%e?|C}2PI(5dsp5C#m64^0PJ2&mOV}) z@^N$ztdPpPn{>pEJg$xeaiZ_|_H6T<*g@MPDQ)C;=dq~q*kz*#bY3)hc{HqrSZxEJ zqj|l_Cu~k!c`0unqRV!+j470O+11d|g}a$HHXQ;_hxiUVUyqMTkj_@!zaMEhZ|i#fB>=|?s_#z( z*6RMm!tUpw9Pgx|gnO4tdI)$sz2ZC~pO>*c`)s_e!kW?YdBX(30qboqMOKh*lbsyD zTz98>4Y&F;oI@21vklJoDdYHHbB~Oi>jx^>ge`T!u3Z9GaCRzw_I2Ax)}C=ZU$lqx z!=J~lf>bM0OS+y(Df%WI<@@SKj~RESm6w8^I7$2C$CN_f=0p1M;^MeBs|>dis@j*}>xL#E#-4oCk;jZJX|+FcD|8H}g- zy2DW4EyTj8gmdY^=eFD9_DEfUS0)B#Tz|6g}Zl<2k z#0OBV2VnDp{?z7wFmHoEkGYNM-@$m8<>c(owGe9xwCdt)8+u7pX|mFBaYd~KT&EGc z9a)J0Mn-Rlug4=$58G6g2i9KQpGS`%K>1Cu3bf8^@1}S4Zwk78034t0?|@{`NaZU@ zHR$*6hs`eHOOWRJ^a{xSCcR@in(3XvcjED$b#1K*p8&vT6%9a`Ro`~BQS8prx*Jnk zVmirjNo{bJqWDV0D4hloURzDDB~U!y3#v_fry5 zF%e{1y*&imXjF>f_*IhPi@`&A?=^Moxff%n=u zgX}IXU4O38yW;X8KBK_Hmp5|HVfN-Cb2%l4lm54dQZ8Z7%(p@$WzqW#^Nee$xoK1I zF{~-LsHE}wr!sZ|`FF`~w9uM-mP~L5QwR2wy@GNH4|$$8vE0y0#taW411^+K`-1Bz zLJwGMdeq7~;_A9|K&ffk3pPIwc=vo^op`(&_-xEv1hqJSTxA{uwTX|p>h^B>Jv9d~ zc_6-zxZL}9oI8-=jsByX+vDvs^UJ5;zqHs6l=}ucxD|XW0<|WAe!tZ}Gqb$4I)i%b zZ#^GNtBa2kj-t4GaJqS+yYux=x+pncPuW(lGgVS{a(-w&^NrftCyE35cddvk8E;>! z<7)V@w#CnZ2`k-fK3}7s0)#kAz48L?(Ue!Dt3w$`ZZ*naTuL6t48d_}hN*5=$`%bl z{iQT>-HkNKAYUkxe>dk0_-`=!xbl8fA%?2b1fB7E-#pvbKAygO%xq7*GIw3Q1TwF9 zHAAiaaWzO=rl0|a0c@;fu$-jeE&)}jB9uJjJSpBf59?1GaDeL2!gG$OO;&|rZe|pm z2t0AU*&GsX>xXnN7WB<3xZc?Ym%E4QR$u=`##dXn5e}&Oswj?J+)NhDnm#=b4*{ib zbf(~QF(P8A&cU25jQ2q^fmGn_m3bZ|{0~)8?cDe0)wSUk%gI!bn%8czv%?0czkyXnflz)3c7wLJ$M$_2%yx)z!*AKUMr_I@C)6%ggC;AZBt z`5N#z_m!p<^!N7PZEJs@5d>kqC<=JlpVSdG3iQ?8 z1?1I}vJP^7$xQ-bIveu3ceua&y#=nYCqb{5e#D>a+qGAL@7v{92cMwr@lV*e7Et`I zZbITP3N70{~;2sfJPwOu?vu)kGzcVcqAP$zg?k2b9Dku~epuJoPH z4^-p5YAL`xRpaB_k~oPwd~^*q&4;y!d)tP|l4GcX{Ft(Jy@TeYF~G{`M9ohI z7hmYRYDtW|o#Tdz8cqRv(3kUE%!-mFkiBv$0E;l7>MhUHXdJtQ`aT%&Jfit}-p@hR zk|=@pMc8b%q>4Z_g6v0hr-a!i)cq*O;r@jG44VNg8>2#TIkp>1j6rYg>;?P+x2L~< z!3q$)x2QdR6pq2HhNaM9Yu=JriAD^VXXak_GTxeyje8)DkTNp!DT{m-t!Vms!ys|F z%3u93poS>@b_-)l=^RpU9!7{X_%Esr;rj(@XPX74jkHL z$Qt@GR<2z3@pV{e7_3gG_XVMFgPMw^ZqmVq*O|=~qen6{ur>Ve%+S%7Ek7pUwgTIS zf^2|oIti--CkNK~{)pmgCB#3tJ^>m8(;S8OiJws?|3deb3f?{|%wic`OF5ZpmX*~f z;Vct0GJU$lb!5)*d~`6w^YPL6PKaXl{kHdBMVy@;J>N6>if$7DlCi9;4k}!(y@M=j z%)d*?2?{_2vUK|hhAI$$9^(E>4GF)@_47Ke@v7!i{OF4^Q%)Fg7)f1jF28FQci)I& zL~)O zU{m~@g$^eAcj!aPE6b##FZ}UGfpbN!fB)r=Tbr^gW=o=K5B|Yod>Vl0NTS1uL^^6v zuN{!xKzEB<)*5An2iK{|T$o45o0mPAYDPfeLlKew1!g%b_=oN>q6gIHSSE22VkJ6thgL#=2tw( z>jIZ(zVQZf;~pX^<8uL2s%6~;v{=j4&N`@ApLqFhrByFFbJi>acPWiR9tAwo{WWO~ zgL76mPQ#u)*r!6nyb zrGWb(#*#fTHb+J7!ZiRF1?cbAau#} zIbt}%yhSuARs-p#_bEo}!dZ1nP5GOz9I#kL;)=QarwKA}LwR>ea^}%7RKF}dFjYgp?i#jG z3H>zMbF5pWa=E28r+v+HQCG_E7i&>zK{mR^r#D2$Z?Dz(L_8Qu!*@^Ohl)`nH@S1O zVe(llR~l$yTpdg^Yz@Tm#}S0bn-1rbLq3I}bC+Q?dj?Dr15qo^@~XEtHp((O&ev1m zdNsZV=MSF{Ie!lslaz4(#ySUA*AzoY#7RI%;3Tv&T>OlWJ=^}A28rcrBVr`|H*dxY z{PsMVDf8U6a{HoCW$Wm22hwY~d@JQ|k;kk!AaqMEhT0joSr~-C#^}VsVUL>xIhKz` z*&?1Z2oAhSLD59vmJDj98Q&v^`bEG+8!I$o^4V7t%M z{1~zOHX9`B)z!en`K3QDQf8_VH_`dgs84TF;{+ILdpF7v%3ybl&}w*qTvgk6AVPDw zLzLsrT#A6hD?7-v$~; zPlMT0ZVE0Q;95`Z^*UTpVTWD~I?hlNjOx&6;wfpcWLJQEkaC-XxTo=o!mHC)Jj2U_ zolES&HU^4|;3i7rd7;+|N3-G0*_?u`x?GNmuKsWiAbDUFgnTP22xZD>l=j{`t758Q z7GB@l^r<3EQ!%!D%eG#{^8ss~L|LjD?Z|?OghiWMzdyo)T(;N2PU<@eLO7#_xWar6 z&_-1{%AGI|l4UQp14ET8)joSY)Brumci=U;?o?vE|9upbMZKb2kH zb$ua8pO*@(N)*DyLR3O4iKnEa9&nVGNz}(B2kOrS4#5m`KTBVkUV+Q5wd#wjtHr0e zPt@qNbx=>thkmeoGzwQ@P}JJwVh0(fdq9}NVm17^k-ucsm@jw`e?RlPvp$@WV0a+g zJ`Uo(#U(Dww}1!%Y{fR^?y&M?{NRPTA_rMc}4~(7HorI!6ayoJzv* zF;{o7`S3zf^ZpcHg|~%tNIlw3Ct~0U13a(OtM!>5Y9I#>r`3MMCqWd1oZUaKlVZ)2 zw`P$A_{(|yH-RU;;&WdoeSA-L!Ls{%kd41J9zsJF${eS$zfY%Uc5o~m#o9&N`bNS^ zW=XYYO+EDdtGr~N0WP4shiSVS(7~&|2pTe%TX7?!~aIbJ|^X8IyS zTsY)y5{Tpy_Z!x>SmHKDh@g}`=j6r+(h!Lze7h~1-QIVdV2nDJy2o?+dHnu>S(Onn z1j2MPCKFEvLh_mERs<_beD@ZZcF#L*!|^yK4R)6cvMz%%8zbHlc#JNOvkuXe5aMHP z;x_%3W(kjpscNDC##55c8D}t|1^(+^+<=?~z8hp~*fz`{r4^9-IPWT~HEteJ0H!5i zmH1&Ywji%ecaCr;ml*I3eErZvo(97w)`7Ml6Rvw;<15K;oWR?U0|k4%LooelYL>--EZ0HSj)7LzpqR zQbty2G?7q!qv;P~#m^zbqpwk*vugrB4c1D)$?TY_Z;E@TRqYl`0`j$e_xgiLz6u*~ zLr^rm2|`5``b@w74#6)5wC^$TROvoG4WVrr?q|^aid)$`U9S5n*lhSj!dSzWcr0SJ zb}tAMFKML}R>8UmS7i%=%uc|%O7wqgOllTLbRTZD4?sCWl$EO<_y5+J{53AC zgM_QYY{=Tj)AA64+yq=`NsP_he2L52g4J|v;s=Se`9s)ESY-{ksTE?K>7_{g9(zE2@Aek~@f z45e2nkdWv<6(JqnI;;`VnaRomaUJZ4PW{(K#1EJR%a@edxIV_aI0Q|0H(X`YF&x}LLk<4h4$dhr1-nRW zpnnhnpV+aNP~4976W3v>snnETW4aKL`%@ILosBuH@-LgPP9$I5U?$iHX{=CapBk(x z1^}Fd?i_q&al^qS{`{Ys$}9PrP9&W&&#sKz$hR(<~bRe5|206kFmrlD-F9} z{qPS9nPFL&vrX4H!%6v-+(3uSx*3GOo64?BpPS3!Ha$?8`o7$c8rsFqTXWV%>+vPN z6iS{R-A~2vx#td#*sm)v^&8{DVuoU9-X4(5{xe@ijhmZF32~6I!b17nV1&c8I_~`U zVasR6X7BK7T4D**Z>=A&m1QV0vJh`?BT}~Tx<()BZY{cowe(ePIXam-kV}~;%7_S( z$`a1v{-Id;b4wXqt_KCN1cW}F2Yo&~Ef=zjF)w_VIqx3K2>w zoQ&t=^AW=RL1%pS@E2`|;;>joG1eGpTa;$!I(;0x_rb$6tvjxVxZtZig&T}0JMrL2 zSppoZJZX`6$`+LP1jHSONWD_&!A3bbGhwx?j>9!7JFR`D9&#EU@Nf!3Xdq}CsK1E7 zyq?u5wv;+4B0iwtV#i_FaTn(bYg!OHDI{v|G8C*D1?-NxNl252Z2M0sStS8j=yy?u z9t|)+sX@Wcm!Sy!b$m6N%`bK=mS-cjN+V%m*?J-ApBjqbIB-l5zZey0aeV5!7RQ|e zYAghqRL~MbF9xv~MCUXMTxH^fH0}Hi}T{xk> zi222zfVgs)Y9B8MtJso2nwHdBRt*HA9Gm(RaWTOElhxY`U*?GL=dkGeW+e6F6* zIxWM;*vQStNxv4OawmKMGeS=YusHKd>ryUMyTK5Z==(1sH%Md;~9^2B{LWPy1Ivh-SnOS{DFe5wTxyiw6<3Q zpZFEdw5j&DeAuA#d%y@p!af2gHh$hiyCpal=OL&;k`X57*fEM3Qtb z1|^KqBBb%D=lLvr3jR9?MTe_#$KU;I;AM1joEAfaiEsoAf4P*lEoP($1CJ9DH^}e+ zr~CkPLb?zcACsa z@YXkKQqy=pHZS|gm#KH0bj2<}!IbC;=1BziJ0{xjLIMK4Itm%o}l)$$3f*W7s>+cmLB}4@L zy#l|~cw2s=+VGQC7IQmOn-8qXSEn(a&7vn-Ft$a*-<4LuH~zSEv%`%wmw}fBS~uIU z)PhTQ>j@WH3KPD?aDoTHLBm=G6BS&maWDAyPZR|lyrWm7V4Tmu|DZ)Kr+;cT&IjQT zhxSH(`!X=IXX76(iX`K9p0dCR*F~!zo1tuEIj*DNev91KPs0tewHk* zgnm_W2VQKoFi#kIYpPg$&h|PO$RWtV$7>#SyrPr)4(Bi^t&A~R31=9bp7g8m-KrTb zqFrO{5r>L;O@4(<`dr**e7n9eTQ&1`+)({~4puJCAY%|p^()iod`z9?9yguxyn`Mj z-LQ{!=DHZ5?q4M3Xhl6XcZwpIB>eq|GSWWy)ZK@w`}IQUv9#|wSV5m?zM`dH8awQVhHG7tfY;3fe9^4Qu^}F{Gu~DQBeZsT zS-JHEC%+i)p(@%vZxwMl-YR$yz670hE>h-sR7a~Km)sm9{$&dby#0j76q1N|sQfeu zbi0$y&5k45vtL{$;n=z`u;Lj_?{e72dWmo|gcwl$4G+F!kFa4FY!j_UZJ47*OVPbo zV|4-%G47h({+xiVD-L88QBcB2mKZp8p>zhuWTOH1fcegaoR_+bfiZX_Z|5ue6lqqm z8N21EqC;TX|1C>Eykl;WA?1($f}EJ6Kye2CVG$PA*kT?wC(Y0tkOwsP!{8Yl@3>%v z)3i7%(^-1RcmG2Gz^%I1a@c<(a;4ysC|Sou&rhz0a7%2nxmYUmtJ8I$Y&?iWd%l1N zWu+B-m5;EY82CDAHqF=6U`aUsLx~NQxc4ZLO>@ocxJC2^E<>lvuA|%RfkAhYg#x*v zSbkbt-1e?QYL;*W$gV1nEDZfDc23R~{q4+Y%|wqhm0HJ&Tb!xYGLNJ~P3>euRr|O< zUNCkYvck}P9ocjvEI0ngHnB=sOC5S)U+ahxVv1huaFox+Q!lJtiE(ToN|B5GZ@v{G znHu(lF|gF5yNEAUGmc`I&g+$!A3)TvWiP z!%xj}iSC690!(8E`Z9Cz4d2Y@Zt^ySb>8}NdFxy~klbT88&qYxEb!X%;{LO#3BDg@ zX7mYN`&E-tQ+3TkYX2xL^Tg8Hb!)13hX)}ViZ|J*y-rIY>*v`~lf=6?{h)0dfqVJ< z@(C3F>r}D>us1(Mm7)M@i=>=(QTYq${w!0~egFMhQ3{=^Q?Ei@k>yLGBiMJp5-itY z2Ha&N6iHYP7qAZ2j6ao2hh_Xa+o<&gJ^1q>I=M)~!-NBg@k*=}f_9~UCi5Hp=`mIO zuZcZj@(3}tk|d$9MiE8lMNm8GV6S3?A6ywFXTbb90rMb69czIWoKR%#t09VeIb?m0 zzU&&8?*T^*N`sL|EFDEO)p&X-IkwOxy+$~Ovl~-ocoCrtRj?L1Xkowb>T!7#B@{YW zk}PCTf^&+CYO+*>eK*}*0xN&Z@|9g>leiI>yZWVnb{6$8cx!{Idp7p){GlLKkETHc z$jg5!fu+QG@L=ju;ytMYvF3yZ=#oZirB9>SNf-ItWQ(#){mTh)&zHDFh4`z!xy_6wq*+}DnIex36DMG-#E5+_st6O^@+km! z)`hwVu`8hud}O}YdyG=<)CIh{-AxidAPjXX!fzwc-2UVbYN&0zay#Qr6j+Ki{wr-q ziNN#0A;SKW91cp-a%v6ZB6}a0NT@+MInb%Y{5^+2&>rbDSiwEI9p@wlcRN_>dzfp@ zkf2Rdva&)mHp5XM>9K9DooG`%AZicqWXUb5j);c7oiRU;RcMSR?kvEWIoxk$V;g}; zW_>fZ>^PuF|2ExyMM@gb@f`yDY&^@HR*kQN_D*P_k=o<4GA)gURK$yeagK{M78sP1 z=f?1T$F4A*LbAXgW(y#Xlf{RQFjqhsr&86WHK(B(bYP4|lXkE9=AZ6iEwVjZahJr` zj$T`=B}CEbV<9i!S+dT>YUaPfaCyFBg<7vtolbz-;thQb;kd2>TcH^pt$h=3@+`Y? zn#O3gjeIuxc)9;vg4h=P8u0|vLFq_j2!U*aVNg&+#c!RcNp!DnwoXS8QyJ&m;C_3s|QFN+Fw{oXL z+e{dQ%;Q`iVS?;r;Q{jzS;RT-Nr8E>kU?rmrN?<0gtp7CW&W4KUrezF5s(C2KCNG5 zB_p0m2xCSFD~{L9^&x6Lchng$FnA7&g13SNhJw41TYR@x}Ld%J^6 z=1f#+zOX#2+rZk@dMaHBjpF&@>ZiqK`9+*q;lCA{n5?!oHMM2FvAms77T_XjF+bfL z+(MpSfV}3_Mmyj@Ls$ zd(Sb(EurV7{5k+3pH5`IZO|C1nYd*_okwAmnjte0gT=*e=Sc-1ya+Lskextc$ zu79cNI@!PkoK$zU6DVJ$%3A->+&xb1`#FfP7z0C6wE&OfBIgDl)|hVu_wmC8E_J)* zL5Pvv{9Ym1pNU6ih|d}$01OH~euskAJt*E9oKzrfC@Cis0CWHJQ>(Ohr% zH>7NmuBLco=Ej2HomJYO{a-&u*{O5F`9D7f&d3B9}QY$7F-K1Q@{9L@VrZ>s#l7zcr7f-Hl`XW%9tG=Q3{(q7|jYx0<~Rh<%TNcZcuEAM^#Tv z1hzUZfF9BxhXz5>zr-t=$b@UBr>m)|scE1MT9{5IQr# zijIsS=)|QLkCs7cBp@;d^FzSNc8L^aZOi> zu0wr>`iHO+_MWZFP;a7aAEhN1o_+$U(3QU?O!xo^IA?S8jbuW={)OdNGcN{`?%^$RauM&ZG1;J! z3XP1%WPVoL9ne87IP44AGHInXYAtXBH?u_QQmDHEeOcdAB?ij5r~%I1{N zMbB6q$!coVP?6Q(SlS=$z`OUB7`h_Oaa_7_BCXvfQ-36MpQFE3n(Hx4>Vj37spBTR-JJ&&IJf!Yruv>TWi3q9VESN5djv&-lFK( z|1hg~DTu=Ks3~tXo%wA!%kd~rDeMztQI}+8drDW?SS5dYb zxGx}rEZnf0;0rVQ?HzC^1fJ>OuPYdo45=0L@f{owbsz^GELRH{Zw+RF-trY*j}dM) zV3Sy`8^NZ_v)R7e%l2~lk;=Pv>-;i$tJBXo+Fiegpey6DyL@Fk+G;`9xZ};nNWS!E zBPWUCSz*^r=EvsPqep2A+Zxy7HeRh{t>jxR)dIgUz)p;RXg#o6d8P2Aw?f71&AY$J z%I9Z$d0xxyEJZ4nUZVi(K&w)EZ@fvs=--CCcxivKzlz7-V32OSSJMyl` zyZ#0kczb@9@`3c3Kt0NzJs({cT)(~7y=(8@f%bRBanR1*UtriWmunIv*+~44Ik)qL zUH0dn1hxKdZOlIU597(!aX>9HX?SS(>^hTiE-UORI7}1aG6-p(1EfdKhCA1`X--yo zdh{VDDzmc$IX|;AM8*0DIbJ0o8ACC>mt$o{hkd&(oMfu*z8i^}d}7;ov(Q$7(yP^N zSc_ij9F<)GvNE}juPLIsGU}27J+{PS!}Lebf{EVlaiTlo8AI~ON7Uw+%b75UKZWc8=I{R%JsB-{^n7Q2SLQ+E&;%TS*^>u^_T!nSfAn$c8Sp~A0xEdT z?F<7sKs#iAsslfx&VaYy5C7UORIjv*c2NZ!<1NE%# zhiTk1uS-acs1;vrM%uNQLito>_Udf{deV3E@@$#+Xi?$j7Nt&Y<{ks)go;5w|xr};U*}!dVt`yR6xU>-F`f^7t%~X za8l(y-1Pbsm-yXf4`oH8?jtT#eE;nUEQX@WfybjPDF#`lJP0|W_9-nyTL(&czkU{b z2_ezoKwcZkJWgHhrZd%tf+Idoo;gcCD;*q=np2#8z|LxusEy@y5!P1`QYMG~8vKfP z)#;Np%jP_mc?SIN$sFm6f);Sy^*KO|4?VA23L>D&tDd!gw2=MHe*rDYSYz$O_YB6% zcUf~ikI)Rsb&B{JeD62`T`B z+g-OWov}Ro&GUZdd(Uzk*aLy7#LEH#FXbz~Aj4h(BGN#fRrBgXNFYbI_U+Nd0SPs- zT40fZ!oII^c}}tFZeO}~;i_?;|&*+*sy5^5mk~=ShO@ZGl^I_o!^MuXP0O$oW4Gg|`1X6bL^z0@W0O^8N2xrR**DqPu$tF?46$ z#??Wj9pN1`D4nxc2;JoK&4O?ZXCV-}$Gg^ajb<*sQ6Rc-{aAGxtTsUt20CNuL6lx4 z`wdt$W;AHe7D1AxIbAh=O!(V}S!})zrN7r>$cP&uCkuYcsd*w-TIq_~&kN&KYfTH7 zW6hpo(&h?)et~;N;1!^aSHqh4aH8ye(t!sr!xx2nkg553GUvWwPX%Vw;dB#G@{z4! zOj@UPZQX-jdqDA{b#+G_xCy0dOUE#p~^)%rn9-_1QOS4z3mIyt}g|!HfAm09N^op4>+8vEz#gyU+2kZAk z3MAv5<$5OC&5%lgkpnen4NUyBTra$_CrxbO=Wt^7)sA=2nEI1KqVb*aXiZeq#iZ@j ztOSQ%npftJb?1(H3pM)HCp+4^KL=7dS}-b_WWAQea`JUAhbGr8=e}7tUzq%|+E(%3 z(w~q3UFF8_8;8LP^pn%|)U}EdBb#nV?HjMfRRvp|u^$X}RuWZqSW_i|X>xL?$dTh5 z#^OmuCe&(MRJZ+97#Xx7w5?D*5Y>Fi(iKMaS4$Kw((IlGa;w;ETYJgJ_106RC{`hK zZApYEPM$lY2-2um zKo0H+T}mJi$Xyq#9qx}&^*k3pw2#F7U)a8cGz5)5sclJ1!Oz^oOyj+SS1=VT$*P<& zqWBUfKRcSFlm>u(oM@jzu<{A$4n%hsrcsG{6;79&t-XTQa=Y10xPkb-`R zyU{vLa7QYG`mL7vQ&9-3p<+TWJne|2QbFTJJfe2M6jjel#RrWk==qll@|HNxeS@EI zugL@kU1lh@r)0jE(muu%S-0FA7$@8kAX-8kSMwc0HqXC`j#atGR4E5WaO4LL4El!I z&iJ6hA4hnHqlLuyGE+s5jRc3Iv^8x*%`S697n~w!x0dBJ-L}#;pAzr7au#p5zDU1(vRWoy2}y-krGpRnBR{mz zN9TWTI9bi)it|=0#DXPH3S}C4zav~BUfENS{S16fPO2939IC*A%=XWOu}uyqSJ*ot zX_FFGzU@iVGZK*xm<>>dajJLy1BA)2X8*83A5TU*YDnTkP2o@%ARmWl@*tvQI(Y(Y14yDvdJ z#%H)h|M~ZZa$4DIKK`hz7}+~+!qQY`MBGDq%#&P&RVls4(?XHt&I5#A50oD}h=Tf` zWXQa`(gr2r^Dlc~u3+mo=r!QehsadWke;E2{WthFdb3mwS3Fq@ut6Z1LxB3o820j)F17CF%O!JNQLYLm>HkyX1>$ovFU{zc#=WRSb+A;Qk9=z{Bt2IY8r0r*tm#^`^ z{r=XF>hHb^+p~-(OQ})o_i*st4H!5CrYJ1jiCeWJ44C`U(E@p&PHb*hINg-=pH9wS zKcw59Nj{zSjlcYO9~Pq+6R6E*kw`D+%q5fTc& z3iYmM(V0o=(T#0Zr+J^{IVVAS>wWF@lXok5xh9I^zp+nF@Z4q2DXHr2YSiYC*dI$_ zVoS-EjZ2$33ToZwEfMdn-TTy&S`>{!4W$d^Zp(M#x;OvIhy*i?vN~# zQN!7C)q~FyS}HoMKXc9FdqaoAak+Adhd&7e+Q3tGfDwQyOmmI)4EQ@nzM-sf*~vAX z&o=aa2w~;MeYb>=+Ut-(+aXlXwlvzIm-^Olta9y8hQ8KbGC-jh#@La` zPOcsOuh;lBDwMORRknt+t;AN3Fs*#Gm+-(JT;SvL&fg%0^;riESNF?3U74T2G?7H` zp}Iz3pbEfr6N**m+h+7brj>EA6q`UN-MPK3!(=A#xx?{9QwIBP^T`2Fh0s?ubMJ)EaX;Q4V7`R@uMd4vumQSka;3REaVg|&)gYVdAb;# zY!Kk-{4?Ns!f4a1K$Kf?sVy!H%dW(RtX6-9M=p_lHbs6N$bwE&v65{v9G>v=XB-6Z z;=U6wC?pukXeLL`5wtF4W@ggajn;WO$MOzJ6TspBNUY#cN``cFfA)OZE1lT}-CE9m zJgk1+pDFTg>Qi4#k2IzI@hIu(p+`{)Qqhmc$U`PfE$plFo<-Y!&33D0qoHO=uiFbj zbjqSgVHbyKtVbjuG5cHBg`ikWKxYIDVXdSSy4a&dU<_)UbeaY%w_AdaNV?HKdwU!)B_taFWrG(U?T^F z=Xm#5g23X~l4Jy^86QNk`|R8; z_`DnH9h#6^2}Ur@FfT@E-%sSE0q>_|#0)fLQ>u6C|$Z>?_+=F@#SBKTG!xh?@2 zo5d{u*-E5e7|vQ)iGGDL?rN=28&zL}a#O)QO$Rwxb>x`B9N8qv?2-QV%<8DbW;#tH z+o2)7gkR-QRi)ff{bJ1PX`?yokPU|WhV{4V;3J#zdglG1FRbr9} zMbSe$J>DD@JKcWatLUKF3poA`CBIPm1(?^B`iwnA>D$Mip8O(7`{ZKTs;0fEZF^AB z>mHH>=`G*%R&Kh6**$=OQ8*w0#=1100wa;Xfml(%_)ie`3=HQ6TRNU7R@O!-KRFdk zI8W`V$;s#7yr$CfkpeC9nGE^q)xn?oJ|~lE8=S-l@qEZn_^Z_{vK{lCEd*jzZueKe zh)6gIP;b-lVkZ$@Qm9}o5;D5UY5QH}7Lkv=Qv5f*$F<$6KocPtkS-bAH!of|1q>0E zg}aFm6q-{i`T0s{V3*3%q@PaVg>wnKw1WlT;I^Bp{0{(+Kybg@9E0ytf1zHY`V|(5 z>B~u7c~d=kQyuv$_2aMAjUTBO7dF<2|0|&prw)g_eInLU?r|dNa%j#kD0R)c`zJ9@ zW-$m{ykh+@+3-qzdnHyvrG7L{SpUHojd|sv%{$bjJe4YrXK|`2?wIQ8!1HX8GL9DM zQWS7`4GkhJ_ZT8X06fjOfAHBBAoC|!&Jq)EBh(0vz$uh$Nnq=d3``VpxfjTmRdzb0X-^{1nV zJP=7*0#mJka!pQEDAy?t<&?BMSq*Fj%)7d&6+Hf+<~T#FZYz^%f6D7_j77xZE^x(Z z?@;=S^PQ}kJ&RvJkmrdpho%p{zu?DACFKawUCxJSNf(y~gMFIoD?Pg8=Tg8`i8D%- zp~UUO@Bp_|3e&D?0%}DjAianaG)+Ym^Uv}#pc(k5*T zbOWB`uPN?sXErR?U2fybEOu$yi>7u$$J4$z@=1EJt?gjAsE#ljyXX*}gy0s3_ly2u z&jMMaFbx!2!E5o-dYt7Ds!qI-#k$!6BB&spFQTGeq1*Zff3?P9_nB73et5#55BkI6 zIV55#kaL0ftR30d$gLXZlDyGW)>(^{@EVRYE5p9RG>G`?_BV3W+Jww6Dh0f8juy2A6N+I-oEP6}flT4y#Jf zFg&03^b|qH7fSXbaoCu_oR7bw`+{yzDb)fV;;ChWNvlk1-@R)j(&wFgf z-o*+tf6MF4h1vI<>3VT~|0loylgEF}KmYXmKYb8;Goxo-pp*aI*Mxvw6aRI)zPB5T z|GK@qoyUL8X&ymNylLR>hAZKbg^3XBhERgAd!$(xN9q<&OcY@DO@L{vWR_TjM-3M>>D z zjENVU5#{uFaOgGxN{s0SHab4>hn_nWf9GVrTS9T7<3GEla`~**coo_VuW*86R7YXW z$6G~1kOrmb%#V1L@;a&B773e8OzbHR7D2s{qoUZH+{6xHt7Jh z5IvSo?byU{LE}yI?0C+O`#*-gpw97Kg76RZtdWflgcyyFLIE2w5HzMDZy5+F{V56Y zy7s!hk`N2_V$$F?&Qe-io5$)*f5RZc(*xX9h`8mFv6+Z*jTh5kGaz>m{Mw`7mo#w{ zPD=F{qo&h;qCauzE08L0iV|!YGjXaYK$adZXb8I+j$#_pt4>{F2QC5qBLUQ4MTcD$ zp5tSRxw&%lGCOjoz#O%K7ajw~NA&=|nXKRb(D~FF_B!vopb7sp{MdT;f3ejE?YA?0 zdwe{AXRVVVD(kSXYC@H?IvPhDfz*PB@TYUM0@6X}?dkg_X6%!O@fDH6 z6gI9DR;+cAWh-{mAtAt5LQm-**;4SW%Ap+SsXb;P9a!$a;T@mA(J% z&UPMjs)L?YrC41aJ^&_Mf3vRF)|~EqN2fbvi-^AXE8F>(JoR~k^B%beO6SFt!97{p zHra0!U0I?=Ka@(rHc+RIP^^NIR#K~a6pRn=>}-FxH8HQZ<^Z4lgI&1h0$=^NUS|LJ zhiq+uF9jL6$|~A~Rr1obZGO2eWA^_uj<)hoe*TxA|K;)j^3UVWfB!uDEAu{XcK)}s zx3{|&i~m>KhS&M|Uw;0Vpa13OfBE@ee*X9GcK)X%fRyKUmzax&SHbx$E8uql$h`G$ zTq6KAp@&dAF?r8$)_SV8OH2EnLN4^lby~Zl>lel*oJmd0}WDh zii-s;!IWnn;9}oNf2rWn6%0zhg6_ng8rmwzWTG|>`)7X~JstcvJ;(dt%wd56xv4e@ zWLqFLO!Z9fCG$KRLr4s9)6A+gVnO-^5gA{mZ)`A(pt!gH>UCL#09A%xj!!GcmBUKE z(jQdX_W(#%@*3>uNHKM#oTU*jeO>NG-7|gBU`bQEglB0If9teNgo%pBr&tC4-Ka2? z-Kgjx7CqdkXn%lZ;opr4Q`wD*wz25Z(MDx4?j96($i*6zwUP$1swV-Y=W6k99FVm% z=e!En0w^gd211ZGT$Z5B0b@!vfk&EAfT?9~P|L#mXrL#e9KdUOC>=cOVlI>dJ#MMT z^Wz&2WDJFSf5cdNGmK0j~NzA-`_^KQR+Toh>e?rFD6Q#}w!a1`FsN;f%6`j#9r3DC>sbT`h z)FQzR3lKxINjyy$h+T}liOfu>@KA=_YHS9kU%b?BST+dcSQ?mkG9PFGy6#V4BQl&$ znlSe!f95rsNgthY=Y3Jx`iS%`y9rcBOcXM^wJs9ngmWL7dce{Xah7;plDQ+$O*NPhKPTYi=6D*e7s zZ4dy{bmRac(-Ba9K}myf_yFU?>8ofer4qi#s+2PEc_n<2Rf%^sr4qi#Y$@!1>?|U( zZkO*1DTfk*{gBbbXpl&GLEBkzUND|xS{`2rkNEw-U{-4=IG{;9c4AeG99zBJtG!`W zQ$PWOyz)$1;Wvyi$LeTAVpYV@!b?{0w?S|58!w>hIIXbv7zH;I!lg>*Ve~^{bK@{Qv{v>8I`m6bl zYi>B5g&Og_aZQ1XsVZ@k)SBlh`3Ggeib&DSzat}71VH?r?m8AU63chZ=QF+1f4TC+ z%u&utWkV?y!s zVKbxSbwqyu( zZu}F;0%nE}3Owv(D!7xWu#=DN{6GaDr5tVn-oqAa&%573DSUW%AkBUux|A`rk23p5pd&6JXOYw9;PwD zf0No@@RXNV#DaRFVM%Wyf8;2|AZDtjsOTa5034ce>5!Z-@Yu*0U&#bK?UtTuLIFB~ zBAseWxldzNZ9$>fA5!6JjFH_nT~CewrNN@L>a}9Iz%o(OO~m_s98Mm3A3fKeWu;r&NnuR0a6RvvcZnffsOO!jPDEiN*AhPia}?yRZJ2+1Fh8^l-ZH8I*9|nf2qB1 zBMV-rw32x+R9GKw+6y}#d|PeSQz@hNud>Q|w-F0Q75ymZaFa@c^97yLvKX!000g>z z>Q12D5=wsYQOqQ*g@P>fUBw(TC&Wn@eIKE5VfBp(P%6=W9#~lw*yUKvQZZp4N_C+N zJeJl;j{dEwzm}7eZ2RyX*aOM$e?6GlvBYmcbJUYUV*lpU%@i6jN@W}R>4qAqrzH1& z)Ulh~>i-x6z}gdl2ONo{Ol0v$o(`<(7FWT)vR|%e#JaU&_@8uTj6F;5^ibxD0X?d| zU5aZRE4Er`50mOgVdW7_h8`1c@+27L9vP^^>X=d;f6MHY5dYcg zu3C0;-FvMboCKSx6(?7-O|jifQ-2_?s}%Sss;PQ;1X`++u4v+lcM0c`wI@5=L9wJ7 z2IS`q8aq!sMOR`F4ok|5N!~ckNiy{A8Jo=6dCFrv%t$^TVZxC%<=%vSukSV6t;V2t z+NmHFY;)eIMe}_Xiw{uie_>lD$fo2lvn5c8!XUDywoVB2r7S-}aemERPu~w}+KyG8~ zFUi&=9*yo2#+yf}x{d#WBC)#ADSyi$EnQ>`DY zEgKVlfy=PzaK@uNDI5oYad!SN3)2c{GE4TFxwCeS*`SG3-{SJ4prlmC^0@8<8Am8s z4%75s9BUrkeVu7lN?Do}zuXstP+j9PKSaWVmldd-r_Uxl_QJU&&aqZGLZWGX15HQ_ zc$M=Yt6~ZlR1y@8$C@Wk@+nr#1_sp!Zu$x7hgeBh7gfcRC{ieYlZ^jR&*MLQllTv> zg7^>HZ}wjAz5y}&y0%x}-p+$S{Ab61_`Bm=ndXAx9d3Q~{C~B#ml6Mg&-wc9tJ>cF z-mBg1`X6BS*Iw=Xf$in-AM*G=x%~e|@qhCCAG!R`l>fiVA9{FaUH8oy23s}1F8&Wp z|GoWK{*PB60P^^MKi}emS)LCpxB2`Few$eCeSterVu)gRio4GH!ZSRYF-A5ILIdMp zwHC~_W)|Kgr)Tr!Y-GA3^32>E^UyP}{iMJ(+&{~JedfH%!`aX^TyEhn%#7PMwb`EE z&b?->U`%G#TsP;~B2SyyoKir=e7@%Ef4=_b>wo_Fw(I|Y*F^zbv;ObzY{&C|)pvKd z^Y#B*eDe5zdHlaT{$C#dFOUECFOdH$3>qkd{;7*}M(Dqw2;&J8D0bf?BkomDJ`K@v z?r8(?Z5&3AN9R+|oiTHZoQXu z(^fd2Yf2^%t(+MX3QJ1AnvjL$yeaG6!nWW+Lo1glXp>g z|H7E(Z9U!me~|xo{T1v4a{uq#{*%l9Z)g8`Ro{bYT6?{l+kbNTFXTTN+1ui1=h1e5 zpw;q!Z)b1+70CbnI{XJri1}Z)cXoGk`_J#u{8%vEee-#k|0Uxwqk-`o*JWlmV> zcT@?2cX*!71Go#AraLlS$NM-L$0~S#hG*+Db9`ycEpOI*-|x3hx?JESV>JK-uj-nH zJ)?Yh_=WJ&6&|NFHmOOWufM(m>nBS|8iu40C$hO2km4X$C`{~1*jq&QE)XY zVfxlGccBu&-y0HhV4@#RPlmnYAB*Ma9~f?@VFQziz}cGU__nmM&<48ySf8m}6Peu`=QzDIlrO9_fmitQDO>vR342 zvQ~0Jh>D39DRueemKEq%vQ*0a5jB!;3d!`-6QC2NpiaqHeHVN++>3>O;d*B9YB>T$ zVGI+fpr)}F-+%GYrBisYs#}bbLz%#YU(mBuJ|VdoBr*;P&QNwM^8 zCJXa~)tAl<&ti*yM%RXABX8=`JqZgAsaZ8psV01oIB?1a&8G!)IlKr~|$ zgx51M?_gaA)#L~k%yzH!sfx)MbUkzc)dKMhHR}r4D|a+>&AE9qw5_R$#sO@%7ApaI z=auQJMe5r%HS_Jq9>`5rI#L{R%Q~pM6r7_aE2)jKdp%M0n{t;BMLr)kC8wrR}Ol4Je=%Hooy&>WY! zGzhIANR`k8gqyeGD=NHEMNV_2Zopdb9$=H}qP)AS&x~8$n!wzcc}+~$ts(pHsAKn; zzT+=nUfp2I-I!~i$OnVV5cy?FZAh9HB7bg%xI#pxBH#2-o?`SCH_nnVhL*$)QDq>k%4?)rA#R2cs6Jb%_5 zn#0M?@Tnb5tj$?MEb;Bwy#Upx7{JN(A&FKpeRniGnB$=)Xt~gP!bDN~Fmt*;TU5Ct zZ&ot!C^(wP&5`5S@$650idXmqEyt>fS)E!@&iD$=_+;kSA-|6=S0!8yOYyLApz8*2 zqOYQBL`v|%=o%6kA`mSMH3Q}1NPoncI;Artr2-Of=g z!luwlBV(&xTv&%T2NK8nld)hSHE~fv^f3#UIbOP$`JG%^tlEG?Joj5Iun%G*JKrPrla5O;4EQ1O&S1SGYn;#mEiE!I zOJczAt%q0JsL~d<7*J37| zy2+YKQ63Kxn?)aQ0z+;%(w0*GZe53k!ks~Pd|6><=j))5F^O(tCSfV>#HFc7Qnfx! zBx#*s9}n6j`GvuzB!6F&aY{=jimnK$3oWdb9E6rK!-k&eFBiir0lSnkhZPE(@%?Fd zK~B7*r-edxH9P^qWo=0L!|4uRm`*&YfClE%ObMej;UyI?oV=Kr4%C48g;}aa_&!R7 zk(f5Pv(Tek45bGN-6x@be+T;ab_FI0tDPs+k!RwPEE-E(ynn&^UrfwWghZxBTG7R3 z5%e4f*8LPPu(2OsR?#9<5P2STBue<@N{|tzwvaDcj9r0=QTJ4X|zmNpH&sgiE zT+t=QI_c%jbZtp<+zZ_xON>r@ucJK(G=2%C2hFx%w`cbFgT8>ODQi?H7-58>#=e;P zGD?n;$|7f?uYc4*WIz5pRqcZ1)5p9d3!!53^SdwvxNQ$5J0t6rw2FG^PMW%5!VjTn zba$i~6N>DPp|%4UxaEQW$L?m9gA03=_=VU2!J$MDrc)&~qV{dL>5SqH%yz5Oyz)?G zET!vWWi_NWj5P+iwoTE~1R7*OJ4ChWqc^es@ptzY7crLyx3&6DN(vobiXMucM6 zb9v>ZYo~7LZENICuEbH7MR)#Fl@N7FP3@if@--y zdR?1ypvc$+_PO?w4bZ@Z=S#{Z@-?@xrK!U6>GFF<==U}1dWb@JU>2hA>s9PBa0=?; z5KURz<1;O@zdt~|?(tDrHEwhdE2yTP1cfj`vVQ>-CKaiaFngG=n8@V-)q4Za&~P-k zqHh9E!tasEt}Gj-H^b}V*Ar_(`FZq+axe`&i5 ztRJF?8BH0V2$({u{+zN?A?DD~)$HfOdVeb-&DxSi2T&TB1*`Qm<@?k#+0=49vJ$u` zIEWDVV~e5_B#e8203LGBSL`QIZBv!BMyejXtgr>pWVw(fn_7CHnJn63V`%m!(k>rV zHOS(PRTrsMN1z~F6*jWI()g^ly2^@V*<2Y*WEI*@Xom*Q@-H3U0kYp}kA~#K%zwi8 zwUY&?$-a)NQoyW;uaFIfZB2%xM{SUUWs`RKTip_!sBiCRUkeLlpKFS#Zj8o{hv&g& zmopZf(>@+hm`;wM{hXa>OzX&^z?oOJV@$f3kP;+$b7Eebc6(>{b$^uwJ_I0^?xuY; zMye*KRnn@sqECENpeMSvoYPWc>VLo7aFV0?_*A?>l6VctG#aI*IJlVf@fRirn~$x9 zVK*r*bE>RgFY2S_;aC(Q(5IR~IT;EG&m-`73|r93YZnKsXy{}h#TEBF0#PQ5+6W46 zo5|nUjDiC+qe+W4fd-Y*pr~1rHU$%ru^deBq!2e-qY{go2NFT(U5K7B5=s&Bc*Z@9 zOc6Zcr$2@G!gN((*TK>xNNoA$%qx{M&ffT@v41!m8^bU6L0+w^G9rClSl>dB9&FQw zf(mQfi`GLc#n|wMGzKcSlb~T0e-LmU?GR9DKtj*-%#P#8X7&_wyWK6!(luFP(e#)u zWC-9kK-@-<*VzEWesuH;ofXLu5SS4*!EO`xLR-GG6>b57bjZck2`&Soez}i~LItqm zBeDHZk*R=8A5~>%6WElqw>42)cM$(DV!NPMNJ`rNk)>HzO1&esj=Q(Yf3SSy0nhzI z*p}Qe#%n6kyM)VC`oT-+kyT-J<>=)Z$V@(wada<@Yn;zB$HnG>$^09KL-yhZw6MMJ z#Qq_C@>jMK#S3kFXIih*I{ftIUgx{#fbzHO&wx^B;tQ6Wbf+5!e+ndVL*dFD?Q!><$U+lR`D!vA@`jr1%WWg!8k*uf5d(wNao5F+aw*d<#qqJ6c?PE?93UUoP`r9MZi0OlimbV5 zPRN@Zb3$_q5o%>PpJa4WdHx(IYhDbPZ~^y?5%-J@d|#P3wMHyZEfCifcmPYWwnZkI zV2!|%8hb7W6?K%!e>nZ5jue_ zROj~K{lHUITUEZstO3z!gLbT;!oEZQ!uLvuLBjMXjBcc2;GnHApL4rt;FACU+q?GW zwv8lzfAdoy@~J??Bt*R&hqv5|*Os%oIJV1DHg~6}SP%(He^`+K7XU5mT)NMG(><@A z!2qOe*_FM3mB=J8m=}6_diwEes^yg93>Vj^;RKpHoJ^eRSmDV3`sMMD&MLSR3KekM z^m$z!#Q(pkBDVUidKA_61dTc#h8a}lK)(!UktSz3aR`#KLC9;C!ZZ82`J<{`uHKNp zq@LvmKcF-We_@ZaJT9(0zR0~NgMuyH8En|dUvRB4c|6>_2apg4o(g$AVII%3%OrK2 z=N8e2?rv{m{^s>r_X6)*Z+9>lSkSS;saN!NG_WNrYGtYFh2_3wSX_)0m>S1pis>+p6WqTPh~!~hAKpft{S&BNl>#xx_*av zv<25x++4+I{3VfW@}Q|^B#&iv#)Jr5Td=)SOR~{LFqyEyIQLn!FqbQK=0V+|Xu?0o zpUR?3e`_M+tI&x__fPD1{3|QZ(?p9=v2FM=qlmr6*r+|tcw;WobQyl@=tFy-9-#*} ztEnoVZ3*XTFo#c*DR#x|EXbL$$A}ywkU!q-)JddH!r`*0aS7tu(6H89|F|^71s1)H z`8FrAT>XG@=9S^Kt@z)yIXx$qQ8#wPj?YJ!qJm#E_CS~$ImaKR&btD*IR31-O7)L^% z@rtXM@f;L?ufR}?KTJPP5d#20V&3~GPeAZxhtRk*#ahYD&`%R;^$Rclkdz!2pgX`t ze_yO75ZM-lrWOIX>|*MkJ&+jK2K>l7Cfx=HPb$FhLR*j6jtmS2&iKtdrg^c}tjOBc zr$qQkl{t}K@>l~Y_=V`gX3DVdvxJ>ie0G|?@Vyd{8lbZCcoy%dLI**D!^aY?{+f7f5F>M08%yWR%t;wd7be4sit&Z6TFbkb=mDBL!-47q{&ON1is@rWOf1tgo zSzXQN&=9NdB4QGFzn@Ql7Q8=oCyYP*?@kDp^U1$?h?@VK|Mp~ewDs}sU)jG~zxhlY zPE^CJUrm-dGV}kHt$d6&y+QXki#@F_`p8UrsLSoj0_l<=g3FQ7?@1D zLaRDm{X*E^y7}XbopHP&B~9^Y@DL+3j}5YpZ$`FQO*(ag+X~FQggfCvDn9w~Zs;|M zNuq_Grw3unx}u2Q2iN%L{1`*rB!)v{gtn5b*Es2j=1hlU%D<>2T;XjZe-jIC9s0aB zzL;kht}U7~XlI|gnr#O_#v=C*GG^Le4gj=dGCOZ|>t-teKZ@C~*?2^|U2gyp{JRUz zQvP-^+Ae-eF_8z<6$BOUYirESt_DWskA6#y@LXdyhJFw-0E#B!)<6@F0cu*Mmr#$qMM$HQK5U&R{E|I99 z>-YOI#4(!i3K*x#<8jGW7rO{fg(DdSS<6m@sYD*4*$P{Z{G*ho7UuoIDlQo>XzmOT zDV6QYU#_mL?TitMOoM&TM7`(rb?_1Jz?}sOiD{;zkZ8dBpGOe$f4&z!Z5|h>eblhT z57;(Fj3_>9C|0EB+A*QqlJ>~y8ejq^e|^?Z)3m3c27K>vy;lM8+rKlB?_ zQHyIQK#(Ow*ImxPSda6~J8Za!H}_o}4b=ch=$9ZnwjF*}?@5ho2k=@Y5f>@f zMDH4hAYClw5cKZ{e_1xNy0x8mL4h>)e6qCrPCW7-sQaYGwIf)IN`i#dmy{Ja{nl#6 znr3|+=*|b5W@CNnlNOByhT$I7i!A-kFSQyL^ju6tEGMkd8SHuWd`q8Y3hZ;9-GXm+ z<2zkU>8q3P6|c%}V>T)AQLd4BC4B;fn&`soTi zmPbSYTmb_bfA1mw1AQ!{21&&gOTZks8R8GKSp^n^1@MPeKu0}c5od_73DAV)^cAaU z1#Ah6!&R#wK^5XfYD(m)fHmzV;1d=hk#3#(5MVwRa%I~?Rlq`Ue8Hlc0b6AS)({qg z?=}77LCi)65w0gS&-noN6~Y=Wx^~$ByzkG>g@?`ce?*2}O1zhEza{5Wc8&$p?Ifi7 zV*b(=GTKEQ-@!1jL{LSgjhuX^8X#~A6WVSb`QQgZ1LZJkNEFx{=XLgy~TfT@!wng_x1Si zZT_cLfByd`%Kxn_uz1U5Ay#V z?6>lNEC09he=GmD@_#G;|I_Aws^&~dF4Jhv)2zUgX{*t%lZ^(~8>xJAhGDD1vUoo@8IFe6r-(ceE4$;y) zeoNxNp7#WVf1T?94FV(HgFL}ReQpWEff`%DEE4Qr>y|RPfkUj7`~yqCAlNP z<*fe{fUeleSeo!vSf4bp39LIX^|{0vbSj@xCME3a_7~=ZFc; zx|W*$!(hx?N*@{U#GEJT>cgH3Wlk3prTleBK9U3AmBW%a;ltwa!iwSM2ekg*f z+23>BHu{MyWe_a;M%kXEH24#zf7349wkE=O7CCm0aUC@8yBx&%a3Dq^f>dO3MI)1e z?NCLv2JIKA&7kqzTrtK}S}bi~`2IQ71Xj39sx zM7E?`&qn|}E_+`^@448G6@pmp<2OH~Lo!|mAC#WBVaEmMBL|7mscf81f8ZZh$dQl+ zVix4%ZpaD9bjTP@Tt(Ug&yS>#a}K@GFU{d9uPU;~KxV$)7ha4&g`bOu&Iad#{9z>W z;_mr9#v(`>($UxpPNxc?^jTC0G)c!0FC5oXC@eU6n4*+gxJZ=~s6h;ZKcAi7k31gg z70Ch8cZ4TFcqjbMHzQSQf76QHtiXIH1>w9Ln%TkBez?dJRzTo8L&PQz?oOR>0_$fE zMr8s)*MS&RU%2|tqiP+UQ6)skiVVu1&9kW7LGt&s?{X2t;T!Q$Tg3_n@~U}aM*z=;ga}%{2S1aFpkQ= z`$aJ+Km`UtvWQJ#N;PU0^=ZQ;6MS-iD^L+57idJjN}14Cl;g=_>H=m{-`}l39Jpm| zgI{@^gH{ioWM5YoS! z`*rOe)FO7V)gqAB;PgndB$CI^?HE9u&|=Og95QfQ<3S#=F_1{}>zBx0lC@CsMeMIh z1_)g>AR`Y>BhXMCfz<9pCgF_8=AiidhtNP4u+UHMMVO;a2oqYXqe@Xs(e9sJg%p{> zNwDiH+T_Rve;6E$vw9;4cwI_Azk=j|^c~a@xIyUah;oRVNVCBvD;drX`uy?1=w)zDe?ePl0*wZ}1+X!7%&!D{i_RKa zDpQBmGj)GtqZRNSe0uYE&P%V-q(?zj7TmqdC82dNLl=p7Ppk(wn5KLkhEcKIqH1Om}uy;y#m&NP|Q{+8zD;! z!k&UYf1f8=s{<<#hDC@5LgjmbIz~|F9KRI=jpxiLN#ooXP}OjPdY{BMrn=`%>b*Jn zPwxg5+*t7y+fQz>-jKJ!!$M$Ngo|9w#!f)5K<8@I#VTR6yjsMxgQ_1Ow-(&G+3%-u z3DEH)s8QHaSLTvIz#+r~0%oKP4O|Oap&3iRf6Hcm8s#@h>N|CIzH4+;eeA4}E1BID zl0}3xLW(@W{*H-XKtR_YX ze`!32eSiZiNBEeRa^1rPo^4W8O?ju!-wKpPars zdiHZw1S-M{NHNoe41LgJhj=Cig4P7661ZG9INLpUR0i)jwj?c)=v^&#cUqMV(y2IW z-~8Pf<1tp$TIH`|S@u$rnW=e$ zaoOQPw}zLEoU*P|Z6D&vSiq32f3^40!bNe#PB0#5D6wU=wJw(%Z>l{{+1@|`viB2h z*kuiBbbJ#FALqt7A^f#e%u?%iXR~#sPu3}JQ2AaM5ONiS@u~}jabA4n9XVkh(fAh= zECm;kHu`HQUzjc&tEyPp<7hhV`mdgkpS^kor^n|$P15iR%tar4P=xwJe=iJK0TG!U zhIiZ=iLQn`zs(ed>Y^v?0}fXZysvHiP9GDU1(W)kJo4VM3GUZB>faa`Kuww}H?HpSe>AbP+47uM>^hVG z>=(%GwIcvVNl4~2tsBGoK@8t6l&(b*G40Lf4mW+}b7PHsBM%%+VPD=G2LwE~hRyAH zf{Tkgbik4lCS811|KznuW7;fRrPI)%wlV?ssj7|G!6a4LYLZ&j*O)4b%Dk&*lTIY& zLvZN7;4k_kn4^c(e-cP~FzDH;g!J&BOS!mJM#jKnN0ewDR*~v)Cz6K$k8)R2P0B4# zHhmXVu`cJP3dyazlNF475L|sznP&}1mi2}sj0a${BzvX@^lc1az2Xdr_Y0`b_-m7I z$$B}>3O2$G8nl~q@e*`nF}3z1{GY%!6w7&u0k^5ZZd&JHf5<)&UB|35;&yZ2K?N}e z2HdQewO;k7jD=KEomV*N2$59|k&OTk%#|1Im%#oC(I$68fdfr%q;!cpj5pt8`d$xe?s8^>_^2UNvy1n z68;ox(*vsLe;S&W3=DM)y5(^x9O@hAbc1bO)`JHfG5wqqj`a)Kpq^rNQIsONNd4=cTf;%Jcc-6Lwz5K{uwz$1ksF%|Iu$hp zy#Pn^Yzb#5CY)Wc=NF+X!-X+bCLNSwEgD70+LVi_f4hMx#90>=w@&k08JVCW4Lots z)dq9d#sE?>pkdaWa@V|P-m%AQQ;=OF6b=(9R)Xjoo?_rq*XI$J0LE8gPU^XaGtKcE zo|D0Uj@p9lLJxbp&>{?aP}^-;p6x(4w)mA!(JN=9f>gS)Q_6K;tmh!a3X*aq7GB5! zyy{;jf2qB$(vInlCRbiPTT*qx1`Y7ekB#RZ<3+bDM|iQtF$DjWb+2FY4#uj5Lh*VW z5}fTRnG-uII52Vc4l4MRx;1-TQKa5Gu3Lx5>Il&Zqk@mvXmk$>?ver9e^EM8&h2K5P2pbRaaS%@HgM(uscN8$l14a{WWOt&Pc!ISWy^CXSAmlS$u zYI1P81dv0vYQ-nYS%EaFld!<~W16kX@oXw~KK47#bNRc*qfh2(5qc--t|0QNsPHnh zf8ovv=N5-A)h20f=*0kBNxi2XVZDINJ?eJR2kmH4aVCC z<{ly+K#huR9d_HavRT<Sf1s5dtLmyt?=X^d{2J5#IWX;2U!kr>YZSP% z#5LKTx$o7>2(}1x`-w}>XnBY!8Jc{rd@eZ^w*}jsrLxx#q*Xf(nmRvlys?)rJiN_b z7jCVhv~%Vwo@LHSSCd+-ndcr`0=(v)YSwA$`x`J96b{zSE}7iIp4{%oyWYpUe*os# zZ_Es4cdF;blyFPX-7{gns??Sz3AOg!#|d@9B0R#6?CkLJ9>^59XQi2Cs+1j(I*pmL zC!F=H?+GvG?g_tlubx!Tlm5;$e!r8l-5;zDIgN`fr%77dKvU(wgv}%>Z?P@T_H}9) zRxm~;H%muMR%WzOBpYyA$iafze{P-KspozBtzu{EYv8YORM`zidd4@YBS}YclFR_m z89LdD8M0okI&)SIooucQA9ttg+sYKpIOD;?n%d&T_G=#8w6F(OF)r?UHxSOtO)#jU zoZ+Dkk6gMQQ)j2Hj+6pf$P}i%M3r-&Eh=7(L?f-27+(cMTjTS)z*T&Df7XTHfwf9x zZEBe~Yh!meuT2uRw5_5hxz8THj}6+4;d2;^gkh{U2CZW8-G=m#?2JFX^;BTOw$o@Qn*bK?PRc55 zaEJYpn1hJ zvfqixkqXAu5=@0`tFsS!*f^vooDaR1_+7Oj8o*NlWa+R%FB>B=JC*GeolIC&Jay-6 zHKz^q@Q}{8IXCb7%-9+{TXxC&$>#HJAm6S>bdsAGm$3?xiB42w(I|;i7=(vZj*(j< z)&tbhb(}`GFmm?7j(jM9KCGH1e8>J%gx9Q<HXRt!fpz;#!^6P^LAH{Z3=(Pp9rt9&lCfe+oo-U8CGXLT-_?U_j!~ z;pR&w->;8776Gn~GrP(Q(OqJ~;=oKZs$L~IL)o&-JqZD_*%Adf-;Lj#)a|?ab&Ey< z4C02ZUhZ5|_V?et?smZrr?zW}22^)ssn`=uS>1hG94btF2@#06 zv2cPLfw8?C(eJJuLK!$3645rYbkU>Sk?YjWU36UIkn(sT^8to-WFtds((WOz}^gj!@fxD?}FH0+0~B}CP7b7bHVYzMZ@I3sF#II{8%!+2T|POIS{0#oItF%|F`XN2SO zjQf|JyP$E_dn+{uYv?hr5LJLf_u$sT$LUE&tuyY_cFw+5lLD&&R(QC24o0eqZP-Pp z+626*#i)}4Ji=-Q&GiT9SCB8tJhuIunwhl_e{6CjK*y8!vDM%t3sPoNEX5b_D5c^` zS9>_dL0}?>AP=kASGB15CvVqzHl}Q_Z=J;(T@mzVq8q7HK*BFr-$nFAb$9VUZWlkW zF>eHravg?@h2EOgzv;3Y>{#7!G(*ldUIXMP3P0mihQ~3RtlNa)~@yHvlK4T4> zaIZvs);mt8G5Bl6Y1t8v^G;8Dd`O)i>qmImS-+z%5B`8z>v5Jebu&@99W|H47 z%gnyc)BKG8ep9xr{*V1><-b<`YvsRI{(p1Je&(B^TzdSlQ4U@F6J9di6PTiTK(?E=sds1GS+>Yl-dR;gRn>pOJl z&2L==in-(Je`gt3mz-U4Ot)hoJKSO#7FSJ$=Go=J-$RidC4sqli6+MzgGTh<-hjx|5pBUZL1G1 z{|&18zx}-@`>p)(?R8@(eND&W5vlG#wzj&B3);fX z)z#I_^gof@nQ#4awcPCCjt)Ql+T(s*e4#$Kt8Qro2z*D6AJ<)d-cc@|P> z`P>J9TAvd?0P17VP7ugW_N%wI_0!YM&g(i^caj$b(13)`TKGplr|s&#p55FUUqDY+ zPvAG8%Pqb#A}_Cv|9W>nNz^a3YCs^xuN_xxw;C6Z!^ZixsTQlpPr8r$jsYS3)Teo) z>~IS`jYf@p&P-Oi6V{anx#vdSIfURG1KxUpH1Wb>J6raN ze9Zc2Q~l{X0sqs`fDG}UjLgI>a01{ps+JPVdbD-K^xL+Eu75pVpBSp8(Gz6|90yMh zUf&NLK_p0X>3qK-unsN_2vzzxzbCZlQ1bpCD^%0&yxcaqGzfKh)7Z0H{%%Z}1U;vk zBU1<#a**O!SP%J9fzMk)=eb=(-j}TP5*y*Az~uwu{KI|y{NeqgXtaamS8N9uV&1dn zXrOJWp260^)7+ZD^NUU}IGC$9o4(Xm>=rmCL6%D0oLB=szsIDItLNqOP2*#U*CNr! zkFc&w-w)>=w^J+jroiY{{t9ta~BRH&nD0K6K0Xy=>`om3J}U#XR1{_k<8d z9prOm0XjiCQb5PLu++y~-i=XHWGasod}7-)wS}QXQheDj|ISm|6!*Cb;Z0EbpP{ z{LsP-slRp`79dM#-eFg;{S=l0pLhgmgJ%9G$8Vs~Wrt^*QsG&~_l;9f-R8N&Cn!(% zzo6Z%!{6Ka$Eq3ArH0YQ@8)CPlGb>U!K1n@*pbz#!F6$fMS7*2i}sEpMTW9#&TG!nZ&;^zC{A zYpq(GnCSL3RIwh^uD!&fS-t7w5U&+|NOwED#x){-EtWuk)7{sS$droUbL|dj&G)Xu zmk2!Kt}j4et~O9C_<4G>iYZFg{Al7CN7u;eeEzOBg~I1zs#VUwWfPsIDdj8qfGa^# z+EbVFIE2%xj<@?Wv#QF=bZ-nlvjrwPiwNKmdUe+3$;K1P*c+g}OyBcZUStFRq8;^$ zfo`T4?9qk@r~;q4&Lbk|y%ebXa4qT*fziu)=mZd(=vQ8?6>gq0zw*6_${ib6bOuME z-`qsnd21&-o`$s3ZelMd6;yXL*-y&r_Hf`E0)xFlxs#y(1X!K~>6W^@)ECZB@BIF{ zUPf`QrW5`lKj30sc8&R%}W zp*W+!ZdDADz~MvMwLNNAkJ(=wcJ>P3bT^-b9cdYNB$$&6!T|3q7hA>dGcq%JB#<4RL#U3p|aHTe$^RiQny4 zsUW(}pS4Q2s&}KoBM268Fh^vls~h{@xwBPKel2$>UcUb#!NqLj@{t_p46RpJ z4?98?Ha7b?aCj$1%dZ@pjZt-S*!)fdv&_G_GwEFrI^TnvDOJ60^ZK|tlr3L@_XjObOEFB%+C_|c0cB@xu)ZS*xk$&-{MZ5`Ly>UafFDxlHE8IxO|&z za^PI-q?$z^k_`@{+aUtrF4te*q#LdEbUvJz6VgB%1ylDH)y)v1j0h6V?!ghEK268m zdS5+UAkhS0w6VW9&%e)xj{kOlLWD+Jboct54emPC#c6*wkm*0z6>I|B!)3F%1k}gI z3QY692$&)K_H|QtVWO!<0LxLhS%=VWrYm9Im^=4G^7XRKw}-D_e$TahoXBGQ3jN9A zrmZ}so^`^QmR(o@b8Y4gD@7$vd^Jg7Z6qK`J}@8jotPBm+;VT#kK0;C#X1)-@BW;{ z&uM|aZk0^wmZ+>g<3|9jE>5?x^dBoBt&mEEyuO}M2AB89vb8}BKlQ3(bTFuOlhq0& zkcH#m!Ux}UUQRRmH_H@|@VANl!JA=g)m?^idQY>x`!F8P&>j{CX`zga9CkwMc3$WS zu@oTn;bfhO(|EWy(TohIv>0cXOWvQf^b2!x1PQKq@jm5>Rj)Ww{_y|31Gm^3!dbUIv6q9x7qul2HB*7=MlqXK# zM7|BOa@YJt)pe21JK17CYQqu&1gY~BXR+|L#LDa8Ra`oPh}cN%jQ4nEg$N!xgtM}d zy{M@Jk3ewai1UaV-f-EDH6lHv%!A?FjWvJS_E}j=$0&gl6VbyQwGlJaz#gfff4VeG zw>A6&33RhTS-c}b@284dCwxhEY>w}MEokNv^m_c<2zr(yS!zB#y1)KRuPmT$&*f!j6+-nid(OamX!t2DPpK(DwK%7}!e9KHyMl4%S+R9NoS3LjacLlP*35Iy= zH74kz{{A(O&0w7-AI+S<+g>2g*<5>*G)rfW+K>(6#II^pt3b|1ANeU<|AMO-Vv@piOBfQ;f>H@wzf-+q-rnLJv?M)3R}$Q#3@I|A4p|nfFkX zhD9lMFcm+IcLe$+f8_7iolw!EJKDV^UK8 z`gd{!hxtk(@4}{S)^y|f?J_1q$tj`6674om`#6QJ91+3Vt}s~K^Vg8U)-xFkF>@I- zXa{!R+<50111b{`1q8uiOvkV`0*$jMLI@d9|7rsE(;Lgpbk>iIal!sM%!e2izAmSE z9I5q(%9QYPQ)3JL*1V}5GjT8L!CCap*f!?K+H0CF(D8-%D4OTWNe}7%Xr15qs($F* z`~uA^fd1QQHK5@Crbr)zvg`P&1$|LpfpW3n-DRBzXLo1RE$wg2g3T`A_5+2De&LZ3#(S=l?#--H30X4j~MZy)`0hBxWZvvuSc2T2#2+cFs09zW=fXjb944 zf04|APPd1I7rq*MJ9{%X0LQP~dr%hWI2S+$QIm9YJFCZNfc5;h;A*b>a{udp`Y#Bu zzk@`5YE?IYX1&SF{}$)3J5V@ndm67>huh=758xVX8SruIMFw47uZZGoZkJylet~u- zzG7QI@q0S)iHFFv?1@H4=}wU1-JVDs3|h)h+CBYX9ar-`HM99(c1QE%ncO3SN%g!a zo#t%DM~LvzwNtm8iJ~a9;Gv0DD1^pY-h5Ymsmz_x6{7V{*9e!1u_T>=NB4qY4!+Z0 zLlwOZhO=VvI@)!>M_}^QZSr7*8L^3kyDkFkSf=Mn@%&)ocHRB0^UXq?@JT)Dh@N}a ztY^C-51+?#i`Sxs5cA(!AE%zga})}-r?eO+cQ7+z_%j5LO*q_{x^ieQaXWW=Sl+6= zH0<_7V$`6qIia(rm{7av4yn)|@t`;65&_Hg{e+SE&I}q6@-#SmI+|tc27Yj4hrHJV z1|Qc`{KWJ`6I4pk7y9`9`g~%o z)N;rFz#$@ZL97wVb`~39=*+5^3LQ3PJ&6;jL;yFt@V>v+zj2YITkIHNO- z!T^rP61)&v+#X(T5@rW>4vf{qG5PgMv0qSq!te)@B@XKcI|nQKVP#SpWtRhSfrga1 zL~^Fc=+Mo$#?FVZt8Q&i4^M5sFf*`xdYyZ?W3MUN4*gpEo1bhX1s!u*N#53ucP`Z~ya#vos_tI(nS zP@2$ony(TSMV-9K`qxo)gQFhXRXWE@eH!k>ge#vwF~e(qcct6Gt}1Bw2j%C_ApjDO zFDIygz(WWTqVv;bqJpPaq+3U2P-OS7Cm!C`D){A)ZiYqJO)#Zu0b%BLfhPd#Vhk!2HA4 zkq9H4@tAQ{Nu$XtDb$?|dQe^Vfsr)$B{h!V0Y2W*N0Gf)G)uRLspAOUK)jsRESM>b zC22#~VN5isd4?=J90ywBDy7*d1{?~(^2%mH3tR@mp9byPkfGrFpEB7tRsXjBmV4;^shAjs$V>mvC*Eho|ML$JU}g)_FaV|V zrgrw8AmG$azo(p@wl!vf<7`@gn!N+dR-KP&|JlaZlWq>^bz>DU-fV}PKzlgGfCJv9 z7kfop$6Kf8bK>+!>6_o|CJ}uNFT@lNBpg!)*yr=d;82OfVg0AtpwPw|Jy=4dB~jmQ z$MZI>SR$B-+>znk#QXqpp{S(fK=eqtHk3kEqc#iYV?v~rE{;vO=bNOMxIFUo8aETh zSrn8;PVfqJjM%_puKYw{UfDUA;?4kgEUE^b1{?jr8FjeGu`!?FJ3Siakh`L%y;bH z`HFGivqsxv4oG-YQ%g1kF1mQVV4QEMPr@0tAvswe$VVXhN9A#mcv-qc2zL3|8^2E! zt1Xy~=vZDV4?gF^NIiI(=ve;@BEd?|IA*58xfl-UPHP0_BOP9YvqdsE9Ktml9>rHx zHy;W0ou9rRVf;P&0fo4DCm$YAV&!&I?)#yhEmv{9Q(F^m!;uT@X?fC|p62=P*xhb% z390LRD*cdyyrt0$IJx=zge50eLaUtp~UzsbR!=Oo|yvBiF>hJl_`7gu+jb|io zo!+l=+&q}XO)O*JdnxQ>X*~b8^}?}iI5T#~K+A4toB3x`_7J=qnjZ)ziV{dV9NZL_ z;gcFV+Ij&k-TYovBpJ#^y!ZUM21Yl;iwzoOrV{@g@o9)fe%LMKvLmYZI{ZD3T2@u5gORZ}YhPR_(C}y1*95C{cdVC z7PojjT--ZoxqiEjkJd3hPZ&EhT|W`4z?mBdp_P!BrDJGL+sP?EuV zgD?_9aBu-KR}!cHZYUsSADmC&XJl8a_jmqBpC-`YkEu)0&d3+@0Ka5t=6Jt^jj5$h zLU?DNIQ7{wlq)NXRMof-c%a|_^M{ijtf6pNfYSjs!h!h}4hx%qxTGfyv3D;sZsH72 zu0UCF3M>gq@<7>4Uk)4@PpbF}w$ME}%H^Rp3Oh#KAiX;aB8f<&><*d(f=m3xR#PGi zt|-9!Hocn8`Ck1sck@sdjB^DLVhRjqo;qPHu}x;uN0_Py!jB@9K;lTG*4;f8heA3- zgSeu`HyaS3C6i}aEnc#XGCMezY^1nH*nJ{kq_TeU=Kg)`)dsz1p93CWGuoiZMNkhF z=BnkV-L<~gJ81^Usj~-&I|hAlz3o6A0D#TP{fnN*N3Fn@8Zn4I7Szo3m6|{ydL*&i zo3MrJPRz`OI^q}b z61^dPznrT<@j7(-k9!7BFSF+(xWXh~F|VrA0YhAR7j{QNH_Wh}Mtl1e8Ix!QfI#^i zyos@}M@sHvXdm1N&0&31QcW{~8p06IRHh0{Phnvdl{kXRGs>{I$8D*?0+ZH*!iGrP zec!1j!g{e_*1>5j_D-RJ&pRySTKknEu!UJXP{T6Afk%T;XA`p$MBaalD}lE4 z2|dlHR+le=cm5@&>}7sP-9-!Dz7^+v{*&=lMaBa`_9L=?RRq2s^`B_C|;=3 zEDGJW%tJ367$m|Tv~!}4`}HRnA)H|HfK3K_P@LpIv)u^o9gLi!OI)-eE|%W zTW$G8M&FFC)uN$qwpz%+KrA+aw2lxQZT*u3Vvi=59b{`Dd5ZlB1)VGd^a3iP1PGt( zL3*c*NVSArbz@O@#QB41flbJog>LTeBSoD8Yk&7gktKVL&r&^uEts1fo0NtnL-NY) zlfUV`iXcd>7scDS+ED&l@G#Q%b!CBy@#qBpqda@oeg6$tQj_^TkKfYHl z*anZ}&D%X|l2nqaRJtAlMJ`IB~@YZo=neO@9s2+E->3xx$h?jRb;ta^u z$~o9qej>l1Lb0Ac`0>vqXNzCtWYvFw8-|9&&)Lx_6{G*7xq{IEG$C-65EvFohUfw` zne`x*wIIA0FNgHYLgreHapi*xF^x8TCuAORY5wI5qOoPs@jKeC=K09X;0gw6$Z!kJ z%?zUjQMIT!`tR`EW9oyIayB9o1EhY$j5uuaG(HhP9vRtf7{)<0t19`mDPk0u(?DAk zh9e)Du&LiH#dll4-8;bTNgd7->v+Vbp0H{I+We}u{*02@PrRLCa@4GtWqou0JFDW7Bi7?aUW4oSP}kGPS| zs7Yp?Cg8`}Taxv)+JiA_;oz@1Z=-dmj4y?fr$_VEF?i{X#3l0W4oPKWTv%+N7FWEL z$*7p6{jC7Rz%4?!M$&0w@~=9`Yn(&A*R)sn8gkdMa#_zO>8r_}>0>R4T1e{yAk=ck zeAz0{#B?kD#h>6Mri6xZPQ~t1)aE>UOcM?@DD$7#_{qy%tc_e^w=3H59Cp zsWNuK@VCT7wIT>_r|{^kW)Z*TG$fQ>_ytde4^g!)$;O+`tC7I8AcvA;3YV_USr`% zpCyH<#lS-NH%lo&dXGEJbc|C5KJs;#ML+`x>{Vt_vQM}ein>IkPqIblPHTER;3`Wj zEL5l+kOQhp!4Pn9z{gBWF%g7TbuCS3^XN5s$Ju^j{s^Td4Y|2wvOu_ss_dMeddSf3 zt_^|#)oY|y)xNzWP`dnE(`CPKMf};+2wt>!H=dsIV^57%*`@z2L+^?+TknGbYk3U7 z4K_%bL?eph(jw+g3qlt=E9Uum-0E+M1w*i)=dh~Z#D-1U@XN?Lg@bA$xG*A+m)&M@ zzXuYDVxYl-I7pkXkD-Mv@EE+`eYor;!ZQF$vCwE~)tJyIu7E?Hg7(lz-rQ}I0dMIn z&$xsGjF>fD5mKp(OROEh%q$GJAiD*^zNEj|VIlD>{=?5DK=|j@?+EAZTyz|T{5=Ez zo&BA6UbpRr01ZJ|Tr*v)g8R3nlq?i>xul;}7#+>fAe^SL&UnQBga9I>MOxXQ=$E^d=qa`XU$6Fb%ZuPklB(lQ55DJV{*y@P zrk9{NGm6wwnZ1)W#W3zFVeN?kCbsh+Qpm7%I;zK%qx;<*;A6j0j$$pS82c^T+;dSe zwVeu&tU(e)r@YBjR3AF>j$evfh)TMP^f60>HfoC&db6`(P2gA%U|J$JtDey@l;6L7 zL$9Ezk-<8ZFlgiqsZwTecZrTqBu@8aP(&XqLY(;fGLc0fc#rn5LbmYBM8V@?yDRcX?^M1FUU6s5-`xB?+ZTqEEcQLG zYu>v(PSUA8up=xW%9@{kT$HX-#t0WuW|lW?9IYHzGnW-<76Gvfo68+xP{&|6Ljirg*F3-cEWD4<_X8fh=awov%KI8ohB`bXj*Ghx?3<@qOXynEjfd$p@I z2`(nLT2ArYTp&~iXz3@JLyUZXHL64v93VWY%WyABU&{_tIKrHXer zJ^jMXYBX=AM5+P4O~5Nz5%oMQby!2tM0%%_E)^0(E=m42dH+`yK$<+SXmH3d%TA@I zd4IT+XXsvIV~Zgu29AD}9S~BMZG60xEy<(f;Q0fpt{~ppCjW1XYy9*mIt=^V7o3+; z!{VSEEZQT+jB68aH+A7;f-B|wg1+~~!s}1XB~=!92IS(piN0(hPlp^73|F=XmaHgV zL_1J2L^4yyp>C%R06P~$WDEU?Y$Zhp{S>B;lyGv9=pXE(^o)B-V0^cHJX)GKYqeu# zib}_3?f6JDmTcr5vNL+*NzLswTV7dnug>pY=54fzweOAWU%lQ2-?2}yjs>xW%5)L# zPo#Q&3ez9Cb~isz2=$ksb42_4pjWe?j29#BG%w^jbOk&J5bQ8HFMYAY7hygy=L^Hg zQ0X5u+BU8D^xJkw6u~zcV9OGc=+KyBNLkZcXsNjMkl|!+X{M!_Pm!XxR*WbV1z83o zvGk#qoYJ5wnNFLO7wx%hUEOvcnCl3N)}GYy z1_zIgc76B(q6!Ri2_-+&<5pUZoko$}-q%CAFgX$^U~XPdSPX@$jO+57iWjJM*VSGl2WbL zUzLc!+mfL6o@hL0{4!vLt>-TBJ0303(RE|UDrxx-F!Ip0{2C^546^7^h8yJg47`d0uQ%hMcsPO^aj_#dG+s!gy8{`{t z*pJ>Cpja{t89)l@^j0)kCVy0cfx)`_zf(Q>h9s{Fw0aw0-ez<*ue>Mg!|J-@2wP<~ z$NKUBUpiQB9|3j!?05YGDZa~LILTc2^DiC~rk?AX*>3DCG*SlSe^wi`nW~OZj+frP zjfeG`2e&awAJh=7^JSBY_qJnCQ+BLg&(cDvISj4+h6i?IsNqg!V*{P$F--@~i#>6m z#M$s*{0>?!%aqmI>rCx=_{#DjXQSt)QdBd6!Y6wl;h~5xQ+pi8eikS^JADi~ayCO} z2a;1hw=pUY8$&t>Cu;_y7z(mH0u0Th5*c`!pnTSR7~SN+=aIa(j+8<(Fp-Vae(V~P z*y>faaNA9?V_My27DY8f4^9p zpGWK8j@RjK`G;3ay#;RA`zS=zeSyEFl;B#S*uh}0!`$2IM}^5u-4o%E+~{Ed$E&x$ zbkvP{7F$T*K^c&#_(u2^lLUSmym90E{isTVfOHEXCJl9P{XS#2DmnkMF=DjoMAAQ> zc$Iv*eq-X>ku=F6r0Y&9{+;qVzPG(U>S!DB>;s8&8SM!3F}kD5vd$z9kyarI_v`Eq z68?VE@~9Ar@@wWp)MC`+^H-1n^T3J1PaMzhfl2vhuVc7+=V}7;*$OxY8CGsdXznkw zStrzt?oS~(pZZf(a5;HR-@MU&+Fb_uM%0v}__nu?wBaQNLl!?4^J7?Mq=5y5>X(ys zY0vQF;0xO!o&_nm5w&BVM&s-R{cHfvzA3tDAw1=^UK!n zD`BU@WNcdpv?maHi3B*E(gg@KnOlmbY*J_uD z4NEl}If$*F;dj4Xw3H&NG~(j5?o*84q_!;yYyA6S7T(Z2N+(Pny^({1G4od6zn1tx zaz*_*R!BnXKeq^m^%)O#@>~K&sMTEtopV&cJ@z2)i=qrY zCq@(Ujv(rM>t&bcj$r~EcL_X3bzgqH)Gz4o z%-F%ium>_WjlUuvp5!C=&Qh6dQ`{q!(J7J8eM`D14CDryQ_^Ne_162sP2Eo33SM#F zDdGHe(phXRdQchyDWCtvit+KMP$Qdf-fSf>5Pje0x8@+eXW7~?%3yQ|b^ql|bIsXD zI&!TBfqae?4AZF70rt9-_i{m$P(}i!Ba;4Wfd3(l<~&cuqvw!0U&K#@cR{X9@%iN6 z*449s*>5;N@E0w>BN-B+L8Q@PwlWOCZgq&W!8a6=Lybf#-NefQQsy0_04h1&Z&rlh_XFQMNXfo? zSJW!oFVb{#1+&nM)5|7DNoXpB9x*`Zmr2ZpM7gnr5f!}2W0+YNy)b?AXs`bHO=2v+ z)gl$pPUDVt^K2L`^e?}S38)nxDoY5YA+ncD8v0{Aj8Y4;| zTnhbR&Yc6k5hdyj>{*xLiuGa=4IH+92`C}}e|fu=dGjJ;ezVSG3ZqRLtfR{~~su49N63h(qi5>|?Xr2tX9REpWgcu>}5jEu)sCRwqFB=cwpV@@Ai zU=4akeu@{N9$kGul{^p3lsG&D`errxmN#jUVxh?+!Dyouu}OL^=yx){3I}RT01ACE z0DgpcSt9q&N`o{p=y*Y&HvkLgjl< ztwo0pI_^X3Y+iCKoj$lGXIl1o&(2}SKCW<*HBTg|!Bg<-p}^|PHSbzC-hQvF_F{zm zeKPrR_+>Y--f@^YE2CL>*jA;2Qv<;Vbf^y=AL2$r$B?uaR@%cVt{_F@T23ANR1|)& zQ@AmK+)mpj+rQRsS$pnMy%yUWY=xkEve3FF%BS3j7M3)GIpXe`b}>(WL>!*j;p z1)?P_L`W9<1(cdoh7`0FXZxaQR-xvP|44=L79ZrSN^DMG@m!YPSJ$Ml~@&tY0M85>0H84 zx2{KcM}dGUuF>c=K?FDZgcgBxPib1+X?Vcl;zh@~V8m1r5`i+%E{2!LzlhZZAI&E;bKmL6lzKg~x8jVi!|{&?rmj!VlH0 zDd6K?m0DxFlDi9FoS4xE+0fj5FqZBp0!STkG%rUd7-Ph5yh-44uqgH#77| z+d-ou^82M-2IG_ZGb5a@kRILj`vI3In{lwVdw(lD$x%d}1n+^2pFRSh4(l9(MYn_B zY!`f%TJaU`s%GgOq1p=XQC3#Rp&g%ip<_Xt98D{m_~$SonjgzF=o57(mt#t*LR9!r z;aA2k-Ef7l5%sL{N=6#FWE!s3bb4xJxTt?PT78ueWlIC5A1~)7^kj!*vJig1u}-T+ zIPyn&AD;+AoB^JBs`3?(?f8-Ja)GpUVT-gaWy=wpa#HHomyP#II+~XjViwP9&lUO2 zDQb-qm5t#yOgD;<7L&bThW|s`)Ynwgh|DWSIuH2xi*`9IMOqzL@Py0=BRQHXM_m*r zS3Ca`!x(!{q1`#>rqn&n3|oxFYTrcKp8*Y1enIm;19aIkX!6oPm}CT|Tk;sW{@sPU z?HjZv;?k6*&#I3+qT4(?u|+s7drq)jx_oCJX@%-Hn*p}*jzb)Bp%*&18`bZM2Gk0A zcrP|r-XU-+r3R5LV}bJna-Us+tp-Mqtmk~Xq}@Rh-GqCuFWA={dL>%g@YdZztK2BM z>7C8p*!j>6^6<>8m^R2rk$z`g>!Q+qE_eK@*w;`QvcBuf)6Fi;Mrxw_fYxSG^wTuT-_U59V68 z*2nI^?7;P>N^i6;vtuBCjPI*q5ofo-3@Q) zh6_0`glS3~toK|s-h-*qsuz_YDMf#C^qM*j8%_Nj;WfC~Fc@iq8`a~Fr#)C|py8Yc z8R4a797k@y_p5l-SRg5W$JkZlFwMQduXF7N;oa%zW%n%#LWqs6}W!m=Bwy(>DmqGN|`De&q zr|iD-x2MXfmBo4YpMjJXtm!D(A4f9?xT+owyFH3L zqfO5#J|0Rhh0dE64PU!;3+Gsl%m0H#KABfQcfV>t5|6?jpwB({UB1s+U&Y9C;2r3E z{O_%T>TPEVf1FYehb{e6QBpY`-k?8wwM<%$76;CoBnD}tp4BPku@cXeL|Jx<@Yu5* z_YU=Y_-MnoFT=Ao?hyWbenfX20}Je*(iGp2))M<9e4O=e7@hT^Q^Ey1O9`9yuP#nv z_83ME3OaO!Y1|PG)(PPc&BKId5H5fsUQulkV571Z+C-XeOzAMq?fex7FYciStt?;r z7Rx_sesU9%Bf@&%KQ#0=7Z*)2G#6#hwh^|j1Gso}pqQFCy<9*tw>E1Qr!Lm8lj@zs z;twIJZhIkzBWO|yVbK*e`+fXwzN^`9KJwra!I(*?XGUH3cMf7RQ@nOHphMtI??252 zkUoFuZTc*|%`Ah?y!hchYkPpY+;d=O|Ni2amTW9xOUr*ni`M3=S{fKJXRK?SNe>58 zii=)R0Y*|n>40=kc&~i75bx>~`y20r{yvnmP6@Tl zv{x;8*Vz`Su{OzH{ zprAqq$9y@MB$9& zX^lal{pEE$xTz*STGzS| zEvmKSOPczE_5bZPyD?~68#sQ|Il9-%HH%s&P*L|~QMxtEoK3<%TdaCr8sWEo{n5)# z1H}xE`|HP!Suy93|E)#tBOeKxos1y>DiJ_{?qg$rO*02>#${RvLIU?tu3Of(r}m`;WW#{+T4iTY?N( z{6Fqqz33a#k6<6q9sgrNN(QB>;Gh%8!*$4F+`d3A|0Kxb(u=nHtFreRr%m8$cLO8n zzI@|4$zZd?hhg>*39uBcG+T5atY*j8ddr{MhmWQ^?=h(!BM`MKiD zJXV7P?;Ia8sbc!~bR84)2ep^5R*}YDcu1XND50$}mTbB0nn-sO?(V{u$GR}H!jo!X zZFuaT+Rh1i>>v&|Y@6&t=_gRYBpR$+&6ZnpW0b7nVo^$+aDMkQCPrSbz%6_q?n1)H zb5Xa0Nm3tuW?$mfGzIW{AbQS(^GmBy_b#vyre0o!slxu~uo=>73tMF5dP&ym(io#a z_%T^VU{9k}WcNrF*Pzh)O!uHqKw`ZRNM2d-ev@oq4=pN0Re?5=J`8|+5U(jn)kom- zM!2Uh8vEZO;WgtNnV3D|XO0f|;TTcHIirFHed0oiy5{}o*~q`=jNWpKk(kwrZDK~U zedN}U7Q@;fY$OJhIL(wjMRuC9WKoc!2Q|FKJ9_Zv*s_bAEK4Er{GeF0&+w z+S}+64qM`_DGWfa7z0~l8&L$ecZ6ITS{|thxxbN&tERW3Ln&0Ff2!^D%&T%@WgZc< zV%17TUoIZSGZgQK7!RFEDjCy-jSoG{=9Ot*|52Bgas7R0L(Xh0a37|b=E!$bIXa=7 zE=7awq{}Fl%ATm!9i+p0z^E%h-1#5*sz|TWU%bf>_a8+CatE6K+`hB+#@D?@6n9aL zOEwIU%lKjJW#+7T~$`?40O>p=A+-7tscO}gCyUfI(gLYO{p3NJP{Z09c zs)v-Qmn*@}f3D)khy03=+)tsbF4)Tq_$YNJavtZ0RteBKkNWe=AFH8aQa4QffVfgY z{Z=A;a<2h~xtmr9c9>M?pGu$`n!oO{9iqCmhMdWQJuzTM8w6viU4S=~N{inIP3Ab( zbTGFhmWDn?S}DUYZ-<9i+CNi|*-siUfvBCXNXt(QcIEBK2GMi@7mI^_6(xFKXLt)Bj}0OWXFbSLP+0nl@lX$4ZL}URu7_zY_et zHWb@Py*BduchID>yz>G!pg_)Wtu(b1A(B$lm!j612mD{(#=8+Xuz3TF#m`bVKRncBN?@XH*ZqD}BR;VoW9By7P}L z@8Aw^LLq2%N2B=?JO(OOKE@BEvI-xHC}xyo37&|L7bX&78gCL3 zti*s4kvulwel|zo$l-~?T6u0S3DAs>GV6;hs#tpL%ANZ8de;>#TGVkppkB9C_27wXO+Q5YRFBYAt+;Cxo`L*hlg+z zMBaYiui9QXJ{;lka46I-7^ISxTPyuy*=fNGRp@AXuoBJp!wM2WN9JTibEa#XGdIW zXE^OGPU`(Fvji^P&SRXy%v&dJ7P$2j4Z9BUYG=`&N$S;!X;!28nB_YoMt)~`GBlQ)_ihVY-qD8VaRTY}<)O{N@V|S|i zu+~~CrVAl064LGM;S;#SX5GVx za=-65NgLJ{!H4nifDG;8&$paKJymDD`xkpm8=s4KR0kYAQ+!X}Mu1QQ(BXMlq){dm zLV+5((J{+g6-R{^2znLnd+GCmxg_AG(scdFU@we1d`zK`0hZ8KgATPUS1Jv z659uU9f?znKeY+ZmrLYlv?*2U^J)yMWcoDaKvr_j4$mq)aqX`7 zMwe2)x|2Q+w){RUbWy*(ELr|;FnhRDU;&$S(ghd~nLg28o`lSe5>#5()8FK8X_@R+ zwrd>Z|Gk;>y{H3HJl}*Pcy>#PBuyoAsYm&82>^ZdO7BT4ISCC&_~o@BIEO&B=`bE>X~ z%ec7(Q4$$AG|)`q1cHZ5ercbZb*5<&XRGtILPNV04P@o^V^1{uRZip6pIICf^G61^ zRDXpBq5KW|)@XIE^JNe(5yj~Yd zuYp&dlP^!U_m}fu{9Ag|m(!t5>Pzk=y}k6vih;^{Xc4_g1gQu8bzZBeJ8#*pwd^$1 zEa`Ro#R%S6k5 z0Cm>LUQ6iOwC)EP6M2(I`-IKPh^8}qX>13*u?qo*=yAuMn;TSAL69}2u~Tc+EQLN4 zxL3g|v$NQ#@y}Al<)ZLyWQUyem=_2xgrSS6c4I$vsQZh64y5Sa%mjvScCfDBdHDBy zVc_hudi!HUL0=0n+1frTJU2SLA6mF>z!(R`oF5GY3u!)QfKE`)!)S|>@<7{Uzi$Bt z>s6O2QtXrmpPF=px+OG{vp;L8wxLyd2fdKXf?K%GX=xFQepuO zMq=Rh_?~WVZNHgQ^xA&2C~aY~;5U280GZCGqFjG$LY}@vv9R2|F5qPcV1!lxaamSl zX~c*4NIZ_+9Irc$+LCiaQ8ET*th52XfY}Q=%RP;UVHnh{ah(_C3=D5J2>W5|a zgH-_wRIYaJ0z?z;KlA=9!vS?M+ z+S0NqR@vKAZ8}MMyyu(P%%ozK_6&WfPPA$hNHF` zIzF7}lb7lI2bF z%$0X=A4%m0KUwr#BN|Pw5JqLh0#%aSqbcGlf!;?J_`pkKRt>FfOut*XBD2ybeB{u> zVNp$2c}`b(%t!Lu?aJ8*5W5(OKwvclN}df<4i(=PLTgC!%KF(snEv#=I7-6}M9Gz? zfe2w%t1c{e*ZF%#H#}LfoFp1!8DNn#()HjuAhP`J`x9&Qo@WWuP3{lh&4+4HRG~x% z<~MC!@xZYpFGMtQupCqE7fXb=SB$X6p@9up&HxH~cwQ~^RZmJt07F8XS6W%t_ZNXs zxt-LdnD;@m*0#BnR516}d~_5X&W4r8OKZuPB;}gF2Svh~0I!7z#R%4It#q2@=P3<2 z%8!JUJ{9`Br15h2L%!kstOR80O8;v8yBb@|VM0LB$y=<-mfmSTX=MvHvTjf(4eLLY z{*?Lu0Es|$zs`5EYSt=#0YQJBC&nC_-uwQ7A2XGdBSd#;57Ck?E)NF#G}l*pbj8o5 zU{)p0DAfriZXbpRxTR8X?Kh^uxkKN1ovl_e0unNKxH};wC5^ z$YP7~a8_@*m*m@VN^daJ6(wiHY~$L1A5JS&VMvxuGmS+)iosYcOOAhwj?UhOK%%5o ztKyP2X=9)p@Z@?;ad$hjVJUaHjVrU*l{s59wNpBt_QjD;(u-|v2ScGc!Yt^bLwFK` zTO8gm`hz_Sc#Se?pxBDJ7B8*GryN4nX>Mdq-RuAnRFKXWQBkkZZGD4UW3l^8t71Pq zVbBNtVeuRiF%`(UKzx7Jj%;k?R*iE>-e@Z8ti?)r4ab?6VP9dIyiJ*{ehiD%vJJQ}pszTNOZM&|krbXN;z5G{a-KEBU0HKQ_=QvixGaNxBEOr6bF#6u6>e2u zOadX2On8{+NLmY_2-~QYP%>(<)?RGZo)lx7wZ=<7vTMKGTzh}3+rje#ZBxuW!ZCV49}-MJ%xXfaY4ymBn}%hCgYD~pMU!OpFV%Ey}8jdFVV^W?rTE8u8IG; zUEkY{#ed!2-Ol5`=J8+iOF;Q0p!^b0ehKK`^b(M|38ZxOZ^(fjH-Msa^l5MrXSuW{ ziHkSlx3?U@8b!CPz8QjdQe~h|0&^K0H=J zhJ_5HyuxCdnB|WOJ)4X(N*M95opk7;Wd<_jCCN1P<-YOdK8RRSNDYM-SqhaxX1F|I zW#A(zW8%eDL^(Yk9J)<_5@Wi7jgBw;q2~_8IhpU4ke%rG&u*z)KCd+nLYv_=E^v(M zD6D_^c&lg#+@KU)`4O*DUMJPtB4M+Mu|37XBB(cV93QYWA`Ah@E4jqLt_H( zeQ07bL_G+P8@*V(+9b3mLo|p*DBCVX4l%yg@}6REr^Oo@-NRI%f*kzFN{Nq&QCP)D zE+mVW;FrubW3MO|>qoU}n?BlT zn{#(nCLY1^S?@xdFgW+-OW2YG|fz*OW@TYUU0@7jU&DpyqX6%!O z@fDH6lxbWktdMmPu@$@NkYL~|At_>DWjQXo%aNJi_nk!$R+OW@Hg>8UID9AwvfiI! zW$%Bxvz-T>>Y!&;DOQ(<4}b;Ntn0Nkr#s)#=?>WS5!+1dVXYhr%hvH?E(2fK951-|-k zz0Cgc582uRUkWmCl~uF}Qu4~2&G_ZEjMe|oINHiT`T1Xd{+Gx9%Ri4h|MO;FnfGzC z^S_Bx)Xb9XsaNTiP|{qpZ#(4bnt)Q^c?TKV}pPJ zzNt0|WLqFLO!Z9fCG)%)hL9NIrkPb~#Deq-A~L>A-`HRnL2+;Y;8j_L09A%xPR=SP zm7_|((jQdX_W(#%@*3>;STS{_oTbrR`nueWx@Y>L!IGwS3D43b)@hjt6BSR+unPRU zQDG{(QPCqTdbCl|{s4c=!oM38rm`CqZDY~nvF0Z6ZiuPEu5dd#p``o0X~ zCkPGw=7S(~FS>IYMOlEv7(3d+oNJ9(E>Hei$5-Vr&<@v}7c$PCD0NN{E?}D6Xk|2d zPLs(?pafz;*-L+-qY%^7z65yzl@I%a6j;Sv=bLtw%W}XB`Q+(w)z9{qd$OON8-m3d_NK2 zPvCp|SXMbMQ*fwB%t9P=GG%clI~#l#N9}^QC9{&rb))Mr0VlvX#a9qQ@~h|0GyqVg8+Y+mLmrcS&o463rZS4y_dj0Ulk7qp!f=LKUM)AIO&dBpDr2CG^_#sN*@u@kFe7W(h?Yl@XWm+q(MVhqC9H91 zFVr4np0D#FlAJ-E@IhhK8K79_Fh&V~9)7sz*y!NGxa34B1lvh%Ej2j}ffQs_z#w;3 zq*R2M*#VJCO1F7;a4M;TDA)u1Nvvk{SMwd$+;BMyHR5~YGX*ZDsx+IV)@-NbACv_v zB1L~Q|Be%}A^_s|bl0(!x|R*rI3DjQ0v7`{xaN(&@e2_MIchRCN> z2w%pl!8VbEd9n_?3Gh*LlusD%f`xY^F&wmQiqP3&NR(Y0-knw_7A)3siVh;s$ksbV zoDEIHZEVf_W6-~vhsXU!7$rgVYFZ__L>Pa~(ibVh8qmvoMLgT1nKPoK*TqmE6@;KV z3%>^6j)HIf;2X%j!ky`%)CEblh8xPD*)P0?$_M4V%I1Q&*>jv3u{~pU$$I@3ur91a zJ3_v2;a&6>pN{;eF|&Z+p>Ic(?kHk>Yr!pHH;(94VUSqAbsQx+`|DpjCvShYF>QY~ zeA$>+u9O|CIp2e8g^Qk#Y&yeiO(>axvRqRJBz(KUkUU}-+O3}Bz!(U#O_dfW6%cfW z8TI=mw8~SXdw_J3cW$}aO5qnNG$)Syn^0Ss^w%cf_9q?<(UtUJ z=<+3uJNQlL|DVe>!a2PO&k$r4s>ZjVuEHe}1pd7fG*oHJi2$S0a4lftP%j?U>M zK^o9a8RX9f)~$ZW)DrkR0&aYTr^=Yo!!#y%Qu_;@^74vUP){^0=}qL19HkgEnW`x& zdI&!Nhh|(lBqt0!HZsQ7G68>2yQQa^P=HRLNT=FT?$f5KwxCe#52noPuBXQT z(m-gfdaYP4uuRl+6Y)NzW0U6U)mKPZdtwxbZK$gFdX+h5rCmnW4=pqIDW;hSK@;_t zqfMI6Y8`5?Zk!KJd>gr#-e#JT_;|%@;L{C^Px9-1>REUXD3xeG53H;T>~bt-saUWNrMl1s9!u-wjQ*{uzjh`kr|rXcU=Jj}_u$En zC4K{%qn;EJ`!}ajoXDWX|DiE94w6&0v3QRlo*j(bfxYe6q9s^yFPa5#)&ty*`0+{V~nlC4WT z8r>y~H;+{1rs=;p zmL1)FooiKDniap?7lTk;<1#-)!h@F;s9dD4COr1SwIt55RyjhVX?+7tNDO$D&p}qj z9TYC8Bq$n>WhYPaDOSt|2Gs{{`U&ZWSV>nGRoRnoQz(C%jQ_Bm$A9=H@gH6V@gKHd zZ|}T*y}PaN9qb&`wqNB@ApWD{Km6Tsu1#}6@ea4Xdi@{l?PbJ&;A_6Vdr;fmt?%vE z4*meEzjjdj1KZ2vKjiU$a{m8~;{W9NKXU$`$^UV4p4rR!$aF>I8QUE5&@-_8 zq`)=YKg)o9rd{RXZ0H&;x9}I{#%-J0Y}>cC*Q^zc$=tGa)5aEg+RWyZ0xIU~HJAUn z{Lkfo{`t1@|LdXvu95%yJKOR6U-jMH?OgtUi%%Z^FOUD1$N$UY|K;)jfBpsXe}zE< zWzauWNN0rp3yLtFFo9zCJu>261^Lqu9p|1l0N=)8^mueW_1qaVw^%N}FNYk-VS%lp z$~#=8GdkV70#@b1&eQ?^T5=ZrD6pJ*nE2M1&B7aUGzy|2rTDMcLmCcf8UKLzxw*2& zS1Q2+Pp%Or{#e&)D@d)yf2OW!@)R>N^4j>y@l2cZ;~PsR7r+nRxc5wtX(9!R#!8f) z1grDY?6kO!$)JDgTzW|8zsyATU!C$O$Z^z+xl9PshpCTJa1izkLPL%I2G2^t%hZ~Z zj+D^jMahNtyX+s#e;%XjN0`GSSk$`A6vwm`&gYtv2}CPr#)QI>(yu0*LUP@db#F1V z;6XzxmnmspZJk>M3aUeTBMpUBXFv}Y!!U#(ud#?&v%GH-e(AAn&+e-2N(hr_?3pO7{APQLokdi$X@JZ%l$ zQ?wfHf17qqb6{bNuZ>HSxtQT0a7qi>==X+ay(1BLBcw|BzJD|b==(w~t#NVMXBgVf*Cm zard40@!_yNBsOr=Az*y6Fn*D1!pEm?a_=Zgf31rd3YTMl=?3|ABc72_*U7eYK?~f- zfUUk8OwcQkZO+q@ahtf`5{0c^JxD*<}vwdt!x>f1Fn^X-Qo@J&`aRvdE6I;gx9 zqu*n#5A3@p5L1N;6#x!TGonE1xB5rCFI&`=7zHpdGUbrK+`{;yiC=i8X*KL3e@-0W zdtq(K>(bifr6Y?lUeQgfcW925m(t5yiRB7U(~7rereUik$NT}5MUke^9K~E3gjNuw zN@xPY&0Fym72c>Kr@2x$m|E~2V3X^jyt}K8A$L=$I$6vm@ zzQL5cG1oqk4+g~$`DIFNNSYQRe{O~-AtFnWZ+a+CF?x%eXqwPd9GF#*2?u7yiBqg^ zj%m_87hXxO$DJEhWFnriR65g-HS}2)*(G@WXoGUuC!)eh#-S#iyh~|7lpEU1IfjL^ zY`=8Ny}oa0o2`_QZkf7PzBq5}TqqeMF+DN+!2q7rab3q<-_Gj_!~b2xf4W0!IN2G# zw4;f&`IHbMz8$+4p!yU8I4K{JXeHBkN5g|T9%_P?3%w^y6txdCr~C6ol{@lgB?FIw zqlMfYInFGe{fRH}3SXe*ST(V#Q!C09U!fJB%=|j!_wl7vLUCA%hm8YWH+U0$6_pVw z!3U!K@T60daS6Twy8wNji?FcUn7X zoekc<>A+5|7j=K8><}rHV9g>+DPv0SNbO{kJjHi!X`>vZ2)eOZif80<&mr-GeEo^1O5x-ic zD2$XJ%{4)#;FkH})Ic)^=#g{FF6~s5X4cFn4RnejsCGU#SObLJH5I~Lm8zSpsT7^# zL1MG$<4s`54M&I*(?XKg3HITz zO`KmCY)bM)e;JpwoJ3KHkSb_FR&o$p$_g8LroUVauLbN<${JQEaL)Iq;RQMIj-D0@ z+12m_1jX8r@`uwMzA&A5QUMLjrMdp2!3L~*>aA%=M zw-`zf61q=9{r(R0@9hdK5>~rNsw2^#g;(By)PDxVQm3q#+B)LX?6uk-o_`?@c%LEbBwx`b#yaWc z&2()^bKFbaAWMu+^Ik`L5NP}oN)MWC!EVp&@dteYRa0bC$QWUSqQ<_M`Z7w6k~&4s zMPI3de>na4?^Lx5#HWvWNftuI=I3`|2=Li#D5o<{y^>Z@FWpH~H!S!e6pikVG-E=M z-7(a500Xx?@c-D|%yMvHuM)oy8z4B82*Przq(;=f4L6-pykWB4>a?yrR2fU@x>#8a zsSRU|L9S<}=xG9tGA@E}p;g$=(IhTk?%7`%fBa+KE?}fFS4N!DMH%rsF{M_t8d=;a zyV|*=NXzO%lBy+BJ9uS8b6E(*|6vG%1d{a zx}l$0BX@Ewj=C(m^Pj4Os3Jkx;YQUMaV2!T!aAUf3#H|xcuTED6R}iQEm#wCCJb1K ze;cXX?(Db?oZMq;gvnQYXA#j&BZes6gtI_1?${LPQej)Xzd+cu=K3b6mJ6iUXVV6X zj7^w6pS@%QG%(@$l5&ZB%`I$cs_=Ze{GJi|eT}*vq7WXKg(&=b9lH#ig1R_FQ)GL5 zrDgW_2dLLQIS#AFjqYIu)zp)q5GF`Ae}KZIB9#(m4-qrnw@6XqoR z9+~Xgn!)mBcwPMS#F|ik9zCKQi~=)Ue003n@BnkB*=bI+au&&z1IHH>hF}y58lYfl zgm&Mt4o{NPPF-f%({IfJPR>RZ?zt8Wr7nhY<57;m5mJ;-BYkSjt=Sz{=RWUhe@RU= zvA0LVO}Hse(?=uiNdUS8ps>6+>_U7i^#CjGh)ikCSWU$=8xa*)_`G&Ocj&pV6g{b5 zfOxCd)^mV5mD*z)>fbqegz?@n{5b#+;r#9)lKYjbKeuyGw zG-Z4uU<#@FbIMMICWnTuWO;yqwse16T!WKZ2k&!%*^YVV@$f3kP6v> z=2Tw4p4CUq!?7qrpieb}axxSWo=4#E7`C95*9r%$Xy{}hMTvV6fhdzjZ3G3k&E#)v zM!^A^(WFJ2K!ZwYP}D3*n}Uh(SPmw5Qiz+aQHjOP1F<0VE=125e~AcrJmVflrU;(! z)1N|IFkM&J=Rh>mxs#_-F1kXP%vj7WbjtZyMm54LGTL4~#L zMeCuJVr+Or8UuB=we431YmF;?GR9DKtj*;%#P#8X7&_w zyWK6!(r2>7qUkZ6f5{NQZGgCqAU|gV4Exd1FLhQVM?jd2unBgXxEI>;ovm;S5Trvc zu1;_n5cSJ_WE3iZ6(5Q1hq6osWcsKoJDb3!oV~4y+PZ`IhY{Nay+Tsb_Kz&hx>D*b zsde1FRfgpw4|wh$!nWj&FS)vCgd^XXWCN$aL-i?am^Bni%w&GYsFNXn%iCA;?x?k zK(#e^PmdubWp5+U4dA`Agba{-%}HIgHmH zr>^N;F}}#ze$9iWJ4Jy_{DsyU=;Qk47+^vitfg}}x@*pyOUq8#&&`e7QoXVHYd~tuBh8O=gyU5-&TMSB}zfRV?;v8DcW_9;?a8CQgL%-(&4Xi zlTB_k1B$q&1Q#KQh--^yuj)y11VubDe}T@V%2_&L_Ls2dA={y7Lc5@!!HKR4N$Fo= zoF(+Nf3^0; zY@}&B_^qQ4>3w>D9^9m+s(iM_oTuS9JW3+$ir!g}Gh>GkIYuDgKWx-Vq*lVAvZ!_m z;@a?Ft-Jhjeh}vy^fsj1IC~XrPfsG-YGZDLO!-o0UO0594jRY|MTIuNYxYHfYD-Iht?;m#D#tA&;oDaukauah_ zJc}#|1_+hM;|0c%sLwdXRZMseioaKIP>VlwKTZ|{06}8TO_(Pj_!1>FE_Jb%b2IeQ zgj(Igi9aSKg$3|7anToxF+{fcfo??rE<20tvj-9b*?^xphf24>!BZ7re`uk#XJkhP z1_L^NGmmLrtTijLcJ&Dneq3cvN-t@w0T=uNbYatF*qbaN+KNlG>5=P{fYbn;h0&^& zoV#ZS?|wn~r4pk7H@c2{Qxcg(cQW}kPnP?bz*GVLx_r0e2@c+L0^qVSt1Muimq|M0 z^I|YuDpf!qw8L~(y|0h&f8A^9vT$~AdWPaP4wjX*z4d+DSnwr_H_Th|tBT5!Ji#)Loob0dJufBE>o97N6k;J!WG z8m!$v{0I4W?So6i;Y2h{`qgwYr_B8SW(yaiO>fct&0tT9Q4g6(7j3zY3mnCDY!=lu z2x2h(i$I7(CdBiR++~ZLvZvrfx@L$2af!TYpv2$?WAF}SGt&j)9FYIJw2FZqXW0CS zkpTnq9Jq)X1Cyz)f6%H;SH2MPw{HF@V<#N1uaYKv)PIT*>S_b8&QWvvMwtk_rWFpIX}e^H;LiU7@;jB>m^P)tU2A`Smj@; zBwXQbEE5ZE9a3K#j;7hjwnbA0ZS9jM*|ra4%yR!AV^;f1e*u8zL}sV0uHCNr;72j( zHy<9U-L5|XVf@b6z zfqps(cq`JlMAUMI!&yi+tD)PcZxCbfLG0u4_`$3E5^;Ur;vFxFi*cTyqA+mJXy~OH zHOtIHv>vp$e}t?8&+YYih+{aW6%bC9$HS7WE^-l^3I{w2vX-5&qY|z}lNB}}xCbds zElm4^Ra_EY(A*gsQp(%qzieGw+8HerSq=6*XZ2px*Fi@>19xU9B&L~+LQ(_X|1yA> z_wC@j=5c}9M+}SofNf(yh~l$`Vnli_9TU1ONsk;)e*+UZ{oAWvnx%9sT zPac2C=+(jF_osfNB5HBz1TeCMh`i=L=d#QB7wd7pdWSU^@#elW)ldvT3H@SZN4CSy z>b+9q(g7TcB;qXP8tYx-5V(saAAb_Fr(h)30 zB|*aKe{st4t$u4YV@44>UEZWa7$^11v?iJ5z8@YbOL)` zJ>QbLOo4sQvpew3ZhWV+NPTtuz2a5gZOo*&^j*wm2;7%3p&LLGNngSUmPf_93>1>sbAABUl0k#W%E{U*;e=_qT84w^qHtrY7NTcjF1<$=0To=eI zBr;9&vtqv$oPh`yi}X5Zo+1WZKxc>ksBPn^89X+(DtA|q6DT;tfB$;MVw#pvCN*y8 ze4bu=1PQqAvVPh^kGYBnfGco-Mtg|=KpzWkK@xGr954rN2KYm7R)7Uz0sLVUkWo)q zf5aLhtOGP*IeEn>S^-UuuFeFd7{KZ`n zfGxqxA-1^1e{b>Me`ombZT`O&|J}rYf0`VC%kuy2?d%%)|F(8t?zQ;uzeoJ{HveCX z|K8%ixA^ZZ{`+$L_cs4iEC2t2@;_nzzn$-1wDSL-UH)hI@t>XednNgQkI4Gle?0%+ z?%vKe+qi)ZsGS!VR! zVv$7r4t(FGmq%d{zt|Nw@@WxH;^FAUZj4@u4e49ho#~XNcaR0}To1eo>yx(>S7dch z`7aO``SP?U4)=4iNo2G4tV5)n5zvUO461bMAxV3FFuE(_ z!Y31kSKUQ9>3s)4S7c?(b@(c*j~}rytUEaB!i)xUm^bG>e=^{SIZe{V$88(RoGivlxyz8ql_%78j7V1uh8y61UrG8P zmH-W|fNvCXfd2?b1x7xIf<9e6ezlP#8Ui{3scEZIMMaM)Fbj?jCydpU-k-|cxXjRe+$vb-U5dv!e!bw zeayjS8gD}f;Xzp;;x{4^1LHg7E6!fOW2*pZ2_sv0K^}W0Y82284H1?hwxfAUBN@ZR zL~W{mD2%Gf-wW9``iabW5G?tovOP{|@Fz|q4{jS1p*@QnyGytZ9`D;6#OZJ#Mk0h% zWPC*T5fu z-qn#27|;RBmUQR12!Kar@2l{JirrWth}ABBa|3lq+UuZ$k`veO*x-ElATc_XjMEDI zLkd|E(m>1tf7}i^0hta7qlv3Xd*Jzz6nxI17xJZ9T;)YYf94pJnXmVSXCn~d=j@^L z%?n2UFcN-o>*5h(VI&RdXv_sCQ-!GXSyTu#Nrw!sJ&Bk2izgU9KzG1KnYy>k$ePjYtp*(1R`BfBX^Tn#bTx$e_C-pmH2-A z>#HGQ6qSMYOU0xB6Bq=^A~uCC)reV?+J;Rg_~iappdtn~(1?7I5}~gshvQjf17;K7 z-z`8KxFceJ9kxdk6cI3AVvNHdH{3Z&W}HlRp5qhEgR;G@E@}JS6N7pjKBk9+1#oHRgRi z(k6pR|90-@nLVgQ>>{g$A+LeBkcKJ}RKcs!Lx_+} zG*Ey?M*A(F;;s<4>48sRi}ybNh-#P8tiQ@i2HipGA0M?|`WGr_i<&^A!EOO;OsM&l zU~kr0%}S-)u)4bKPuXZ0db%B!#QgTB(L~lTHUn+OW1Pbv`FdG;qmY0c(I6 zW-FA9kR=*nM_|vVN!IGX3fRFSL<6Dmy+9iy7<3NbGJ-}uGfdJrcNtVQXizsve|)Q( zdrng4TQqQE#aCoMxx;z`+6D~^fo%~kQZpOTfR4cCs?|j*A+)@h#cBsdKR|9R z=;>y!m&PSP#}A-Jc8i!=A$sbO;XotvvX~uQ_aWL z8mW=lUZIi*uT!T(t-ViKQaBLAe_72M*oD?60uxX(h_nGOJ*wfzyXFatsAAa}i_8tp zb#&1bg82%AkI7kSy3RobS&rGc?FfB}9sId!2$xomBofn5v53o2DHEdivWo;Wg!4Vy zR&c~t6Cs2&p29vrft5pi%t@*3VT8I(il!;&Z2v7gbJInI5;UXnG%N7zf3aks&^Qce zL8(t)fw+zUqaj904*}(gJqR_4%%)gDcuCe$mx9uQc@`BleG!sn6$Ezui;%ULjd~j+ zHZ+hWn|LQJy4lfGyhP$yZ*>9w(!}1gC4{XcVdo|3TnF?YkNtY}W_WmfaMtB_PfzxL z9-f}PJ9za=RRk)65u}*$e;`92^w=Vvv4P-e0#pfH&hyW=E-aP7JB|%WizRv&i`|}9 zWrcJq*4kHpciMQg6}8m(D_E8_%@1F&;=~DMAo#|O&W(Ml?~k=F_9oBaFPt2z`O?+B zl-#aozuwwDS~W0AQw30WJjo)~3*V_nKm#^WJ_AzOOhs}?SbE26wu7WULb)nGo#b?fe z73L8S|6&D8!3Ly_{u;s;#sjsgij_SKqsViQ_J^;IjzD|dchw{fM{rzp?}8!J?K?q0 z3b4rRAb6msk?3kj^V>uyR2N-#K45VL!Ta)z-|Ax`vtX*ee1MzwcQ{}Ya zIHSks5(^)V3wS_4SAay2W5M@SYDVTw%uR%Z#29!72*sU zzfrff2lEZ`e++(emKaQO09!ec08XrWYboZ+pp+z6XmNE~1P%i|*;JC;)r?y~+yRC- z3lW!01)zMlNHJFdD85vM^>TKJuPi`(EQ}R1to4eqfTWs1{VM|lh)J{M#+5ye5;L1E z&3Q$x6Zy}4f%Lp~0Kh1Tk~vM<#;|@6!`BO8YvDw!fA(f|hnt@8xiLn*k_V2YurKe8 z0|K7i!e;k8!No-#I$%k$BVBw}{-m|=!?a1ZNTa}Fw$cIiRa2XC4#ugEg5c`v#yn#{ysXn7U_1bkCCM{gpl@ve%ME8h zykEd{Mqe9$OUBD-Hn1t&puxJSE?x!Qm`$zu2>r)54~qG`#DLpWU^fjt7_v`<*D>h~ zyWQM(FhLA~0XG|FjaU6CV^J!p&MRm-Ol0LleF14zk$hFPQKE_u(ie`C+drog+VP&iB^TM4XhsKvk~&!rKU0LE8g zPU5)+o#ya_=45d9(OR%x$YHM+QiMSdYO^iFvmNNh6u;64y>bR3NX6rw60ZAfJ^KMx zkdzCy@B#|pRsS-T+UqLq=CN0cqf;Ur=^ANd{UIsaYb(Z}<&FuN0X zS73QnM0gpV;Z7CK&6F?ErqW#Bi4zP(MPE^mMm~^wDX=IJ;6rB;t*E5~e@*b{JFi6l zkyLrnJbj379UC>&N{ffE-agk797x?<$!$Oh1knmDxQ~ zd1G&0J9wMCF5FsKY3s~aJj<+;E+(~DGS6MI1Zd4&(X92T@9)4|P&imNyUOGi=Hz#C&GNysLeGAvG(oigwU`Q9^r;&e|C6&4`d45v(n5s zs^lHvHjRn3CzSOp?+Gnu?+Ly4sGd~yNp~ZPZ#Gi0`@^Y2uEs@{QIgg+&{R1vVLge; zTWpK6eVH1D6pWF{P0|68l?iPW$qJknaN#;uPaksj@Ep*Y0G9Fac#1_Z4UtMvN!XBK8QE}I~ zg>YVa1cN5Z2`Y7{a>;sho1M5i5C&u>Q|RXmiT*Xy& zUHI)AtAy7kmWj1Ca(DIGIAQa%Rg`G>NLLUMq{yQWb+t#Ce{`get75yrbIZ>Ul_*S4 z+)oRRAa8%Apa`Ut)OvcjibTOT&dC}X2YL<7#VIA170DJus(VE#mVi4p*fgC0|wT^tq_s0~y7Uh>J$9O(()G2q?3*kF@ZX{(7uz zKxkEI(;_obe~Uz1NeM{$+#?S^<_ubo;jeN1BszX{=ZL_Bb?c!W zZvt4nJt-?YljHSNi>N2(O2~yA+dPTtMutqRi7Rhfl{P$rs8;A6S)4uZtf4_@1SOcgPK$Z?8^s+G`vs2kl z(aD5VMN@Z97E`r>E-LACo3r!2&y1}>-O^LuPd1%;K7YH8Vv}6ua2cymGSRVW3>sD9 z6gGoBRgU3XBh~}d;aRlNHH=)ne-Mm@eD7+sq$k5`NGHijvg9$L?Z|}!=))>$!nN!_MQ}}8nP0+ALDbLrC0o5>gm4Cq_&&ku z;ZHvw9v#&%aofz>ZACg0=8J;|w~AG0i)*!3qcW{&>~!!K(txXaS0I(wHOf6o z$Ssl<42UTmcD|(Z{rcEr7T{{p*+pImZ;lCze|27>- zTDR}Y*UcL7F^C(sdcJc-5oHpr~H4slP+(T7pHCu``6?V6(j)u}9%^F&=%x8G(; ze}#@O!2%Ii7EW*@(6)Ci`rWoehy#v>M6``8o%JZU`GCf9GbpZs zapBwhjH;LL9#!_iQkoxv7KU5(yy32=Z?hu7R9ewuHoNPya0QTDlG%J3gGH9`wb32O z(0HEl6al}AxKz#T{lx-%RobkC(Dpkke}{tddbCj9{?mqjs-c#_=CWYRdaMx+GNR0Q zMv=b)L_!2mNW?xocrpz?5>m4}iF*fA@s5?vbQP4}~a8>oxg4DdZ6&J$crkSpMs zoA7L;cy6}C_s1YUD4v5*8ur7*9HMIJais4k*bZcyu}0L=aAe^fhVis4oL0g?fB3q| zO=2qGGu8;l`8oA3yRbpyEcaGw4%U!kULdLfh3>(vg^ttVj#}r`sqKP%ttJH)1FZ0H z@f?g)6kD;2Rx(61n$m3eIXIW;qDCfMXifR-m8b5?_s%t)C` zF&AIJqf`}FJn7*W`@W7Gf;_Bde_xfN=J(E)RX0}IV9(IS8(k6PW+EG@RG@@kkiN6% zi|X#;-me!wkTI_pAgMh_aYb>F6+Y7|*@{%JTDjvbNeoR_7TL3Y$N2v{yH))E?H2$4 ztKk2Cr{n+s=k9i}_j2du=Jt1O7Jxq~{(lf(UiOl7l5ITQ`hU5%SC{__f7d+2|KEGD z`4aH|G5^P2i~rx|e`)#uSIz$d{J+ir((-@G|KI-nQ}6Kj&B>R{|3dbL-Cc$M6JD^r zxwEy~=70GbpI_oK1i$9q-)Yok-`NhfI>+F@?@(&!h&z->I;V@-EF_w$b4KnbAwVj= zp97Q+^Crk9aEy1Ehmg&se^`vKKvxVIWIT@t|64YTd-L#;G;>v!^P<18aha4?i%~Gn zW*ad}EyD=1^J-CEWqH4nd4^u^#YEeg_MI8t{eeFm6mjP$8OJHn^PKm`KOdj`dfa&{ z0B!gcbeT^kNlfa0OZzs7&&LhijYvoyYf8fT+=^`zjy4Tm16s`0&la=~KKmKjztWoRbY( z83Vl==c72!il3vg06SJVWYAQ6%E_Z(mSaysA=8os7YIq1U0Ore|(xz_XIvUeZmX!+UqrHaG z>hx^em@?ZP;x0U~F+v?NXf5=Zm{@dApvDwOhU*pp* z^*{Hg-T&MDzuo`a{deEK|En7F7qkCtZthg Date: Wed, 23 Oct 2024 18:30:54 +0200 Subject: [PATCH 04/19] Address comments --- src/rdkafka_mock.h | 1 + tests/0016-client_swname.c | 5 ----- tests/0120-asymmetric_subscription.c | 1 + tests/0147-consumer_group_consumer_mock.c | 20 +++++++++++++++----- 4 files changed, 17 insertions(+), 10 deletions(-) diff --git a/src/rdkafka_mock.h b/src/rdkafka_mock.h index 1b8f3a26b8..45a823708f 100644 --- a/src/rdkafka_mock.h +++ b/src/rdkafka_mock.h @@ -499,6 +499,7 @@ typedef struct rd_kafka_mock_cgrp_consumer_target_assignment_s * @param member_cnt Number of members. * @param assignment Array of (rd_kafka_topic_partition_list_t *) of size \p * member_cnt. + * @param member_cnt Number of members. */ RD_EXPORT rd_kafka_mock_cgrp_consumer_target_assignment_t * rd_kafka_mock_cgrp_consumer_target_assignment_new( diff --git a/tests/0016-client_swname.c b/tests/0016-client_swname.c index 170e213ed0..335925e328 100644 --- a/tests/0016-client_swname.c +++ b/tests/0016-client_swname.c @@ -120,11 +120,6 @@ int main_0016_client_swname(int argc, char **argv) { const char *jmx_port; const char *reason = NULL; - if (test_broker_version > TEST_BRKVER(3, 6, 0, 0)) { - TEST_SKIP("FIXME: check this case with AK trunk\n"); - return 0; - } - /* If available, use the Kafka JmxTool to query software name * in broker JMX metrics */ if (!(broker = test_getenv("BROKER_ADDRESS_2", NULL))) diff --git a/tests/0120-asymmetric_subscription.c b/tests/0120-asymmetric_subscription.c index 4439d5e51d..ca32502796 100644 --- a/tests/0120-asymmetric_subscription.c +++ b/tests/0120-asymmetric_subscription.c @@ -175,6 +175,7 @@ int main_0120_asymmetric_subscription(int argc, char **argv) { * given we're not testing the assignment here, * as it's mocked. */ if (test_consumer_group_protocol_classic()) { + do_test_asymmetric("range", bootstraps); do_test_asymmetric("roundrobin", bootstraps); do_test_asymmetric("cooperative-sticky", bootstraps); } diff --git a/tests/0147-consumer_group_consumer_mock.c b/tests/0147-consumer_group_consumer_mock.c index 7af526646b..a36d246ed9 100644 --- a/tests/0147-consumer_group_consumer_mock.c +++ b/tests/0147-consumer_group_consumer_mock.c @@ -155,7 +155,7 @@ do_test_consumer_group_heartbeat_fatal_error(rd_kafka_resp_err_t err, SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_set_default_heartbeat_interval(mcluster, 1000); + rd_kafka_mock_group_consumer_heartbeat_interval_ms(mcluster, 1000); rd_kafka_mock_topic_create(mcluster, topic, 1, 1); TIMING_START(&timing, "consumer_group_heartbeat_fatal_error"); @@ -299,7 +299,7 @@ do_test_consumer_group_heartbeat_retriable_error(rd_kafka_resp_err_t err, mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_set_default_heartbeat_interval(mcluster, 1000); + rd_kafka_mock_group_consumer_heartbeat_interval_ms(mcluster, 1000); rd_kafka_mock_topic_create(mcluster, topic, 1, 1); c = create_consumer(bootstraps, topic, rd_true); @@ -431,7 +431,7 @@ do_test_consumer_group_heartbeat_fenced_error(rd_kafka_resp_err_t err, SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_set_default_heartbeat_interval(mcluster, 1000); + rd_kafka_mock_group_consumer_heartbeat_interval_ms(mcluster, 1000); rd_kafka_mock_topic_create(mcluster, topic, 1, 1); if (variation == 1) { @@ -483,6 +483,11 @@ do_test_consumer_group_heartbeat_fenced_error(rd_kafka_resp_err_t err, rkmessage = rd_kafka_consumer_poll(c, 100); TEST_ASSERT(!rkmessage, "No message should be returned"); + TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, + "Expected %d rebalance events after assign " + "callback, got %d", + expected_rebalance_cnt, rebalance_cnt); + TEST_SAY("Awaiting partition lost callback\n"); /* Second HB acks and loses partitions */ expected_rebalance_cnt++; @@ -492,6 +497,11 @@ do_test_consumer_group_heartbeat_fenced_error(rd_kafka_resp_err_t err, rkmessage = rd_kafka_consumer_poll(c, 750); TEST_ASSERT(!rkmessage, "No message should be returned"); + TEST_ASSERT( + rebalance_cnt == expected_rebalance_cnt, + "Expected %d rebalance events after lost callback, got %d", + expected_rebalance_cnt, rebalance_cnt); + /* Third HB assigns again */ } @@ -509,7 +519,7 @@ do_test_consumer_group_heartbeat_fenced_error(rd_kafka_resp_err_t err, TEST_ASSERT(!rkmessage, "No message should be returned"); TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, - "Expected %d rebalance events, got %d", + "Expected %d total rebalance events, got %d", expected_rebalance_cnt, rebalance_cnt); if (variation == 0) { @@ -597,7 +607,7 @@ static void do_test_metadata_unknown_topic_id_error(int variation) { SUB_TEST_QUICK("variation: %d", variation); mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_set_default_heartbeat_interval(mcluster, 500); + rd_kafka_mock_group_consumer_heartbeat_interval_ms(mcluster, 500); rd_kafka_mock_topic_create(mcluster, topic, 1, 1); if (variation == 1) { rd_kafka_mock_topic_create(mcluster, topic2, 1, 1); From 5719d94c361776df3c3408b1a0874b48f65dc3a2 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 21 Nov 2024 15:50:07 +0100 Subject: [PATCH 05/19] Static group membership mock tests Address comment about using rd_kafka_buf_read_topic_partitions Asserts on non-nullable fields --- tests/0147-consumer_group_consumer_mock.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/0147-consumer_group_consumer_mock.c b/tests/0147-consumer_group_consumer_mock.c index a36d246ed9..c1fe4c075d 100644 --- a/tests/0147-consumer_group_consumer_mock.c +++ b/tests/0147-consumer_group_consumer_mock.c @@ -116,7 +116,7 @@ static int wait_all_heartbeats_done(rd_kafka_mock_cluster_t *mcluster, } static rd_kafka_t *create_consumer(const char *bootstraps, - const char *topic, + const char *group_id, rd_bool_t with_rebalance_cb) { rd_kafka_conf_t *conf; test_conf_init(&conf, NULL, 0); @@ -124,7 +124,7 @@ static rd_kafka_t *create_consumer(const char *bootstraps, test_conf_set(conf, "group.protocol", "consumer"); test_conf_set(conf, "auto.offset.reset", "earliest"); return test_create_consumer( - topic, with_rebalance_cb ? rebalance_cb : NULL, conf, NULL); + group_id, with_rebalance_cb ? rebalance_cb : NULL, conf, NULL); } /** From 37be85a30b5ac9750a04141ade3cf7ac02c3eb74 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 28 Nov 2024 16:59:45 +0100 Subject: [PATCH 06/19] Rename mock configuration setters --- tests/0147-consumer_group_consumer_mock.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/0147-consumer_group_consumer_mock.c b/tests/0147-consumer_group_consumer_mock.c index c1fe4c075d..2b0d8e58f9 100644 --- a/tests/0147-consumer_group_consumer_mock.c +++ b/tests/0147-consumer_group_consumer_mock.c @@ -155,7 +155,7 @@ do_test_consumer_group_heartbeat_fatal_error(rd_kafka_resp_err_t err, SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_group_consumer_heartbeat_interval_ms(mcluster, 1000); + rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster, 1000); rd_kafka_mock_topic_create(mcluster, topic, 1, 1); TIMING_START(&timing, "consumer_group_heartbeat_fatal_error"); @@ -299,7 +299,7 @@ do_test_consumer_group_heartbeat_retriable_error(rd_kafka_resp_err_t err, mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_group_consumer_heartbeat_interval_ms(mcluster, 1000); + rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster, 1000); rd_kafka_mock_topic_create(mcluster, topic, 1, 1); c = create_consumer(bootstraps, topic, rd_true); @@ -431,7 +431,7 @@ do_test_consumer_group_heartbeat_fenced_error(rd_kafka_resp_err_t err, SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_group_consumer_heartbeat_interval_ms(mcluster, 1000); + rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster, 1000); rd_kafka_mock_topic_create(mcluster, topic, 1, 1); if (variation == 1) { @@ -607,7 +607,7 @@ static void do_test_metadata_unknown_topic_id_error(int variation) { SUB_TEST_QUICK("variation: %d", variation); mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_group_consumer_heartbeat_interval_ms(mcluster, 500); + rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster, 500); rd_kafka_mock_topic_create(mcluster, topic, 1, 1); if (variation == 1) { rd_kafka_mock_topic_create(mcluster, topic2, 1, 1); From 75cb26f812578c7a00961f8a3ee0f925a2ec20dd Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 28 Nov 2024 17:14:09 +0100 Subject: [PATCH 07/19] Change parameters order in rd_kafka_mock_cgrp_consumer_target_assignment_new --- src/rdkafka_mock.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/rdkafka_mock.h b/src/rdkafka_mock.h index 45a823708f..1b8f3a26b8 100644 --- a/src/rdkafka_mock.h +++ b/src/rdkafka_mock.h @@ -499,7 +499,6 @@ typedef struct rd_kafka_mock_cgrp_consumer_target_assignment_s * @param member_cnt Number of members. * @param assignment Array of (rd_kafka_topic_partition_list_t *) of size \p * member_cnt. - * @param member_cnt Number of members. */ RD_EXPORT rd_kafka_mock_cgrp_consumer_target_assignment_t * rd_kafka_mock_cgrp_consumer_target_assignment_new( From b4cd7ce968d76d59c55d5535aae7652a85c0f8c7 Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Thu, 28 Nov 2024 17:24:29 +0100 Subject: [PATCH 08/19] Test the range assignor with 848 mock cluster too --- tests/0120-asymmetric_subscription.c | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/0120-asymmetric_subscription.c b/tests/0120-asymmetric_subscription.c index ca32502796..4439d5e51d 100644 --- a/tests/0120-asymmetric_subscription.c +++ b/tests/0120-asymmetric_subscription.c @@ -175,7 +175,6 @@ int main_0120_asymmetric_subscription(int argc, char **argv) { * given we're not testing the assignment here, * as it's mocked. */ if (test_consumer_group_protocol_classic()) { - do_test_asymmetric("range", bootstraps); do_test_asymmetric("roundrobin", bootstraps); do_test_asymmetric("cooperative-sticky", bootstraps); } From 3e406dabb33b28766304091a8c794f23443ae41f Mon Sep 17 00:00:00 2001 From: Emanuele Sabellico Date: Fri, 29 Nov 2024 17:52:38 +0100 Subject: [PATCH 09/19] Move test 0147 to a different PR --- tests/0147-consumer_group_consumer_mock.c | 686 ---------------------- tests/CMakeLists.txt | 1 - tests/test.c | 2 - win32/tests/tests.vcxproj | 1 - 4 files changed, 690 deletions(-) delete mode 100644 tests/0147-consumer_group_consumer_mock.c diff --git a/tests/0147-consumer_group_consumer_mock.c b/tests/0147-consumer_group_consumer_mock.c deleted file mode 100644 index 2b0d8e58f9..0000000000 --- a/tests/0147-consumer_group_consumer_mock.c +++ /dev/null @@ -1,686 +0,0 @@ -/* - * librdkafka - Apache Kafka C library - * - * Copyright (c) 2024, Confluent Inc. - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * 1. Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * 2. Redistributions in binary form must reproduce the above copyright notice, - * this list of conditions and the following disclaimer in the documentation - * and/or other materials provided with the distribution. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ - -#include "test.h" - -#include "../src/rdkafka_proto.h" - -#include - - -/** - * @name Mock tests specific of the KIP-848 group consumer protocol - */ - - -static int allowed_error; -static int rebalance_cnt; -static rd_kafka_resp_err_t rebalance_exp_event; -static rd_bool_t rebalance_exp_lost = rd_false; - -/** - * @brief Decide what error_cb's will cause the test to fail. - */ -static int -error_is_fatal_cb(rd_kafka_t *rk, rd_kafka_resp_err_t err, const char *reason) { - if (err == allowed_error || - /* If transport errors are allowed then it is likely - * that we'll also see ALL_BROKERS_DOWN. */ - (allowed_error == RD_KAFKA_RESP_ERR__TRANSPORT && - err == RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN)) { - TEST_SAY("Ignoring allowed error: %s: %s\n", - rd_kafka_err2name(err), reason); - return 0; - } - return 1; -} - -/** - * @brief Rebalance callback saving number of calls and verifying expected - * event. - */ -static void rebalance_cb(rd_kafka_t *rk, - rd_kafka_resp_err_t err, - rd_kafka_topic_partition_list_t *parts, - void *opaque) { - - rebalance_cnt++; - TEST_SAY("Rebalance #%d: %s: %d partition(s)\n", rebalance_cnt, - rd_kafka_err2name(err), parts->cnt); - - TEST_ASSERT( - err == rebalance_exp_event, "Expected rebalance event %s, not %s", - rd_kafka_err2name(rebalance_exp_event), rd_kafka_err2name(err)); - - if (rebalance_exp_lost) { - TEST_ASSERT(rd_kafka_assignment_lost(rk), - "Expected partitions lost"); - TEST_SAY("Partitions were lost\n"); - } - - if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS) { - test_consumer_assign("assign", rk, parts); - } else { - test_consumer_unassign("unassign", rk); - } - - /* Make sure only one rebalance callback is served per poll() - * so that expect_rebalance() returns to the test logic on each - * rebalance. */ - rd_kafka_yield(rk); -} - -static rd_bool_t is_heartbeat_request(rd_kafka_mock_request_t *request, - void *opaque) { - return rd_kafka_mock_request_api_key(request) == - RD_KAFKAP_ConsumerGroupHeartbeat; -} - -/** - * @brief Wait at least \p num heartbeats - * have been received by the mock cluster - * plus \p confidence_interval has passed - * - * @return Number of heartbeats received. - */ -static int wait_all_heartbeats_done(rd_kafka_mock_cluster_t *mcluster, - int num, - int confidence_interval) { - return test_mock_wait_matching_requests( - mcluster, num, confidence_interval, is_heartbeat_request, NULL); -} - -static rd_kafka_t *create_consumer(const char *bootstraps, - const char *group_id, - rd_bool_t with_rebalance_cb) { - rd_kafka_conf_t *conf; - test_conf_init(&conf, NULL, 0); - test_conf_set(conf, "bootstrap.servers", bootstraps); - test_conf_set(conf, "group.protocol", "consumer"); - test_conf_set(conf, "auto.offset.reset", "earliest"); - return test_create_consumer( - group_id, with_rebalance_cb ? rebalance_cb : NULL, conf, NULL); -} - -/** - * @brief Test heartbeat behavior with fatal errors, - * ensuring: - * - a fatal error is received on poll and consumer close - * - no rebalance cb is called - * - no final leave group heartbeat is sent - * - * @param err The error code to test. - * @param variation See calling code. - */ -static void -do_test_consumer_group_heartbeat_fatal_error(rd_kafka_resp_err_t err, - int variation) { - rd_kafka_mock_cluster_t *mcluster; - const char *bootstraps; - rd_kafka_topic_partition_list_t *subscription; - rd_kafka_t *c; - rd_kafka_message_t *rkmessage; - int expected_heartbeats, found_heartbeats, expected_rebalance_cnt; - test_timing_t timing; - rebalance_cnt = 0; - rebalance_exp_lost = rd_false; - rebalance_exp_event = RD_KAFKA_RESP_ERR_NO_ERROR; - const char *topic = test_mk_topic_name(__FUNCTION__, 0); - - SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); - - mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster, 1000); - rd_kafka_mock_topic_create(mcluster, topic, 1, 1); - - TIMING_START(&timing, "consumer_group_heartbeat_fatal_error"); - - if (variation == 1) { - /* First HB returns assignment */ - rd_kafka_mock_broker_push_request_error_rtts( - mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, - RD_KAFKA_RESP_ERR_NO_ERROR, 0); - } - - rd_kafka_mock_broker_push_request_error_rtts( - mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, err, 0); - - c = create_consumer(bootstraps, topic, rd_true); - - /* Subscribe to the input topic */ - subscription = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(subscription, topic, - /* The partition is ignored in - * rd_kafka_subscribe() */ - RD_KAFKA_PARTITION_UA); - - TEST_SAY("Subscribing to topic\n"); - rd_kafka_mock_start_request_tracking(mcluster); - TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription)); - rd_kafka_topic_partition_list_destroy(subscription); - - expected_heartbeats = 1; - if (variation == 1) - expected_heartbeats++; - - TEST_SAY("Awaiting first HBs\n"); - TEST_ASSERT((found_heartbeats = - wait_all_heartbeats_done(mcluster, expected_heartbeats, - 200)) == expected_heartbeats, - "Expected %d heartbeats, got %d", expected_heartbeats, - found_heartbeats); - - rd_kafka_mock_clear_requests(mcluster); - - expected_rebalance_cnt = 0; - if (variation == 1) { - expected_rebalance_cnt++; - rebalance_exp_event = RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS; - - /* Trigger rebalance cb */ - rkmessage = rd_kafka_consumer_poll(c, 500); - TEST_ASSERT(!rkmessage, "No message should be returned"); - } - - TEST_SAY("Consume from c, a fatal error is returned\n"); - rkmessage = rd_kafka_consumer_poll(c, 500); - TEST_ASSERT(rkmessage != NULL, "An error message should be returned"); - TEST_ASSERT(rkmessage->err == RD_KAFKA_RESP_ERR__FATAL, - "Expected a _FATAL error, got %s", - rd_kafka_err2name(rkmessage->err)); - rd_kafka_message_destroy(rkmessage); - - TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, - "Expected %d rebalance events, got %d", - expected_rebalance_cnt, rebalance_cnt); - - expected_rebalance_cnt = 0; - if (variation == 1) { - expected_rebalance_cnt++; - rebalance_exp_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS; - rebalance_exp_lost = rd_true; - } - - /* Close c, a fatal error is returned */ - TEST_ASSERT(rd_kafka_consumer_close(c) == RD_KAFKA_RESP_ERR__FATAL, - "Expected a _FATAL error, got %s", rd_kafka_err2name(err)); - - TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, - "Expected %d rebalance events, got %d", - expected_rebalance_cnt, rebalance_cnt); - - TEST_SAY("Ensuring there are no leave group HBs\n"); - TEST_ASSERT((found_heartbeats = - wait_all_heartbeats_done(mcluster, 0, 200)) == 0, - "Expected no leave group heartbeat, got %d", - found_heartbeats); - - rd_kafka_mock_stop_request_tracking(mcluster); - rd_kafka_destroy(c); - test_mock_cluster_destroy(mcluster); - - TIMING_ASSERT(&timing, 500, 2000); - SUB_TEST_PASS(); -} - -/** - * @brief Test all kind of fatal errors in a ConsumerGroupHeartbeat call. - * variation 0: errors on first HB - * variation 1: errors on second HB - */ -static void do_test_consumer_group_heartbeat_fatal_errors(void) { - rd_kafka_resp_err_t fatal_errors[] = { - RD_KAFKA_RESP_ERR_INVALID_REQUEST, - RD_KAFKA_RESP_ERR_GROUP_MAX_SIZE_REACHED, - RD_KAFKA_RESP_ERR_UNSUPPORTED_ASSIGNOR, - RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION, - RD_KAFKA_RESP_ERR_UNRELEASED_INSTANCE_ID, - RD_KAFKA_RESP_ERR_GROUP_AUTHORIZATION_FAILED}; - size_t i; - for (i = 0; i < RD_ARRAY_SIZE(fatal_errors); i++) { - do_test_consumer_group_heartbeat_fatal_error(fatal_errors[i], - 0); - do_test_consumer_group_heartbeat_fatal_error(fatal_errors[i], - 1); - } -} - -/** - * @brief Test heartbeat behavior with retriable errors, - * ensuring: - * - no error is received on poll and consumer close - * - rebalance cb is called to assign and revoke - * - final leave group heartbeat is sent - * - * @param err The error code to test. - * @param variation See calling code. - */ -static void -do_test_consumer_group_heartbeat_retriable_error(rd_kafka_resp_err_t err, - int variation) { - rd_kafka_mock_cluster_t *mcluster; - const char *bootstraps; - rd_kafka_topic_partition_list_t *subscription; - rd_kafka_t *c; - int expected_heartbeats, found_heartbeats; - test_timing_t timing; - const char *topic = test_mk_topic_name(__FUNCTION__, 0); - test_curr->is_fatal_cb = error_is_fatal_cb; - rebalance_cnt = 0; - rebalance_exp_lost = rd_false; - allowed_error = RD_KAFKA_RESP_ERR__TRANSPORT; - - SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); - - - mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster, 1000); - rd_kafka_mock_topic_create(mcluster, topic, 1, 1); - - c = create_consumer(bootstraps, topic, rd_true); - - TIMING_START(&timing, "consumer_group_heartbeat_retriable_error"); - - if (variation == 1) { - /* First HB returns assignment */ - rd_kafka_mock_broker_push_request_error_rtts( - mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, - RD_KAFKA_RESP_ERR_NO_ERROR, 0); - } - - rd_kafka_mock_broker_push_request_error_rtts( - mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, err, 0); - - /* Subscribe to the input topic */ - subscription = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(subscription, topic, - /* The partition is ignored in - * rd_kafka_subscribe() */ - RD_KAFKA_PARTITION_UA); - - TEST_SAY("Subscribing to topic\n"); - rd_kafka_mock_start_request_tracking(mcluster); - TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription)); - rd_kafka_topic_partition_list_destroy(subscription); - - /* First HB and retry */ - expected_heartbeats = 2; - rebalance_exp_event = RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS; - if (variation == 1) { - TEST_SAY( - "Consume from c, no message is returned, " - "but assign callback is processed\n"); - test_consumer_poll_no_msgs("after heartbeat", c, 0, 200); - - /* wait 1 HB interval more */ - expected_heartbeats += 1; - rebalance_exp_event = RD_KAFKA_RESP_ERR_NO_ERROR; - } - - TEST_SAY("Awaiting first HBs\n"); - TEST_ASSERT((found_heartbeats = - wait_all_heartbeats_done(mcluster, expected_heartbeats, - 200)) == expected_heartbeats, - "Expected %d heartbeats, got %d", expected_heartbeats, - found_heartbeats); - - TEST_SAY("Consume from c, no message is returned\n"); - test_consumer_poll_no_msgs("after heartbeat", c, 0, 250); - - TEST_ASSERT(rebalance_cnt > 0, "Expected > 0 rebalance events, got %d", - rebalance_cnt); - - rebalance_exp_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS; - - rd_kafka_mock_clear_requests(mcluster); - rebalance_cnt = 0; - /* Close c without errors */ - TEST_ASSERT(rd_kafka_consumer_close(c) == RD_KAFKA_RESP_ERR_NO_ERROR, - "Expected NO_ERROR, got %s", rd_kafka_err2name(err)); - TEST_ASSERT(rebalance_cnt > 0, "Expected > 0 rebalance events, got %d", - rebalance_cnt); - rebalance_exp_event = RD_KAFKA_RESP_ERR_NO_ERROR; - - TEST_SAY("Awaiting leave group HB\n"); - TEST_ASSERT((found_heartbeats = - wait_all_heartbeats_done(mcluster, 1, 200)) == 1, - "Expected 1 leave group heartbeat, got %d", - found_heartbeats); - - rd_kafka_mock_stop_request_tracking(mcluster); - rd_kafka_destroy(c); - test_mock_cluster_destroy(mcluster); - - TIMING_ASSERT(&timing, 500, 2000); - - test_curr->is_fatal_cb = NULL; - allowed_error = RD_KAFKA_RESP_ERR_NO_ERROR; - - SUB_TEST_PASS(); -} - -/** - * @brief Test all kind of retriable errors in a ConsumerGroupHeartbeat call. - * variation 0: errors on first HB - * variation 1: errors on second HB - */ -static void do_test_consumer_group_heartbeat_retriable_errors(void) { - rd_kafka_resp_err_t retriable_errors[] = { - RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS, - RD_KAFKA_RESP_ERR__SSL, RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE}; - size_t i; - for (i = 0; i < RD_ARRAY_SIZE(retriable_errors); i++) { - do_test_consumer_group_heartbeat_retriable_error( - retriable_errors[i], 0); - do_test_consumer_group_heartbeat_retriable_error( - retriable_errors[i], 1); - } -} - -/** - * @brief Test heartbeat behavior with consumer fenced errors, - * ensuring: - * - no error is received on poll and consumer close - * - rebalance callbacks are called, with partitions lost when - * necessary - * - a final leave group heartbeat is sent - * - * @param err The error code to test. - * @param variation See calling code. - */ -static void -do_test_consumer_group_heartbeat_fenced_error(rd_kafka_resp_err_t err, - int variation) { - rd_kafka_mock_cluster_t *mcluster; - const char *bootstraps; - rd_kafka_topic_partition_list_t *subscription; - rd_kafka_t *c; - rd_kafka_message_t *rkmessage; - int expected_heartbeats, found_heartbeats, expected_rebalance_cnt; - test_timing_t timing; - rebalance_cnt = 0; - rebalance_exp_lost = rd_false; - rebalance_exp_event = RD_KAFKA_RESP_ERR_NO_ERROR; - const char *topic = test_mk_topic_name(__FUNCTION__, 0); - - SUB_TEST_QUICK("%s, variation %d", rd_kafka_err2name(err), variation); - - mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster, 1000); - rd_kafka_mock_topic_create(mcluster, topic, 1, 1); - - if (variation == 1) { - /* First HB returns assignment */ - rd_kafka_mock_broker_push_request_error_rtts( - mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, - RD_KAFKA_RESP_ERR_NO_ERROR, 0); - } - - rd_kafka_mock_broker_push_request_error_rtts( - mcluster, 1, RD_KAFKAP_ConsumerGroupHeartbeat, 1, err, 0); - - c = create_consumer(bootstraps, topic, rd_true); - - TIMING_START(&timing, "consumer_group_heartbeat_fenced_error"); - - /* Subscribe to the input topic */ - subscription = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(subscription, topic, - /* The partition is ignored in - * rd_kafka_subscribe() */ - RD_KAFKA_PARTITION_UA); - - TEST_SAY("Subscribing to topic\n"); - rd_kafka_mock_start_request_tracking(mcluster); - TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription)); - rd_kafka_topic_partition_list_destroy(subscription); - - /*First HB is fenced and second receives assignment*/ - expected_heartbeats = 2; - if (variation == 1) - /*First HB receives assignment*/ - expected_heartbeats = 1; - - TEST_SAY("Awaiting initial HBs\n"); - TEST_ASSERT((found_heartbeats = - wait_all_heartbeats_done(mcluster, expected_heartbeats, - 200)) == expected_heartbeats, - "Expected %d heartbeats, got %d", expected_heartbeats, - found_heartbeats); - - expected_rebalance_cnt = 0; - /* variation 0: Second HB assigned */ - if (variation == 1) { - expected_rebalance_cnt++; - rebalance_exp_event = RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS; - - /* First HB assigned */ - rkmessage = rd_kafka_consumer_poll(c, 100); - TEST_ASSERT(!rkmessage, "No message should be returned"); - - TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, - "Expected %d rebalance events after assign " - "callback, got %d", - expected_rebalance_cnt, rebalance_cnt); - - TEST_SAY("Awaiting partition lost callback\n"); - /* Second HB acks and loses partitions */ - expected_rebalance_cnt++; - rebalance_exp_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS; - rebalance_exp_lost = rd_true; - /* Needs to wait HB interval */ - rkmessage = rd_kafka_consumer_poll(c, 750); - TEST_ASSERT(!rkmessage, "No message should be returned"); - - TEST_ASSERT( - rebalance_cnt == expected_rebalance_cnt, - "Expected %d rebalance events after lost callback, got %d", - expected_rebalance_cnt, rebalance_cnt); - - /* Third HB assigns again */ - } - - expected_rebalance_cnt++; - rebalance_exp_event = RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS; - rebalance_exp_lost = rd_false; - - TEST_SAY("Clearing mock requests\n"); - rd_kafka_mock_clear_requests(mcluster); - expected_heartbeats = 0; - - TEST_SAY("Awaiting rebalance callback\n"); - /* Consume from c, partitions are lost if assigned */ - rkmessage = rd_kafka_consumer_poll(c, 500); - TEST_ASSERT(!rkmessage, "No message should be returned"); - - TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, - "Expected %d total rebalance events, got %d", - expected_rebalance_cnt, rebalance_cnt); - - if (variation == 0) { - /* Ack for assignment HB */ - expected_heartbeats++; - } else if (variation == 1) { - /* First HB assigns again - * Second HB acks assignment */ - expected_heartbeats += 2; - } - - TEST_SAY("Awaiting acknowledge heartbeat\n"); - TEST_ASSERT((found_heartbeats = - wait_all_heartbeats_done(mcluster, expected_heartbeats, - 100)) == expected_heartbeats, - "Expected %d heartbeats, got %d", expected_heartbeats, - found_heartbeats); - - expected_rebalance_cnt++; - rebalance_exp_event = RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS; - - rd_kafka_mock_clear_requests(mcluster); - /* Close c, no error is returned */ - TEST_CALL_ERR__(rd_kafka_consumer_close(c)); - - TEST_ASSERT(rebalance_cnt == expected_rebalance_cnt, - "Expected %d rebalance events, got %d", - expected_rebalance_cnt, rebalance_cnt); - - TEST_SAY("Awaiting leave group heartbeat\n"); - /* After closing the consumer, 1 heartbeat should been sent */ - TEST_ASSERT((found_heartbeats = - wait_all_heartbeats_done(mcluster, 1, 200)) == 1, - "Expected 1 leave group heartbeat, got %d", - found_heartbeats); - - rd_kafka_mock_stop_request_tracking(mcluster); - rd_kafka_destroy(c); - test_mock_cluster_destroy(mcluster); - - TIMING_ASSERT(&timing, 500, 2000); - SUB_TEST_PASS(); -} - -/** - * @brief Test all kind of consumer fenced errors in a ConsumerGroupHeartbeat - * call. - * variation 0: errors on first HB - * variation 1: errors on second HB - */ -static void do_test_consumer_group_heartbeat_fenced_errors(void) { - rd_kafka_resp_err_t fenced_errors[] = { - RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID, - RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH}; - size_t i; - for (i = 0; i < RD_ARRAY_SIZE(fenced_errors); i++) { - do_test_consumer_group_heartbeat_fenced_error(fenced_errors[i], - 0); - do_test_consumer_group_heartbeat_fenced_error(fenced_errors[i], - 1); - } -} - -/** - * @brief Test consumer group behavior with missing topic id when retrieving - * metadata for assigned topics. - * ensuring: - * - initially a partial acknoledgement is started, with an empty list - * (variation 0) or a single topic (variation 1) - * - fetch doesn't start until broker returns an unknown topic id error - * - when error isn't returned anymore the client finishes assigning - * the partition and reads a message. - * - * @param variation See calling code. - */ -static void do_test_metadata_unknown_topic_id_error(int variation) { - rd_kafka_mock_cluster_t *mcluster; - const char *bootstraps; - rd_kafka_topic_partition_list_t *subscription; - rd_kafka_t *c; - test_timing_t timing; - const char *topic = "do_test_metadata_unknown_topic_id_error"; - const char *topic2 = "do_test_metadata_unknown_topic_id_error2"; - - SUB_TEST_QUICK("variation: %d", variation); - - mcluster = test_mock_cluster_new(1, &bootstraps); - rd_kafka_mock_set_group_consumer_heartbeat_interval_ms(mcluster, 500); - rd_kafka_mock_topic_create(mcluster, topic, 1, 1); - if (variation == 1) { - rd_kafka_mock_topic_create(mcluster, topic2, 1, 1); - } - - c = create_consumer(bootstraps, topic, rd_false); - - /* Seed the topic with messages */ - test_produce_msgs_easy_v(topic, 0, 0, 0, 1, 1000, "bootstrap.servers", - bootstraps, NULL); - - TIMING_START(&timing, "do_test_metadata_unknown_topic_id_error"); - - subscription = rd_kafka_topic_partition_list_new(1); - rd_kafka_topic_partition_list_add(subscription, topic, - RD_KAFKA_PARTITION_UA); - if (variation == 1) { - rd_kafka_topic_partition_list_add(subscription, topic2, - RD_KAFKA_PARTITION_UA); - } - - rd_kafka_mock_topic_set_error(mcluster, topic, - RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_ID); - - TEST_SAY("Subscribing to topic\n"); - TEST_CALL_ERR__(rd_kafka_subscribe(c, subscription)); - rd_kafka_topic_partition_list_destroy(subscription); - - TEST_SAY( - "Cannot fetch until Metadata calls replies with " - "UNKNOWN_TOPIC_ID\n"); - test_consumer_poll_no_msgs("no messages", c, 0, 1000); - - rd_kafka_mock_topic_set_error(mcluster, topic, - RD_KAFKA_RESP_ERR_NO_ERROR); - - TEST_SAY("Reconciliation and fetch is now possible\n"); - test_consumer_poll_timeout("message", c, 0, 0, 0, 1, NULL, 2000); - - rd_kafka_destroy(c); - test_mock_cluster_destroy(mcluster); - - TIMING_ASSERT(&timing, 500, 4000); - SUB_TEST_PASS(); -} - -/** - * @brief Test these variations of a UNKNOWN_TOPIC_ID in a Metadata call - * before reconciliation. - * - * variation 0: single topic - * variation 1: two topics: first gives this error, second exists. - */ -static void do_test_metadata_unknown_topic_id_tests(void) { - do_test_metadata_unknown_topic_id_error(0); - do_test_metadata_unknown_topic_id_error(1); -} - -int main_0147_consumer_group_consumer_mock(int argc, char **argv) { - TEST_SKIP_MOCK_CLUSTER(0); - - if (test_consumer_group_protocol_classic()) { - TEST_SKIP("Test only for group.protocol=consumer\n"); - return 0; - } - - do_test_consumer_group_heartbeat_fatal_errors(); - - do_test_consumer_group_heartbeat_retriable_errors(); - - do_test_consumer_group_heartbeat_fenced_errors(); - - do_test_metadata_unknown_topic_id_tests(); - - return 0; -} diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 15c1cf6eda..93ec0d57d8 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -137,7 +137,6 @@ set( 0144-idempotence_mock.c 0145-pause_resume_mock.c 0146-metadata_mock.c - 0147-consumer_group_consumer_mock.c 0150-telemetry_mock.c 8000-idle.cpp 8001-fetch_from_follower_mock_manual.c diff --git a/tests/test.c b/tests/test.c index 1a1a4116f6..aa03310efa 100644 --- a/tests/test.c +++ b/tests/test.c @@ -262,7 +262,6 @@ _TEST_DECL(0143_exponential_backoff_mock); _TEST_DECL(0144_idempotence_mock); _TEST_DECL(0145_pause_resume_mock); _TEST_DECL(0146_metadata_mock); -_TEST_DECL(0147_consumer_group_consumer_mock); _TEST_DECL(0150_telemetry_mock); /* Manual tests */ @@ -522,7 +521,6 @@ struct test tests[] = { _TEST(0144_idempotence_mock, TEST_F_LOCAL, TEST_BRKVER(0, 11, 0, 0)), _TEST(0145_pause_resume_mock, TEST_F_LOCAL), _TEST(0146_metadata_mock, TEST_F_LOCAL), - _TEST(0147_consumer_group_consumer_mock, TEST_F_LOCAL), _TEST(0150_telemetry_mock, 0), diff --git a/win32/tests/tests.vcxproj b/win32/tests/tests.vcxproj index 738227e057..b11bfdab75 100644 --- a/win32/tests/tests.vcxproj +++ b/win32/tests/tests.vcxproj @@ -227,7 +227,6 @@ - From f2078819acf007c42d700fda19aba229c4486263 Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Thu, 28 Nov 2024 21:12:46 +0530 Subject: [PATCH 10/19] first pass --- examples/consumer.c | 7 + examples/describe_consumer_groups.c | 18 +- src/rdkafka.c | 1 + src/rdkafka.h | 45 ++++ src/rdkafka_admin.c | 376 +++++++++++++++++++++++----- src/rdkafka_admin.h | 4 + src/rdkafka_op.c | 6 +- src/rdkafka_op.h | 3 + src/rdkafka_proto.h | 1 + src/rdkafka_request.c | 59 +++++ src/rdkafka_request.h | 10 + 11 files changed, 468 insertions(+), 62 deletions(-) diff --git a/examples/consumer.c b/examples/consumer.c index dad3efc43b..76ebfbd633 100644 --- a/examples/consumer.c +++ b/examples/consumer.c @@ -139,6 +139,13 @@ int main(int argc, char **argv) { return 1; } + if(rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, + sizeof(errstr)) != RD_KAFKA_CONF_OK) { + fprintf(stderr, "%s\n", errstr); + rd_kafka_conf_destroy(conf); + return 1; + } + /* * Create consumer instance. * diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index daacc1d021..d58116cf94 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -174,6 +174,18 @@ print_group_member_info(const rd_kafka_MemberDescription_t *member) { printf(" Assignment:\n"); print_partition_list(stdout, topic_partitions, 0, " "); } + const rd_kafka_MemberAssignment_t *target_assignment = + rd_kafka_MemberDescription_target_assignment(member); + const rd_kafka_topic_partition_list_t *target_topic_partitions = + rd_kafka_MemberAssignment_partitions(target_assignment); + if (!target_topic_partitions) { + printf(" No target assignment\n"); + } else if (target_topic_partitions->cnt == 0) { + printf(" Empty target assignment\n"); + } else { + printf(" Target assignment:\n"); + print_partition_list(stdout, target_topic_partitions, 0, " "); + } } @@ -194,6 +206,8 @@ static void print_group_info(const rd_kafka_ConsumerGroupDescription_t *group) { rd_kafka_ConsumerGroupDescription_partition_assignor(group); rd_kafka_consumer_group_state_t state = rd_kafka_ConsumerGroupDescription_state(group); + rd_kafka_consumer_group_type_t type = + rd_kafka_ConsumerGroupDescription_type(group); authorized_operations = rd_kafka_ConsumerGroupDescription_authorized_operations( group, &authorized_operations_cnt); @@ -212,9 +226,9 @@ static void print_group_info(const rd_kafka_ConsumerGroupDescription_t *group) { rd_kafka_Node_port(coordinator)); } printf( - "Group \"%s\", partition assignor \"%s\", " + "Group \"%s\", partition assignor \"%s\", type \"%s\" " " state %s%s, with %" PRId32 " member(s)\n", - group_id, partition_assignor, + group_id, partition_assignor, rd_kafka_consumer_group_type_name(type), rd_kafka_consumer_group_state_name(state), coordinator_desc, member_cnt); for (j = 0; j < authorized_operations_cnt; j++) { diff --git a/src/rdkafka.c b/src/rdkafka.c index 656076df1b..b874f9634a 100644 --- a/src/rdkafka.c +++ b/src/rdkafka.c @@ -1660,6 +1660,7 @@ static void rd_kafka_stats_emit_broker_reqs(struct _stats_emit *st, [RD_KAFKAP_AlterClientQuotas] = rd_true, [RD_KAFKAP_DescribeUserScramCredentials] = rd_true, [RD_KAFKAP_AlterUserScramCredentials] = rd_true, + [RD_KAFKAP_ConsumerGroupDescribe] = rd_true, }}; int i; int cnt = 0; diff --git a/src/rdkafka.h b/src/rdkafka.h index e5d47d3264..7724c490e8 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -5593,6 +5593,8 @@ typedef int rd_kafka_event_type_t; #define RD_KAFKA_EVENT_LISTOFFSETS_RESULT 0x400000 /** ElectLeaders_result_t */ #define RD_KAFKA_EVENT_ELECTLEADERS_RESULT 0x800000 +/** ConsumerGroupDescribe_result_t */ +#define RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT 0x1000000 /** * @returns the event type for the given event. @@ -5752,6 +5754,7 @@ int rd_kafka_event_error_is_fatal(rd_kafka_event_t *rkev); * - RD_KAFKA_EVENT_DESCRIBECLUSTER_RESULT * - RD_KAFKA_EVENT_LISTOFFSETS_RESULT * - RD_KAFKA_EVENT_ELECTLEADERS_RESULT + * - RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT */ RD_EXPORT void *rd_kafka_event_opaque(rd_kafka_event_t *rkev); @@ -8881,6 +8884,20 @@ RD_EXPORT const rd_kafka_Node_t *rd_kafka_ConsumerGroupDescription_coordinator( const rd_kafka_ConsumerGroupDescription_t *grpdesc); +/** + * @brief Gets type for the \p grpdesc group. + * + * @param grpdesc The group description. + * + * @return A group type. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p grpdesc object. + */ +RD_EXPORT +rd_kafka_consumer_group_type_t rd_kafka_ConsumerGroupDescription_type( + const rd_kafka_ConsumerGroupDescription_t *grpdesc); + /** * @brief Gets the members count of \p grpdesc group. * @@ -8993,6 +9010,34 @@ RD_EXPORT const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_partitions( const rd_kafka_MemberAssignment_t *assignment); +/** + * @brief Gets target assignment of \p member. + * + * @param member The group member. + * + * @return The target assignment. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p member object. + */ +RD_EXPORT +const rd_kafka_MemberAssignment_t *rd_kafka_MemberDescription_target_assignment( + const rd_kafka_MemberDescription_t *member); + +/** + * @brief Gets target assigned partitions of a member \p assignment. + * + * @param assignment The group member assignment. + * + * @return The target assigned partitions. + * + * @remark The lifetime of the returned memory is the same + * as the lifetime of the \p assignment object. + */ +RD_EXPORT +const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_target_partitions( + const rd_kafka_MemberAssignment_t *assignment); + /**@}*/ /** diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 41934e349d..7d96b6e876 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -7748,7 +7748,8 @@ static rd_kafka_MemberDescription_t *rd_kafka_MemberDescription_new( const char *consumer_id, const char *group_instance_id, const char *host, - const rd_kafka_topic_partition_list_t *assignment) { + const rd_kafka_topic_partition_list_t *assignment, + const rd_kafka_topic_partition_list_t *target_assignment) { rd_kafka_MemberDescription_t *member; member = rd_calloc(1, sizeof(*member)); member->client_id = rd_strdup(client_id); @@ -7762,6 +7763,12 @@ static rd_kafka_MemberDescription_t *rd_kafka_MemberDescription_new( else member->assignment.partitions = rd_kafka_topic_partition_list_new(0); + if (target_assignment) + member->target_assignment.partitions = + rd_kafka_topic_partition_list_copy(target_assignment); + else + member->target_assignment.partitions = + rd_kafka_topic_partition_list_new(0); return member; } @@ -7777,7 +7784,8 @@ static rd_kafka_MemberDescription_t * rd_kafka_MemberDescription_copy(const rd_kafka_MemberDescription_t *src) { return rd_kafka_MemberDescription_new(src->client_id, src->consumer_id, src->group_instance_id, src->host, - src->assignment.partitions); + src->assignment.partitions, + src->target_assignment.partitions); } /** @@ -7801,6 +7809,9 @@ rd_kafka_MemberDescription_destroy(rd_kafka_MemberDescription_t *member) { if (member->assignment.partitions) rd_kafka_topic_partition_list_destroy( member->assignment.partitions); + if(member->target_assignment.partitions) + rd_kafka_topic_partition_list_destroy( + member->target_assignment.partitions); rd_free(member); } @@ -7838,6 +7849,16 @@ const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_partitions( return assignment->partitions; } +const rd_kafka_MemberAssignment_t *rd_kafka_MemberDescription_target_assignment( + const rd_kafka_MemberDescription_t *member) { + return &member->target_assignment; +} + +const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_target_partitions( + const rd_kafka_MemberAssignment_t *assignment) { + return assignment->partitions; +} + /** * @brief Create a new ConsumerGroupDescription object. @@ -7864,7 +7885,8 @@ rd_kafka_ConsumerGroupDescription_new( int authorized_operations_cnt, rd_kafka_consumer_group_state_t state, const rd_kafka_Node_t *coordinator, - rd_kafka_error_t *error) { + rd_kafka_error_t *error, + rd_kafka_consumer_group_type_t type) { rd_kafka_ConsumerGroupDescription_t *grpdesc; grpdesc = rd_calloc(1, sizeof(*grpdesc)); grpdesc->group_id = rd_strdup(group_id); @@ -7892,6 +7914,7 @@ rd_kafka_ConsumerGroupDescription_new( error != NULL ? rd_kafka_error_new(rd_kafka_error_code(error), "%s", rd_kafka_error_string(error)) : NULL; + grpdesc->type = type; return grpdesc; } @@ -7905,10 +7928,11 @@ rd_kafka_ConsumerGroupDescription_new( */ static rd_kafka_ConsumerGroupDescription_t * rd_kafka_ConsumerGroupDescription_new_error(const char *group_id, - rd_kafka_error_t *error) { + rd_kafka_error_t *error, + rd_kafka_consumer_group_type_t type) { return rd_kafka_ConsumerGroupDescription_new( group_id, rd_false, NULL, NULL, NULL, 0, - RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN, NULL, error); + RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN, NULL, error, type); } /** @@ -7924,7 +7948,7 @@ rd_kafka_ConsumerGroupDescription_copy( grpdesc->group_id, grpdesc->is_simple_consumer_group, &grpdesc->members, grpdesc->partition_assignor, grpdesc->authorized_operations, grpdesc->authorized_operations_cnt, - grpdesc->state, grpdesc->coordinator, grpdesc->error); + grpdesc->state, grpdesc->coordinator, grpdesc->error, grpdesc->type); } /** @@ -7996,6 +8020,11 @@ const rd_kafka_Node_t *rd_kafka_ConsumerGroupDescription_coordinator( return grpdesc->coordinator; } +rd_kafka_consumer_group_type_t rd_kafka_ConsumerGroupDescription_type( + const rd_kafka_ConsumerGroupDescription_t *grpdesc) { + return grpdesc->type; +} + size_t rd_kafka_ConsumerGroupDescription_member_count( const rd_kafka_ConsumerGroupDescription_t *grpdesc) { return rd_list_cnt(&grpdesc->members); @@ -8015,51 +8044,6 @@ static int rd_kafka_DescribeConsumerGroups_cmp(const void *a, const void *b) { return strcmp(a, b); } -/** @brief Merge the DescribeConsumerGroups response from a single broker - * into the user response list. - */ -static void rd_kafka_DescribeConsumerGroups_response_merge( - rd_kafka_op_t *rko_fanout, - const rd_kafka_op_t *rko_partial) { - rd_kafka_ConsumerGroupDescription_t *groupres = NULL; - rd_kafka_ConsumerGroupDescription_t *newgroupres; - const char *grp = rko_partial->rko_u.admin_result.opaque; - int orig_pos; - - rd_assert(rko_partial->rko_evtype == - RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT); - - if (!rko_partial->rko_err) { - /* Proper results. - * We only send one group per request, make sure it matches */ - groupres = - rd_list_elem(&rko_partial->rko_u.admin_result.results, 0); - rd_assert(groupres); - rd_assert(!strcmp(groupres->group_id, grp)); - newgroupres = rd_kafka_ConsumerGroupDescription_copy(groupres); - } else { - /* Op errored, e.g. timeout */ - rd_kafka_error_t *error = - rd_kafka_error_new(rko_partial->rko_err, NULL); - newgroupres = - rd_kafka_ConsumerGroupDescription_new_error(grp, error); - rd_kafka_error_destroy(error); - } - - /* As a convenience to the application we insert group result - * in the same order as they were requested. */ - orig_pos = rd_list_index(&rko_fanout->rko_u.admin_request.args, grp, - rd_kafka_DescribeConsumerGroups_cmp); - rd_assert(orig_pos != -1); - - /* Make sure result is not already set */ - rd_assert(rd_list_elem(&rko_fanout->rko_u.admin_request.fanout.results, - orig_pos) == NULL); - - rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results, orig_pos, - newgroupres); -} - /** * @brief Construct and send DescribeConsumerGroupsRequest to \p rkb @@ -8250,7 +8234,7 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, member = rd_kafka_MemberDescription_new( client_id, member_id, group_instance_id, - client_host, partitions); + client_host, partitions, NULL); if (partitions) rd_kafka_topic_partition_list_destroy( partitions); @@ -8279,10 +8263,10 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, group_id, is_simple_consumer_group, &members, proto, operations, operation_cnt, rd_kafka_consumer_group_state_code(group_state), - node, error); + node, error, RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC); } else grpdesc = rd_kafka_ConsumerGroupDescription_new_error( - group_id, error); + group_id, error, RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC); rd_list_add(&rko_result->rko_u.admin_result.results, grpdesc); @@ -8336,6 +8320,280 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, return reply->rkbuf_err; } +static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( + rd_kafka_op_t *rko_req, + rd_kafka_op_t **rko_resultp, + rd_kafka_buf_t *reply, + char *errstr, + size_t errstr_size) { + const int log_decode_errors = LOG_ERR; + int16_t api_version; + int32_t cnt; + rd_kafka_op_t *rko_result = NULL; + rd_kafka_broker_t *rkb = reply->rkbuf_rkb; + rd_kafka_error_t *error = NULL; + char *group_id = NULL, *group_state = NULL, *assignor_name = NULL, *error_str = NULL, *host = NULL; + int32_t group_epoch, assignment_epoch; + rd_kafka_AclOperation_t *operations = NULL; + rd_kafka_Node_t *node = NULL; + int32_t nodeid; + uint16_t port; + int operation_cnt = -1; + printf("rd_kafka_ConsumerGroupDescribeResponseParse\n"); + + api_version = rd_kafka_buf_ApiVersion(reply); + rd_kafka_buf_read_throttle_time(reply); + + rd_kafka_buf_read_arraycnt(reply, &cnt, 100000); + + rd_kafka_broker_lock(rkb); + nodeid = rkb->rkb_nodeid; + host = rd_strdup(rkb->rkb_origname); + port = rkb->rkb_port; + rd_kafka_broker_unlock(rkb); + + node = rd_kafka_Node_new(nodeid, host, port, NULL); + + rko_result = rd_kafka_admin_result_new(rko_req); + rd_list_init(&rko_result->rko_u.admin_result.results, cnt, + rd_kafka_ConsumerGroupDescription_free); + + for(int i = 0; i < cnt; i++) { + int16_t error_code; + int32_t authorized_operations = -1; + int32_t member_cnt; + rd_kafkap_str_t GroupId, GroupState, AssignorName, ErrorString; + rd_list_t members; + rd_kafka_ConsumerGroupDescription_t *grpdesc = NULL; + + rd_kafka_buf_read_i16(reply, &error_code); + rd_kafka_buf_read_str(reply, &ErrorString); + rd_kafka_buf_read_str(reply, &GroupId); + rd_kafka_buf_read_str(reply, &GroupState); + rd_kafka_buf_read_i32(reply, &group_epoch); + rd_kafka_buf_read_i32(reply, &assignment_epoch); + rd_kafka_buf_read_str(reply, &AssignorName); + rd_kafka_buf_read_arraycnt(reply, &member_cnt, 100000); + + group_id = RD_KAFKAP_STR_DUP(&GroupId); + group_state = RD_KAFKAP_STR_DUP(&GroupState); + assignor_name = RD_KAFKAP_STR_DUP(&AssignorName); + error_str = RD_KAFKAP_STR_DUP(&ErrorString); + printf("error_code: %d\n", error_code); + printf("error string: %s\n", error_str); + printf("group_id: %s\n", group_id); + printf("group_state: %s\n", group_state); + printf("group_epoch: %d\n", group_epoch); + printf("assignment_epoch: %d\n", assignment_epoch); + printf("assignor_name: %s\n", assignor_name); + printf("member_cnt: %d\n", member_cnt); + + if(error_code) { + error = rd_kafka_error_new(error_code, "ConsumerGroupDescribe: %s", error_str); + } + + rd_list_init(&members, 0, rd_kafka_MemberDescription_free); + + for(int j = 0; j < member_cnt; j++) { + rd_kafkap_str_t MemberId, InstanceId, RackId, ClientId, ClientHost, SubscribedTopicNames, SubscribedTopicRegex; + int32_t MemberEpoch; + char *member_id, *instance_id, *rack_id, + *client_id, *client_host, *subscribed_topic_names, + *subscribed_topic_regex = NULL; + rd_kafka_MemberDescription_t *member = NULL; + rd_kafka_topic_partition_list_t *assignment = NULL, *target_assignment = NULL; + + rd_kafka_buf_read_str(reply, &MemberId); + rd_kafka_buf_read_str(reply, &InstanceId); + rd_kafka_buf_read_str(reply, &RackId); + rd_kafka_buf_read_i32(reply, &MemberEpoch); + rd_kafka_buf_read_str(reply, &ClientId); + rd_kafka_buf_read_str(reply, &ClientHost); + rd_kafka_buf_read_str(reply, &SubscribedTopicNames); + rd_kafka_buf_read_str(reply, &SubscribedTopicRegex); + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + assignment = rd_kafka_buf_read_topic_partitions( + reply, rd_true /* use topic_id */, + rd_true /* use topic name*/, 0, fields); + rd_kafka_buf_skip_tags(reply); + target_assignment = rd_kafka_buf_read_topic_partitions( + reply, rd_true /* use topic_id */, + rd_true /* use topic name*/, 0, fields); + rd_kafka_buf_skip_tags(reply); + + member_id = RD_KAFKAP_STR_DUP(&MemberId); + instance_id = RD_KAFKAP_STR_DUP(&InstanceId); + rack_id = RD_KAFKAP_STR_DUP(&RackId); + client_id = RD_KAFKAP_STR_DUP(&ClientId); + client_host = RD_KAFKAP_STR_DUP(&ClientHost); + subscribed_topic_names = RD_KAFKAP_STR_DUP(&SubscribedTopicNames); + subscribed_topic_regex = RD_KAFKAP_STR_DUP(&SubscribedTopicRegex); + + printf("member_id: %s\n", member_id); + printf("instance_id: %s\n", instance_id); + printf("rack_id: %s\n", rack_id); + printf("member_epoch: %d\n", MemberEpoch); + printf("client_id: %s\n", client_id); + printf("client_host: %s\n", client_host); + printf("subscribed_topic_names: %s\n", subscribed_topic_names); + printf("subscribed_topic_regex: %s\n", subscribed_topic_regex); + printf("assignment: topic: %s partition: %d\n", assignment->elems[0].topic, assignment->elems[0].partition); + printf("target_assignment: topic: %s partition: %d\n", target_assignment->elems[0].topic, target_assignment->elems[0].partition); + member = rd_kafka_MemberDescription_new( + client_id, member_id, instance_id, client_host, assignment, target_assignment); + + rd_list_add(&members, member); + rd_kafka_buf_skip_tags(reply); + + if(assignment) + rd_kafka_topic_partition_list_destroy(assignment); + if(target_assignment) + rd_kafka_topic_partition_list_destroy(target_assignment); + rd_free(member_id); + rd_free(instance_id); + rd_free(rack_id); + rd_free(client_id); + rd_free(client_host); + rd_free(subscribed_topic_names); + rd_free(subscribed_topic_regex); + } + rd_kafka_buf_read_i32(reply, &authorized_operations); + operations = rd_kafka_AuthorizedOperations_parse( + authorized_operations, &operation_cnt); + rd_kafka_buf_skip_tags(reply); + + if(error == NULL) { + grpdesc = rd_kafka_ConsumerGroupDescription_new( + group_id, rd_false, &members, assignor_name, operations, operation_cnt, + rd_kafka_consumer_group_state_code(group_state), + node, error, RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER); + } else { + grpdesc = rd_kafka_ConsumerGroupDescription_new_error( + group_id, error , RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER); + } + rd_list_add(&rko_result->rko_u.admin_result.results, grpdesc); + + rd_list_destroy(&members); + rd_free(group_id); + rd_free(group_state); + rd_free(assignor_name); + rd_free(error_str); + RD_IF_FREE(error, rd_kafka_error_destroy); + RD_IF_FREE(operations, rd_free); + + error = NULL; + group_id = NULL; + group_state = NULL; + assignor_name = NULL; + error_str = NULL; + operations = NULL; + } + rd_kafka_buf_skip_tags(reply); + *rko_resultp = rko_result; + printf("rd_kafka_ConsumerGroupDescribeResponseParse end\n"); + return RD_KAFKA_RESP_ERR_NO_ERROR; +err_parse: + if(group_id) + rd_free(group_id); + if(group_state) + rd_free(group_state); + if(assignor_name) + rd_free(assignor_name); + if(error_str) + rd_free(error_str); + if(error) + rd_kafka_error_destroy(error); + RD_IF_FREE(operations, rd_free); + if(rko_result) + rd_kafka_op_destroy(rko_result); + rd_snprintf(errstr, errstr_size, + "DescribeConsumerGroups response protocol parse failure: %s", + rd_kafka_err2str(reply->rkbuf_err)); + return reply->rkbuf_err; + +} + +/** @brief Merge the DescribeConsumerGroups response from a single broker + * into the user response list. + */ +static void rd_kafka_DescribeConsumerGroups_response_merge( + rd_kafka_op_t *rko_fanout, + const rd_kafka_op_t *rko_partial) { + rd_kafka_ConsumerGroupDescription_t *groupres = NULL; + rd_kafka_ConsumerGroupDescription_t *newgroupres; + const char *grp = rko_partial->rko_u.admin_result.opaque; + int orig_pos; + + rd_assert(rko_partial->rko_evtype == + RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT || rko_partial->rko_evtype == RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT); + + if (!rko_partial->rko_err) { + /* Proper results. + * We only send one group per request, make sure it matches */ + groupres = + rd_list_elem(&rko_partial->rko_u.admin_result.results, 0); + rd_assert(groupres); + rd_assert(!strcmp(groupres->group_id, grp)); + newgroupres = rd_kafka_ConsumerGroupDescription_copy(groupres); + } else { + /* Op errored, e.g. timeout */ + rd_kafka_error_t *error = + rd_kafka_error_new(rko_partial->rko_err, NULL); + newgroupres = + rd_kafka_ConsumerGroupDescription_new_error(grp, error, RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN); + rd_kafka_error_destroy(error); + } + + if(groupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && + (groupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || groupres->error->code == RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION)) { + rko_fanout->rko_u.admin_request.fanout.outstanding++; + static const struct rd_kafka_admin_worker_cbs cbs = { + rd_kafka_admin_DescribeConsumerGroupsRequest, + rd_kafka_DescribeConsumerGroupsResponse_parse, + }; + rd_kafka_op_t *rko = rd_kafka_admin_request_op_new( + rko_fanout->rko_rk, RD_KAFKA_OP_CONSUMERGROUPDESCRIBE, + RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT, &cbs, &rko_fanout->rko_u.admin_request.options, + rko_fanout->rko_rk->rk_ops); + + rko->rko_u.admin_request.fanout_parent = rko_fanout; + rko->rko_u.admin_request.broker_id = + RD_KAFKA_ADMIN_TARGET_COORDINATOR; + rko->rko_u.admin_request.coordtype = RD_KAFKA_COORD_GROUP; + rko->rko_u.admin_request.coordkey = rd_strdup(grp); + + /* Set the group name as the opaque so the fanout worker use it + * to fill in errors. + * References rko_fanout's memory, which will always outlive + * the fanned out op. */ + rd_kafka_AdminOptions_set_opaque( + &rko->rko_u.admin_request.options, grp); + + rd_list_init(&rko->rko_u.admin_request.args, 1, rd_free); + rd_list_add(&rko->rko_u.admin_request.args, + rd_strdup(grp)); + + rd_kafka_q_enq(rko_fanout->rko_rk->rk_ops, rko); + } + else { + /* As a convenience to the application we insert group result + * in the same order as they were requested. */ + orig_pos = rd_list_index(&rko_fanout->rko_u.admin_request.args, grp, + rd_kafka_DescribeConsumerGroups_cmp); + rd_assert(orig_pos != -1); + + /* Make sure result is not already set */ + rd_assert(rd_list_elem(&rko_fanout->rko_u.admin_request.fanout.results, + orig_pos) == NULL); + + rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results, orig_pos, + newgroupres); + } +} + void rd_kafka_DescribeConsumerGroups(rd_kafka_t *rk, const char **groups, size_t groups_cnt, @@ -8405,14 +8663,14 @@ void rd_kafka_DescribeConsumerGroups(rd_kafka_t *rk, * coordinator into one op. */ for (i = 0; i < groups_cnt; i++) { static const struct rd_kafka_admin_worker_cbs cbs = { - rd_kafka_admin_DescribeConsumerGroupsRequest, - rd_kafka_DescribeConsumerGroupsResponse_parse, + rd_kafka_ConsumerGroupDescribeRequest, + rd_kafka_ConsumerGroupDescribeResponseParse, }; char *grp = rd_list_elem(&rko_fanout->rko_u.admin_request.args, (int)i); rd_kafka_op_t *rko = rd_kafka_admin_request_op_new( - rk, RD_KAFKA_OP_DESCRIBECONSUMERGROUPS, - RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT, &cbs, options, + rk, RD_KAFKA_OP_CONSUMERGROUPDESCRIBE, + RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT, &cbs, options, rk->rk_ops); rko->rko_u.admin_request.fanout_parent = rko_fanout; @@ -8443,7 +8701,7 @@ rd_kafka_DescribeConsumerGroups_result_groups( const rd_kafka_op_t *rko = (const rd_kafka_op_t *)result; rd_kafka_op_type_t reqtype = rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK; - rd_assert(reqtype == RD_KAFKA_OP_DESCRIBECONSUMERGROUPS); + rd_assert(reqtype == RD_KAFKA_OP_DESCRIBECONSUMERGROUPS || reqtype == RD_KAFKA_OP_CONSUMERGROUPDESCRIBE); *cntp = rd_list_cnt(&rko->rko_u.admin_result.results); return (const rd_kafka_ConsumerGroupDescription_t **) diff --git a/src/rdkafka_admin.h b/src/rdkafka_admin.h index ad58fe5cc2..f215501fec 100644 --- a/src/rdkafka_admin.h +++ b/src/rdkafka_admin.h @@ -495,6 +495,8 @@ struct rd_kafka_MemberDescription_s { char *group_instance_id; /**< Group instance id */ char *host; /**< Group member host */ rd_kafka_MemberAssignment_t assignment; /**< Member assignment */ + rd_kafka_MemberAssignment_t + target_assignment; /**< Target assignment */ }; /** @@ -524,6 +526,8 @@ struct rd_kafka_ConsumerGroupDescription_s { rd_kafka_AclOperation_t *authorized_operations; /** Group specific error. */ rd_kafka_error_t *error; + /**< Consumer group type. */ + rd_kafka_consumer_group_type_t type; }; /**@}*/ diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index 60076e835d..6aae7f6a57 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -123,6 +123,8 @@ const char *rd_kafka_op2str(rd_kafka_op_type_t type) { [RD_KAFKA_OP_TERMINATE_TELEMETRY] = "REPLY:RD_KAFKA_OP_TERMINATE_TELEMETRY", [RD_KAFKA_OP_ELECTLEADERS] = "REPLY:ELECTLEADERS", + [RD_KAFKA_OP_CONSUMERGROUPDESCRIBE] = + "REPLY:CONSUMERGROUPDESCRIBE", }; if (type & RD_KAFKA_OP_REPLY) @@ -287,6 +289,8 @@ rd_kafka_op_t *rd_kafka_op_new0(const char *source, rd_kafka_op_type_t type) { sizeof(rko->rko_u.telemetry_broker), [RD_KAFKA_OP_TERMINATE_TELEMETRY] = _RD_KAFKA_OP_EMPTY, [RD_KAFKA_OP_ELECTLEADERS] = sizeof(rko->rko_u.admin_request), + [RD_KAFKA_OP_CONSUMERGROUPDESCRIBE] = + sizeof(rko->rko_u.admin_request), }; size_t tsize = op2size[type & ~RD_KAFKA_OP_FLAGMASK]; @@ -440,7 +444,7 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) { case RD_KAFKA_OP_ALTERUSERSCRAMCREDENTIALS: case RD_KAFKA_OP_DESCRIBEUSERSCRAMCREDENTIALS: case RD_KAFKA_OP_LISTOFFSETS: - case RD_KAFKA_OP_ELECTLEADERS: + case RD_KAFKA_OP_ELECTLEADERS: rd_kafka_replyq_destroy(&rko->rko_u.admin_request.replyq); rd_list_destroy(&rko->rko_u.admin_request.args); if (rko->rko_u.admin_request.options.match_consumer_group_states diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index 3af8a5f395..521cb46484 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -189,6 +189,9 @@ typedef enum { RD_KAFKA_OP_ELECTLEADERS, /**< Admin: * ElectLeaders * u.admin_request */ + RD_KAFKA_OP_CONSUMERGROUPDESCRIBE, /**< Admin: + * ConsumerGroupDescribe + * u.admin_request */ RD_KAFKA_OP__END } rd_kafka_op_type_t; diff --git a/src/rdkafka_proto.h b/src/rdkafka_proto.h index 895e338c83..a1fae6bd6a 100644 --- a/src/rdkafka_proto.h +++ b/src/rdkafka_proto.h @@ -156,6 +156,7 @@ static RD_UNUSED const char *rd_kafka_ApiKey2str(int16_t ApiKey) { "DescribeUserScramCredentialsRequest", [RD_KAFKAP_AlterUserScramCredentials] = "AlterUserScramCredentialsRequest", + [RD_KAFKAP_ConsumerGroupDescribe] = "ConsumerGroupDescribeRequest", [RD_KAFKAP_Vote] = "VoteRequest", [RD_KAFKAP_BeginQuorumEpoch] = "BeginQuorumEpochRequest", [RD_KAFKAP_EndQuorumEpoch] = "EndQuorumEpochRequest", diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index ac04343fac..7ba447b15b 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -6003,6 +6003,65 @@ rd_kafka_resp_err_t rd_kafka_ElectLeadersRequest( return RD_KAFKA_RESP_ERR_NO_ERROR; } + +/** + * @brief Construct and send ConsumerGroupDescribe Request to broker. + * + * @return RD_KAFKA_RESP_ERR_NO_ERROR on success, a new error instance that + * must be released with rd_kafka_error_destroy() in case of error. + */ +rd_kafka_resp_err_t +rd_kafka_ConsumerGroupDescribeRequest(rd_kafka_broker_t *rkb, + const rd_list_t *groups /*(char*)*/, + rd_kafka_AdminOptions_t *options, + char *errstr, + size_t errstr_size, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque) { + rd_kafka_buf_t *rkbuf; + int16_t maxApiVersion = 0; + int16_t ApiVersion = rd_kafka_broker_ApiVersion_supported( + rkb, RD_KAFKAP_ConsumerGroupDescribe, 0, maxApiVersion, NULL); + size_t ofGroupsArrayCnt; + int grp_ids_cnt = rd_list_cnt(groups); + int i, include_authorized_operations; + char *group; + + if (ApiVersion == -1) { + rd_snprintf(errstr, errstr_size, + "Broker does not support ConsumerGroupDescribe"); + return RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE; + } + + include_authorized_operations = + rd_kafka_confval_get_int(&options->include_authorized_operations); + + rkbuf = rd_kafka_buf_new_flexver_request( + rkb, RD_KAFKAP_ConsumerGroupDescribe, 1, + 4 /* rd_kafka_buf_write_arraycnt_pos */ + + 1 /* IncludeAuthorizedOperations */ + 1 /* tags */ + + 32 * grp_ids_cnt /* Groups */, + rd_true /* flexver */); + + ofGroupsArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf); + rd_kafka_buf_finalize_arraycnt(rkbuf, ofGroupsArrayCnt, grp_ids_cnt); + printf("grp_ids_cnt: %d\n", grp_ids_cnt); + printf("include_authorized_operations: %d\n", include_authorized_operations); + RD_LIST_FOREACH(group, groups, i) { + group = rd_list_elem(groups, i); + printf("group: %s\n", group); + rd_kafka_buf_write_str(rkbuf, group, -1); + } + + rd_kafka_buf_write_bool(rkbuf, include_authorized_operations); + rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); + rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque); + printf("rd_kafka_ConsumerGroupDescribeRequest: done\n"); + + return RD_KAFKA_RESP_ERR_NO_ERROR; +} + /** * @brief Parses and handles an InitProducerId reply. * diff --git a/src/rdkafka_request.h b/src/rdkafka_request.h index 932b301acd..ed8693adbc 100644 --- a/src/rdkafka_request.h +++ b/src/rdkafka_request.h @@ -639,6 +639,16 @@ rd_kafka_resp_err_t rd_kafka_ElectLeadersRequest( rd_kafka_resp_cb_t *resp_cb, void *opaque); +rd_kafka_resp_err_t +rd_kafka_ConsumerGroupDescribeRequest(rd_kafka_broker_t *rkb, + const rd_list_t *groups /*(char*)*/, + rd_kafka_AdminOptions_t *options, + char *errstr, + size_t errstr_size, + rd_kafka_replyq_t replyq, + rd_kafka_resp_cb_t *resp_cb, + void *opaque); + void rd_kafkap_leader_discovery_tmpabuf_add_alloc_brokers( rd_tmpabuf_t *tbuf, rd_kafkap_NodeEndpoints_t *NodeEndpoints); From 86272876019b80c2b4c8a8df7fee99c458f7cfab Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Fri, 29 Nov 2024 14:31:33 +0530 Subject: [PATCH 11/19] debugging --- src/rdkafka_admin.c | 80 ++++++++++++++++++++++++++++++++++----------- src/rdkafka_event.c | 4 ++- src/rdkafka_event.h | 1 + src/rdkafka_op.c | 3 +- 4 files changed, 67 insertions(+), 21 deletions(-) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 7d96b6e876..61a6592836 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -7944,6 +7944,7 @@ rd_kafka_ConsumerGroupDescription_new_error(const char *group_id, static rd_kafka_ConsumerGroupDescription_t * rd_kafka_ConsumerGroupDescription_copy( const rd_kafka_ConsumerGroupDescription_t *grpdesc) { + printf("rd_kafka_ConsumerGroupDescription_copy\n"); return rd_kafka_ConsumerGroupDescription_new( grpdesc->group_id, grpdesc->is_simple_consumer_group, &grpdesc->members, grpdesc->partition_assignor, @@ -8396,12 +8397,15 @@ static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( for(int j = 0; j < member_cnt; j++) { rd_kafkap_str_t MemberId, InstanceId, RackId, ClientId, ClientHost, SubscribedTopicNames, SubscribedTopicRegex; - int32_t MemberEpoch; + int32_t MemberEpoch, idx; char *member_id, *instance_id, *rack_id, *client_id, *client_host, *subscribed_topic_names, *subscribed_topic_regex = NULL; rd_kafka_MemberDescription_t *member = NULL; rd_kafka_topic_partition_list_t *assignment = NULL, *target_assignment = NULL; + int8_t are_assignments_present = 0, are_target_assignments_present = 0; + char **subscribed_topic_names_array = NULL; + int32_t subscribed_topic_names_array_cnt; rd_kafka_buf_read_str(reply, &MemberId); rd_kafka_buf_read_str(reply, &InstanceId); @@ -8409,27 +8413,43 @@ static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( rd_kafka_buf_read_i32(reply, &MemberEpoch); rd_kafka_buf_read_str(reply, &ClientId); rd_kafka_buf_read_str(reply, &ClientHost); - rd_kafka_buf_read_str(reply, &SubscribedTopicNames); + rd_kafka_buf_read_arraycnt(reply, &subscribed_topic_names_array_cnt, 100000); + printf("subscribed_topic_names_array_cnt: %d\n", subscribed_topic_names_array_cnt); + subscribed_topic_names_array = rd_calloc(subscribed_topic_names_array_cnt, sizeof(char*)); + for(idx=0; idx < subscribed_topic_names_array_cnt; idx++) { + rd_kafkap_str_t SubscribedTopicName; + rd_kafka_buf_read_str(reply, &SubscribedTopicName); + char *subscribed_topic_name = RD_KAFKAP_STR_DUP(&SubscribedTopicName); + subscribed_topic_names_array[idx] = subscribed_topic_name; + printf("subscribed_topic_name: %s\n", subscribed_topic_name); + } rd_kafka_buf_read_str(reply, &SubscribedTopicRegex); const rd_kafka_topic_partition_field_t fields[] = { RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, - RD_KAFKA_TOPIC_PARTITION_FIELD_END}; - assignment = rd_kafka_buf_read_topic_partitions( - reply, rd_true /* use topic_id */, - rd_true /* use topic name*/, 0, fields); - rd_kafka_buf_skip_tags(reply); - target_assignment = rd_kafka_buf_read_topic_partitions( - reply, rd_true /* use topic_id */, - rd_true /* use topic name*/, 0, fields); - rd_kafka_buf_skip_tags(reply); + // rd_kafka_buf_read_i8(reply, &are_assignments_present); + // printf("are_assignments_present: %d\n", are_assignments_present); + // if(are_assignments_present == 1) { + assignment = rd_kafka_buf_read_topic_partitions( + reply, rd_true /* use topic_id */, + rd_true /* use topic name*/, 0, fields); + rd_kafka_buf_skip_tags(reply); + // } + // rd_kafka_buf_read_i8(reply, &are_target_assignments_present); + // printf("are_target_assignments_present: %d\n", are_target_assignments_present); + // if(are_target_assignments_present == 1) { + + target_assignment = rd_kafka_buf_read_topic_partitions( + reply, rd_true /* use topic_id */, + rd_true /* use topic name*/, 0, fields); + //} + rd_kafka_buf_skip_tags(reply); member_id = RD_KAFKAP_STR_DUP(&MemberId); instance_id = RD_KAFKAP_STR_DUP(&InstanceId); rack_id = RD_KAFKAP_STR_DUP(&RackId); client_id = RD_KAFKAP_STR_DUP(&ClientId); client_host = RD_KAFKAP_STR_DUP(&ClientHost); - subscribed_topic_names = RD_KAFKAP_STR_DUP(&SubscribedTopicNames); subscribed_topic_regex = RD_KAFKAP_STR_DUP(&SubscribedTopicRegex); printf("member_id: %s\n", member_id); @@ -8438,15 +8458,28 @@ static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( printf("member_epoch: %d\n", MemberEpoch); printf("client_id: %s\n", client_id); printf("client_host: %s\n", client_host); - printf("subscribed_topic_names: %s\n", subscribed_topic_names); + printf("subscribed topic names: "); + for(int i=0;ielems[0].topic, assignment->elems[0].partition); - printf("target_assignment: topic: %s partition: %d\n", target_assignment->elems[0].topic, target_assignment->elems[0].partition); + if(assignment) { + printf("assignment: topic: %s partition: %d\n", assignment->elems[0].topic, assignment->elems[0].partition); + } + else { + printf("assignment is null\n"); + } + if(target_assignment) { + printf("target_assignment: topic: %s partition: %d\n", target_assignment->elems[0].topic, target_assignment->elems[0].partition); + } + else { + printf("target_assignment is null\n"); + } member = rd_kafka_MemberDescription_new( client_id, member_id, instance_id, client_host, assignment, target_assignment); rd_list_add(&members, member); - rd_kafka_buf_skip_tags(reply); if(assignment) rd_kafka_topic_partition_list_destroy(assignment); @@ -8526,9 +8559,11 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_kafka_ConsumerGroupDescription_t *newgroupres; const char *grp = rko_partial->rko_u.admin_result.opaque; int orig_pos; - + printf("rd_kafka_DescribeConsumerGroups_response_merge\n"); rd_assert(rko_partial->rko_evtype == RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT || rko_partial->rko_evtype == RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT); + printf("rko_partial->rko_evtype: %d\n", rko_partial->rko_evtype); + printf("processing group: %s\n", grp); if (!rko_partial->rko_err) { /* Proper results. @@ -8538,6 +8573,7 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_assert(groupres); rd_assert(!strcmp(groupres->group_id, grp)); newgroupres = rd_kafka_ConsumerGroupDescription_copy(groupres); + printf("newgroupres->type: %d\n", newgroupres->type); } else { /* Op errored, e.g. timeout */ rd_kafka_error_t *error = @@ -8546,9 +8582,11 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_kafka_ConsumerGroupDescription_new_error(grp, error, RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN); rd_kafka_error_destroy(error); } + printf("newgroupres->type: %d\n", newgroupres->type); - if(groupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && - (groupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || groupres->error->code == RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION)) { + if(newgroupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && + (newgroupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || newgroupres->error->code == RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION)) { + printf("group not found\n"); rko_fanout->rko_u.admin_request.fanout.outstanding++; static const struct rd_kafka_admin_worker_cbs cbs = { rd_kafka_admin_DescribeConsumerGroupsRequest, @@ -8579,6 +8617,7 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_kafka_q_enq(rko_fanout->rko_rk->rk_ops, rko); } else { + printf("group found\n"); /* As a convenience to the application we insert group result * in the same order as they were requested. */ orig_pos = rd_list_index(&rko_fanout->rko_u.admin_request.args, grp, @@ -8591,7 +8630,9 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results, orig_pos, newgroupres); + printf("processed group: %s\n", grp); } + printf("rd_kafka_DescribeConsumerGroups_response_merge end\n"); } void rd_kafka_DescribeConsumerGroups(rd_kafka_t *rk, @@ -8699,6 +8740,7 @@ rd_kafka_DescribeConsumerGroups_result_groups( const rd_kafka_DescribeConsumerGroups_result_t *result, size_t *cntp) { const rd_kafka_op_t *rko = (const rd_kafka_op_t *)result; + printf("rd_kafka_DescribeConsumerGroups_result_groups\n"); rd_kafka_op_type_t reqtype = rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK; rd_assert(reqtype == RD_KAFKA_OP_DESCRIBECONSUMERGROUPS || reqtype == RD_KAFKA_OP_CONSUMERGROUPDESCRIBE); diff --git a/src/rdkafka_event.c b/src/rdkafka_event.c index 7e8cd200ae..41a98f83cf 100644 --- a/src/rdkafka_event.c +++ b/src/rdkafka_event.c @@ -99,6 +99,8 @@ const char *rd_kafka_event_name(const rd_kafka_event_t *rkev) { return "ListOffsetsResult"; case RD_KAFKA_EVENT_ELECTLEADERS_RESULT: return "ElectLeadersResult"; + case RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT: + return "ConsumerGroupDescribeResult"; default: return "?unknown?"; } @@ -382,7 +384,7 @@ rd_kafka_event_ListConsumerGroups_result(rd_kafka_event_t *rkev) { const rd_kafka_DescribeConsumerGroups_result_t * rd_kafka_event_DescribeConsumerGroups_result(rd_kafka_event_t *rkev) { if (!rkev || - rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT) + (rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT && rkev->rko_evtype != RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT)) return NULL; else return (const rd_kafka_DescribeConsumerGroups_result_t *)rkev; diff --git a/src/rdkafka_event.h b/src/rdkafka_event.h index cf63e414eb..7a049602f0 100644 --- a/src/rdkafka_event.h +++ b/src/rdkafka_event.h @@ -118,6 +118,7 @@ static RD_UNUSED RD_INLINE int rd_kafka_event_setup(rd_kafka_t *rk, case RD_KAFKA_EVENT_ALTERUSERSCRAMCREDENTIALS_RESULT: case RD_KAFKA_EVENT_LISTOFFSETS_RESULT: case RD_KAFKA_EVENT_ELECTLEADERS_RESULT: + case RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT: return 1; default: diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index 6aae7f6a57..c8c72a99d7 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -444,7 +444,8 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) { case RD_KAFKA_OP_ALTERUSERSCRAMCREDENTIALS: case RD_KAFKA_OP_DESCRIBEUSERSCRAMCREDENTIALS: case RD_KAFKA_OP_LISTOFFSETS: - case RD_KAFKA_OP_ELECTLEADERS: + case RD_KAFKA_OP_ELECTLEADERS: + case RD_KAFKA_OP_CONSUMERGROUPDESCRIBE: rd_kafka_replyq_destroy(&rko->rko_u.admin_request.replyq); rd_list_destroy(&rko->rko_u.admin_request.args); if (rko->rko_u.admin_request.options.match_consumer_group_states From 589deba40e6ad7c69f00da895b9ad5f018470eeb Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Mon, 2 Dec 2024 18:25:07 +0530 Subject: [PATCH 12/19] API working now --- src/rdkafka_admin.c | 80 ++++++++----------------------------------- src/rdkafka_request.c | 5 +-- 2 files changed, 16 insertions(+), 69 deletions(-) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 61a6592836..99a2de4b02 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -7944,7 +7944,6 @@ rd_kafka_ConsumerGroupDescription_new_error(const char *group_id, static rd_kafka_ConsumerGroupDescription_t * rd_kafka_ConsumerGroupDescription_copy( const rd_kafka_ConsumerGroupDescription_t *grpdesc) { - printf("rd_kafka_ConsumerGroupDescription_copy\n"); return rd_kafka_ConsumerGroupDescription_new( grpdesc->group_id, grpdesc->is_simple_consumer_group, &grpdesc->members, grpdesc->partition_assignor, @@ -8340,7 +8339,6 @@ static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( int32_t nodeid; uint16_t port; int operation_cnt = -1; - printf("rd_kafka_ConsumerGroupDescribeResponseParse\n"); api_version = rd_kafka_buf_ApiVersion(reply); rd_kafka_buf_read_throttle_time(reply); @@ -8380,14 +8378,6 @@ static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( group_state = RD_KAFKAP_STR_DUP(&GroupState); assignor_name = RD_KAFKAP_STR_DUP(&AssignorName); error_str = RD_KAFKAP_STR_DUP(&ErrorString); - printf("error_code: %d\n", error_code); - printf("error string: %s\n", error_str); - printf("group_id: %s\n", group_id); - printf("group_state: %s\n", group_state); - printf("group_epoch: %d\n", group_epoch); - printf("assignment_epoch: %d\n", assignment_epoch); - printf("assignor_name: %s\n", assignor_name); - printf("member_cnt: %d\n", member_cnt); if(error_code) { error = rd_kafka_error_new(error_code, "ConsumerGroupDescribe: %s", error_str); @@ -8414,35 +8404,28 @@ static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( rd_kafka_buf_read_str(reply, &ClientId); rd_kafka_buf_read_str(reply, &ClientHost); rd_kafka_buf_read_arraycnt(reply, &subscribed_topic_names_array_cnt, 100000); - printf("subscribed_topic_names_array_cnt: %d\n", subscribed_topic_names_array_cnt); + subscribed_topic_names_array = rd_calloc(subscribed_topic_names_array_cnt, sizeof(char*)); for(idx=0; idx < subscribed_topic_names_array_cnt; idx++) { rd_kafkap_str_t SubscribedTopicName; rd_kafka_buf_read_str(reply, &SubscribedTopicName); char *subscribed_topic_name = RD_KAFKAP_STR_DUP(&SubscribedTopicName); subscribed_topic_names_array[idx] = subscribed_topic_name; - printf("subscribed_topic_name: %s\n", subscribed_topic_name); } rd_kafka_buf_read_str(reply, &SubscribedTopicRegex); const rd_kafka_topic_partition_field_t fields[] = { RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, RD_KAFKA_TOPIC_PARTITION_FIELD_END}; - // rd_kafka_buf_read_i8(reply, &are_assignments_present); - // printf("are_assignments_present: %d\n", are_assignments_present); - // if(are_assignments_present == 1) { - assignment = rd_kafka_buf_read_topic_partitions( - reply, rd_true /* use topic_id */, - rd_true /* use topic name*/, 0, fields); - rd_kafka_buf_skip_tags(reply); - // } - // rd_kafka_buf_read_i8(reply, &are_target_assignments_present); - // printf("are_target_assignments_present: %d\n", are_target_assignments_present); - // if(are_target_assignments_present == 1) { - - target_assignment = rd_kafka_buf_read_topic_partitions( - reply, rd_true /* use topic_id */, - rd_true /* use topic name*/, 0, fields); - //} + + assignment = rd_kafka_buf_read_topic_partitions( + reply, rd_true /* use topic_id */, + rd_true /* use topic name*/, 0, fields); + rd_kafka_buf_skip_tags(reply); + + target_assignment = rd_kafka_buf_read_topic_partitions( + reply, rd_true /* use topic_id */, + rd_true /* use topic name*/, 0, fields); + rd_kafka_buf_skip_tags(reply); member_id = RD_KAFKAP_STR_DUP(&MemberId); @@ -8452,33 +8435,10 @@ static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( client_host = RD_KAFKAP_STR_DUP(&ClientHost); subscribed_topic_regex = RD_KAFKAP_STR_DUP(&SubscribedTopicRegex); - printf("member_id: %s\n", member_id); - printf("instance_id: %s\n", instance_id); - printf("rack_id: %s\n", rack_id); - printf("member_epoch: %d\n", MemberEpoch); - printf("client_id: %s\n", client_id); - printf("client_host: %s\n", client_host); - printf("subscribed topic names: "); - for(int i=0;ielems[0].topic, assignment->elems[0].partition); - } - else { - printf("assignment is null\n"); - } - if(target_assignment) { - printf("target_assignment: topic: %s partition: %d\n", target_assignment->elems[0].topic, target_assignment->elems[0].partition); - } - else { - printf("target_assignment is null\n"); - } member = rd_kafka_MemberDescription_new( client_id, member_id, instance_id, client_host, assignment, target_assignment); - + + rd_kafka_buf_skip_tags(reply); rd_list_add(&members, member); if(assignment) @@ -8526,7 +8486,6 @@ static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( } rd_kafka_buf_skip_tags(reply); *rko_resultp = rko_result; - printf("rd_kafka_ConsumerGroupDescribeResponseParse end\n"); return RD_KAFKA_RESP_ERR_NO_ERROR; err_parse: if(group_id) @@ -8559,11 +8518,9 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_kafka_ConsumerGroupDescription_t *newgroupres; const char *grp = rko_partial->rko_u.admin_result.opaque; int orig_pos; - printf("rd_kafka_DescribeConsumerGroups_response_merge\n"); + rd_assert(rko_partial->rko_evtype == RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT || rko_partial->rko_evtype == RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT); - printf("rko_partial->rko_evtype: %d\n", rko_partial->rko_evtype); - printf("processing group: %s\n", grp); if (!rko_partial->rko_err) { /* Proper results. @@ -8573,7 +8530,6 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_assert(groupres); rd_assert(!strcmp(groupres->group_id, grp)); newgroupres = rd_kafka_ConsumerGroupDescription_copy(groupres); - printf("newgroupres->type: %d\n", newgroupres->type); } else { /* Op errored, e.g. timeout */ rd_kafka_error_t *error = @@ -8582,11 +8538,9 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_kafka_ConsumerGroupDescription_new_error(grp, error, RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN); rd_kafka_error_destroy(error); } - printf("newgroupres->type: %d\n", newgroupres->type); - if(newgroupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && + if(newgroupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && newgroupres->error && (newgroupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || newgroupres->error->code == RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION)) { - printf("group not found\n"); rko_fanout->rko_u.admin_request.fanout.outstanding++; static const struct rd_kafka_admin_worker_cbs cbs = { rd_kafka_admin_DescribeConsumerGroupsRequest, @@ -8617,7 +8571,6 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_kafka_q_enq(rko_fanout->rko_rk->rk_ops, rko); } else { - printf("group found\n"); /* As a convenience to the application we insert group result * in the same order as they were requested. */ orig_pos = rd_list_index(&rko_fanout->rko_u.admin_request.args, grp, @@ -8630,9 +8583,7 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results, orig_pos, newgroupres); - printf("processed group: %s\n", grp); } - printf("rd_kafka_DescribeConsumerGroups_response_merge end\n"); } void rd_kafka_DescribeConsumerGroups(rd_kafka_t *rk, @@ -8740,7 +8691,6 @@ rd_kafka_DescribeConsumerGroups_result_groups( const rd_kafka_DescribeConsumerGroups_result_t *result, size_t *cntp) { const rd_kafka_op_t *rko = (const rd_kafka_op_t *)result; - printf("rd_kafka_DescribeConsumerGroups_result_groups\n"); rd_kafka_op_type_t reqtype = rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK; rd_assert(reqtype == RD_KAFKA_OP_DESCRIBECONSUMERGROUPS || reqtype == RD_KAFKA_OP_CONSUMERGROUPDESCRIBE); diff --git a/src/rdkafka_request.c b/src/rdkafka_request.c index 7ba447b15b..41398b899e 100644 --- a/src/rdkafka_request.c +++ b/src/rdkafka_request.c @@ -6046,18 +6046,15 @@ rd_kafka_ConsumerGroupDescribeRequest(rd_kafka_broker_t *rkb, ofGroupsArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf); rd_kafka_buf_finalize_arraycnt(rkbuf, ofGroupsArrayCnt, grp_ids_cnt); - printf("grp_ids_cnt: %d\n", grp_ids_cnt); - printf("include_authorized_operations: %d\n", include_authorized_operations); + RD_LIST_FOREACH(group, groups, i) { group = rd_list_elem(groups, i); - printf("group: %s\n", group); rd_kafka_buf_write_str(rkbuf, group, -1); } rd_kafka_buf_write_bool(rkbuf, include_authorized_operations); rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0); rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque); - printf("rd_kafka_ConsumerGroupDescribeRequest: done\n"); return RD_KAFKA_RESP_ERR_NO_ERROR; } From ff5442509466263d089d9cc897b3992b98243d9b Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Tue, 3 Dec 2024 13:28:53 +0530 Subject: [PATCH 13/19] Api working now, tests left --- src/rdkafka_event.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rdkafka_event.c b/src/rdkafka_event.c index 41a98f83cf..eb1330bc02 100644 --- a/src/rdkafka_event.c +++ b/src/rdkafka_event.c @@ -384,7 +384,7 @@ rd_kafka_event_ListConsumerGroups_result(rd_kafka_event_t *rkev) { const rd_kafka_DescribeConsumerGroups_result_t * rd_kafka_event_DescribeConsumerGroups_result(rd_kafka_event_t *rkev) { if (!rkev || - (rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT && rkev->rko_evtype != RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT)) + (rkev->rko_evtype != RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT)) return NULL; else return (const rd_kafka_DescribeConsumerGroups_result_t *)rkev; From e2b0ca95946cf6d5b1354072c72a9809e7cffdc8 Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Wed, 4 Dec 2024 15:11:43 +0530 Subject: [PATCH 14/19] Wrote tests --- tests/0081-admin.c | 201 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 201 insertions(+) diff --git a/tests/0081-admin.c b/tests/0081-admin.c index 9144c400c9..80431be1ee 100644 --- a/tests/0081-admin.c +++ b/tests/0081-admin.c @@ -3137,6 +3137,203 @@ static void do_test_DescribeConsumerGroups(const char *what, SUB_TEST_PASS(); } +static void do_test_DescribeConsumerGroups_Compatibility(const char *what, + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int request_timeout) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options = NULL; + rd_kafka_event_t *rkev = NULL; + rd_kafka_resp_err_t err; + char errstr[512]; + const char *errstr2; +#define TEST_DESCRIBE_CONSUMER_GROUPS_CNT 4 + int i; + const int partitions_cnt = 1; + const int msgs_cnt = 100; + char *topic; + rd_kafka_metadata_topic_t exp_mdtopic = {0}; + int64_t testid = test_id_generate(); + test_timing_t timing; + rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; + const rd_kafka_ConsumerGroupDescription_t **results = NULL; + expected_DescribeConsumerGroups_result_t + expected[TEST_DESCRIBE_CONSUMER_GROUPS_CNT] = RD_ZERO_INIT; + const char *describe_groups[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + char group_instance_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; + char client_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; + rd_kafka_t *rks[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + const rd_kafka_DescribeConsumerGroups_result_t *res; + char *protocols[TEST_DESCRIBE_CONSUMER_GROUPS_CNT] = { + "classic", "classic", "consumer", "consumer"}; + size_t authorized_operation_cnt; + rd_bool_t has_group_instance_id = + test_broker_version >= TEST_BRKVER(2, 4, 0, 0); + + SUB_TEST_QUICK("%s DescribeConsumerGroups Compatibility Test with %s, request_timeout %d", + rd_kafka_name(rk), what, request_timeout); + + q = useq ? useq : rd_kafka_queue_new(rk); + + if (request_timeout != -1) { + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS); + + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + } + + topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + exp_mdtopic.topic = topic; + + /* Create the topics first. */ + test_CreateTopics_simple(rk, NULL, &topic, 1, partitions_cnt, NULL); + + /* Verify that topics are reported by metadata */ + test_wait_metadata_update(rk, &exp_mdtopic, 1, NULL, 0, 15 * 1000); + + /* Produce 100 msgs */ + test_produce_msgs_easy(topic, testid, 0, msgs_cnt); + + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + rd_kafka_conf_t *conf; + char *group_id = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + if (i < 2) { + /* Classic Protocol */ + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "group.protocol", "classic"); + } else { + /* Consumer Protocol */ + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "group.protocol", "consumer"); + } + + snprintf(client_ids[i], sizeof(client_ids[i]), "client_id_%" PRId32, i); + + test_conf_set(conf, "client.id", client_ids[i]); + test_conf_set(conf, "session.timeout.ms", "5000"); + test_conf_set(conf, "auto.offset.reset", "earliest"); + rks[i] = test_create_consumer(group_id, NULL, conf, NULL); + test_consumer_subscribe(rks[i], topic); + /* Consume messages */ + test_consumer_poll("consumer", rks[i], testid, -1, -1, msgs_cnt, NULL); + + expected[i].group_id = group_id; + expected[i].err = RD_KAFKA_RESP_ERR_NO_ERROR; + describe_groups[i] = group_id; + } + + TIMING_START(&timing, "DescribeConsumerGroups"); + TEST_SAY("Call DescribeConsumerGroups\n"); + rd_kafka_DescribeConsumerGroups( + rk, describe_groups, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeConsumerGroups.queue_poll"); + + /* Poll result queue for DescribeConsumerGroups result. */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeConsumerGroups: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + if (rd_kafka_event_error(rkev)) + TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), + rd_kafka_event_error_string(rkev)); + + if (rd_kafka_event_type(rkev) == + RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT) { + break; + } + + rd_kafka_event_destroy(rkev); + } + + /* Convert event to proper result */ + res = rd_kafka_event_DescribeConsumerGroups_result(rkev); + TEST_ASSERT(res, "expected DescribeConsumerGroups_result, got %s", + rd_kafka_event_name(rkev)); + + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(err == exp_err, + "expected DescribeConsumerGroups to return %s, got %s (%s)", + rd_kafka_err2str(exp_err), rd_kafka_err2str(err), + err ? errstr2 : "n/a"); + + TEST_SAY("DescribeConsumerGroups: returned %s (%s)\n", + rd_kafka_err2str(err), err ? errstr2 : "n/a"); + + size_t cnt = 0; + results = rd_kafka_DescribeConsumerGroups_result_groups(res, &cnt); + + TEST_ASSERT( + TEST_DESCRIBE_CONSUMER_GROUPS_CNT == cnt, + "expected DescribeConsumerGroups_result_groups to return %d items, " + "got %" PRIusz, + TEST_DESCRIBE_CONSUMER_GROUPS_CNT, cnt); + + /* Verify results */ + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + expected_DescribeConsumerGroups_result_t *exp = &expected[i]; + rd_kafka_resp_err_t exp_err = exp->err; + const rd_kafka_ConsumerGroupDescription_t *act = results[i]; + rd_kafka_resp_err_t act_err = rd_kafka_error_code( + rd_kafka_ConsumerGroupDescription_error(act)); + rd_kafka_consumer_group_state_t state = + rd_kafka_ConsumerGroupDescription_state(act); + + TEST_ASSERT( + strcmp(exp->group_id, + rd_kafka_ConsumerGroupDescription_group_id(act)) == 0, + "Result order mismatch at #%d: expected group id to be %s, got %s", + i, exp->group_id, + rd_kafka_ConsumerGroupDescription_group_id(act)); + + if (strcmp(protocols[i], "classic") == 0) { + TEST_ASSERT(state == RD_KAFKA_CONSUMER_GROUP_STATE_STABLE || + state == RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY, + "Expected Stable or Empty state for classic protocol, got %s.", + rd_kafka_consumer_group_state_name(state)); + } else if (strcmp(protocols[i], "consumer") == 0) { + TEST_ASSERT(state == RD_KAFKA_CONSUMER_GROUP_STATE_STABLE, + "Expected Stable state, got %s.", + rd_kafka_consumer_group_state_name(state)); + } + + TEST_ASSERT(exp_err == act_err, + "expected err=%d for group %s, got %d (%s)", + exp_err, exp->group_id, act_err, + rd_kafka_err2str(act_err)); + } + + rd_kafka_event_destroy(rkev); + + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + test_consumer_close(rks[i]); + rd_kafka_destroy(rks[i]); + rd_free(expected[i].group_id); + } + + test_DeleteTopics_simple(rk, NULL, &topic, 1, NULL); + rd_free(topic); + + if (options) + rd_kafka_AdminOptions_destroy(options); + + if (!useq) + rd_kafka_queue_destroy(q); + + TEST_LATER_CHECK(); + +#undef TEST_DESCRIBE_CONSUMER_GROUPS_CNT + + SUB_TEST_PASS(); +} + /** @brief Helper function to check whether \p expected and \p actual contain * the same values. */ static void @@ -5274,6 +5471,10 @@ static void do_test_apis(rd_kafka_type_t cltype) { do_test_DescribeConsumerGroups("temp queue", rk, NULL, -1); do_test_DescribeConsumerGroups("main queue", rk, mainq, 1500); } + else { + do_test_DescribeConsumerGroups_Compatibility("temp queue", rk, NULL, -1); + do_test_DescribeConsumerGroups_Compatibility("main queue", rk, mainq, 1500); + } /* Describe topics */ do_test_DescribeTopics("temp queue", rk, NULL, 15000, rd_false); From 1af897d8de7d28ed90fe624cdfc1ac70212d99c6 Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Wed, 4 Dec 2024 15:21:46 +0530 Subject: [PATCH 15/19] style fix --- examples/consumer.c | 7 - examples/describe_consumer_groups.c | 12 +- src/rdkafka.h | 21 +- src/rdkafka_admin.c | 385 +++++++++++++++------------- src/rdkafka_admin.h | 13 +- src/rdkafka_op.c | 5 +- src/rdkafka_proto.h | 2 +- tests/0081-admin.c | 359 +++++++++++++------------- 8 files changed, 418 insertions(+), 386 deletions(-) diff --git a/examples/consumer.c b/examples/consumer.c index 76ebfbd633..dad3efc43b 100644 --- a/examples/consumer.c +++ b/examples/consumer.c @@ -139,13 +139,6 @@ int main(int argc, char **argv) { return 1; } - if(rd_kafka_conf_set(conf, "group.protocol", "consumer", errstr, - sizeof(errstr)) != RD_KAFKA_CONF_OK) { - fprintf(stderr, "%s\n", errstr); - rd_kafka_conf_destroy(conf); - return 1; - } - /* * Create consumer instance. * diff --git a/examples/describe_consumer_groups.c b/examples/describe_consumer_groups.c index d58116cf94..529cee7e69 100644 --- a/examples/describe_consumer_groups.c +++ b/examples/describe_consumer_groups.c @@ -175,16 +175,17 @@ print_group_member_info(const rd_kafka_MemberDescription_t *member) { print_partition_list(stdout, topic_partitions, 0, " "); } const rd_kafka_MemberAssignment_t *target_assignment = - rd_kafka_MemberDescription_target_assignment(member); + rd_kafka_MemberDescription_target_assignment(member); const rd_kafka_topic_partition_list_t *target_topic_partitions = - rd_kafka_MemberAssignment_partitions(target_assignment); + rd_kafka_MemberAssignment_partitions(target_assignment); if (!target_topic_partitions) { printf(" No target assignment\n"); } else if (target_topic_partitions->cnt == 0) { printf(" Empty target assignment\n"); } else { printf(" Target assignment:\n"); - print_partition_list(stdout, target_topic_partitions, 0, " "); + print_partition_list(stdout, target_topic_partitions, 0, + " "); } } @@ -207,7 +208,7 @@ static void print_group_info(const rd_kafka_ConsumerGroupDescription_t *group) { rd_kafka_consumer_group_state_t state = rd_kafka_ConsumerGroupDescription_state(group); rd_kafka_consumer_group_type_t type = - rd_kafka_ConsumerGroupDescription_type(group); + rd_kafka_ConsumerGroupDescription_type(group); authorized_operations = rd_kafka_ConsumerGroupDescription_authorized_operations( group, &authorized_operations_cnt); @@ -228,7 +229,8 @@ static void print_group_info(const rd_kafka_ConsumerGroupDescription_t *group) { printf( "Group \"%s\", partition assignor \"%s\", type \"%s\" " " state %s%s, with %" PRId32 " member(s)\n", - group_id, partition_assignor, rd_kafka_consumer_group_type_name(type), + group_id, partition_assignor, + rd_kafka_consumer_group_type_name(type), rd_kafka_consumer_group_state_name(state), coordinator_desc, member_cnt); for (j = 0; j < authorized_operations_cnt; j++) { diff --git a/src/rdkafka.h b/src/rdkafka.h index 7724c490e8..6f6c7f3dc8 100644 --- a/src/rdkafka.h +++ b/src/rdkafka.h @@ -8886,11 +8886,11 @@ const rd_kafka_Node_t *rd_kafka_ConsumerGroupDescription_coordinator( /** * @brief Gets type for the \p grpdesc group. - * + * * @param grpdesc The group description. - * + * * @return A group type. - * + * * @remark The lifetime of the returned memory is the same * as the lifetime of the \p grpdesc object. */ @@ -9012,11 +9012,11 @@ const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_partitions( /** * @brief Gets target assignment of \p member. - * + * * @param member The group member. - * + * * @return The target assignment. - * + * * @remark The lifetime of the returned memory is the same * as the lifetime of the \p member object. */ @@ -9026,16 +9026,17 @@ const rd_kafka_MemberAssignment_t *rd_kafka_MemberDescription_target_assignment( /** * @brief Gets target assigned partitions of a member \p assignment. - * + * * @param assignment The group member assignment. - * + * * @return The target assigned partitions. - * + * * @remark The lifetime of the returned memory is the same * as the lifetime of the \p assignment object. */ RD_EXPORT -const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_target_partitions( +const rd_kafka_topic_partition_list_t * +rd_kafka_MemberAssignment_target_partitions( const rd_kafka_MemberAssignment_t *assignment); /**@}*/ diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 99a2de4b02..6f7900480c 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -7782,10 +7782,9 @@ static rd_kafka_MemberDescription_t *rd_kafka_MemberDescription_new( */ static rd_kafka_MemberDescription_t * rd_kafka_MemberDescription_copy(const rd_kafka_MemberDescription_t *src) { - return rd_kafka_MemberDescription_new(src->client_id, src->consumer_id, - src->group_instance_id, src->host, - src->assignment.partitions, - src->target_assignment.partitions); + return rd_kafka_MemberDescription_new( + src->client_id, src->consumer_id, src->group_instance_id, src->host, + src->assignment.partitions, src->target_assignment.partitions); } /** @@ -7809,7 +7808,7 @@ rd_kafka_MemberDescription_destroy(rd_kafka_MemberDescription_t *member) { if (member->assignment.partitions) rd_kafka_topic_partition_list_destroy( member->assignment.partitions); - if(member->target_assignment.partitions) + if (member->target_assignment.partitions) rd_kafka_topic_partition_list_destroy( member->target_assignment.partitions); rd_free(member); @@ -7854,7 +7853,8 @@ const rd_kafka_MemberAssignment_t *rd_kafka_MemberDescription_target_assignment( return &member->target_assignment; } -const rd_kafka_topic_partition_list_t *rd_kafka_MemberAssignment_target_partitions( +const rd_kafka_topic_partition_list_t * +rd_kafka_MemberAssignment_target_partitions( const rd_kafka_MemberAssignment_t *assignment) { return assignment->partitions; } @@ -7927,9 +7927,10 @@ rd_kafka_ConsumerGroupDescription_new( * Use rd_kafka_ConsumerGroupDescription_destroy() to free when done. */ static rd_kafka_ConsumerGroupDescription_t * -rd_kafka_ConsumerGroupDescription_new_error(const char *group_id, - rd_kafka_error_t *error, - rd_kafka_consumer_group_type_t type) { +rd_kafka_ConsumerGroupDescription_new_error( + const char *group_id, + rd_kafka_error_t *error, + rd_kafka_consumer_group_type_t type) { return rd_kafka_ConsumerGroupDescription_new( group_id, rd_false, NULL, NULL, NULL, 0, RD_KAFKA_CONSUMER_GROUP_STATE_UNKNOWN, NULL, error, type); @@ -7948,7 +7949,8 @@ rd_kafka_ConsumerGroupDescription_copy( grpdesc->group_id, grpdesc->is_simple_consumer_group, &grpdesc->members, grpdesc->partition_assignor, grpdesc->authorized_operations, grpdesc->authorized_operations_cnt, - grpdesc->state, grpdesc->coordinator, grpdesc->error, grpdesc->type); + grpdesc->state, grpdesc->coordinator, grpdesc->error, + grpdesc->type); } /** @@ -8266,7 +8268,8 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, node, error, RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC); } else grpdesc = rd_kafka_ConsumerGroupDescription_new_error( - group_id, error, RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC); + group_id, error, + RD_KAFKA_CONSUMER_GROUP_TYPE_CLASSIC); rd_list_add(&rko_result->rko_u.admin_result.results, grpdesc); @@ -8320,22 +8323,23 @@ rd_kafka_DescribeConsumerGroupsResponse_parse(rd_kafka_op_t *rko_req, return reply->rkbuf_err; } -static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( - rd_kafka_op_t *rko_req, - rd_kafka_op_t **rko_resultp, - rd_kafka_buf_t *reply, - char *errstr, - size_t errstr_size) { +static rd_kafka_resp_err_t +rd_kafka_ConsumerGroupDescribeResponseParse(rd_kafka_op_t *rko_req, + rd_kafka_op_t **rko_resultp, + rd_kafka_buf_t *reply, + char *errstr, + size_t errstr_size) { const int log_decode_errors = LOG_ERR; int16_t api_version; int32_t cnt; rd_kafka_op_t *rko_result = NULL; rd_kafka_broker_t *rkb = reply->rkbuf_rkb; rd_kafka_error_t *error = NULL; - char *group_id = NULL, *group_state = NULL, *assignor_name = NULL, *error_str = NULL, *host = NULL; + char *group_id = NULL, *group_state = NULL, *assignor_name = NULL, + *error_str = NULL, *host = NULL; int32_t group_epoch, assignment_epoch; rd_kafka_AclOperation_t *operations = NULL; - rd_kafka_Node_t *node = NULL; + rd_kafka_Node_t *node = NULL; int32_t nodeid; uint16_t port; int operation_cnt = -1; @@ -8357,155 +8361,173 @@ static rd_kafka_resp_err_t rd_kafka_ConsumerGroupDescribeResponseParse( rd_list_init(&rko_result->rko_u.admin_result.results, cnt, rd_kafka_ConsumerGroupDescription_free); - for(int i = 0; i < cnt; i++) { - int16_t error_code; - int32_t authorized_operations = -1; - int32_t member_cnt; - rd_kafkap_str_t GroupId, GroupState, AssignorName, ErrorString; - rd_list_t members; - rd_kafka_ConsumerGroupDescription_t *grpdesc = NULL; - - rd_kafka_buf_read_i16(reply, &error_code); - rd_kafka_buf_read_str(reply, &ErrorString); - rd_kafka_buf_read_str(reply, &GroupId); - rd_kafka_buf_read_str(reply, &GroupState); - rd_kafka_buf_read_i32(reply, &group_epoch); - rd_kafka_buf_read_i32(reply, &assignment_epoch); - rd_kafka_buf_read_str(reply, &AssignorName); - rd_kafka_buf_read_arraycnt(reply, &member_cnt, 100000); - - group_id = RD_KAFKAP_STR_DUP(&GroupId); - group_state = RD_KAFKAP_STR_DUP(&GroupState); - assignor_name = RD_KAFKAP_STR_DUP(&AssignorName); - error_str = RD_KAFKAP_STR_DUP(&ErrorString); - - if(error_code) { - error = rd_kafka_error_new(error_code, "ConsumerGroupDescribe: %s", error_str); - } - - rd_list_init(&members, 0, rd_kafka_MemberDescription_free); - - for(int j = 0; j < member_cnt; j++) { - rd_kafkap_str_t MemberId, InstanceId, RackId, ClientId, ClientHost, SubscribedTopicNames, SubscribedTopicRegex; - int32_t MemberEpoch, idx; - char *member_id, *instance_id, *rack_id, - *client_id, *client_host, *subscribed_topic_names, - *subscribed_topic_regex = NULL; - rd_kafka_MemberDescription_t *member = NULL; - rd_kafka_topic_partition_list_t *assignment = NULL, *target_assignment = NULL; - int8_t are_assignments_present = 0, are_target_assignments_present = 0; - char **subscribed_topic_names_array = NULL; - int32_t subscribed_topic_names_array_cnt; - - rd_kafka_buf_read_str(reply, &MemberId); - rd_kafka_buf_read_str(reply, &InstanceId); - rd_kafka_buf_read_str(reply, &RackId); - rd_kafka_buf_read_i32(reply, &MemberEpoch); - rd_kafka_buf_read_str(reply, &ClientId); - rd_kafka_buf_read_str(reply, &ClientHost); - rd_kafka_buf_read_arraycnt(reply, &subscribed_topic_names_array_cnt, 100000); - - subscribed_topic_names_array = rd_calloc(subscribed_topic_names_array_cnt, sizeof(char*)); - for(idx=0; idx < subscribed_topic_names_array_cnt; idx++) { - rd_kafkap_str_t SubscribedTopicName; - rd_kafka_buf_read_str(reply, &SubscribedTopicName); - char *subscribed_topic_name = RD_KAFKAP_STR_DUP(&SubscribedTopicName); - subscribed_topic_names_array[idx] = subscribed_topic_name; + for (int i = 0; i < cnt; i++) { + int16_t error_code; + int32_t authorized_operations = -1; + int32_t member_cnt; + rd_kafkap_str_t GroupId, GroupState, AssignorName, ErrorString; + rd_list_t members; + rd_kafka_ConsumerGroupDescription_t *grpdesc = NULL; + + rd_kafka_buf_read_i16(reply, &error_code); + rd_kafka_buf_read_str(reply, &ErrorString); + rd_kafka_buf_read_str(reply, &GroupId); + rd_kafka_buf_read_str(reply, &GroupState); + rd_kafka_buf_read_i32(reply, &group_epoch); + rd_kafka_buf_read_i32(reply, &assignment_epoch); + rd_kafka_buf_read_str(reply, &AssignorName); + rd_kafka_buf_read_arraycnt(reply, &member_cnt, 100000); + + group_id = RD_KAFKAP_STR_DUP(&GroupId); + group_state = RD_KAFKAP_STR_DUP(&GroupState); + assignor_name = RD_KAFKAP_STR_DUP(&AssignorName); + error_str = RD_KAFKAP_STR_DUP(&ErrorString); + + if (error_code) { + error = rd_kafka_error_new( + error_code, "ConsumerGroupDescribe: %s", error_str); } - rd_kafka_buf_read_str(reply, &SubscribedTopicRegex); - const rd_kafka_topic_partition_field_t fields[] = { - RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, - RD_KAFKA_TOPIC_PARTITION_FIELD_END}; - - assignment = rd_kafka_buf_read_topic_partitions( - reply, rd_true /* use topic_id */, - rd_true /* use topic name*/, 0, fields); - rd_kafka_buf_skip_tags(reply); - - target_assignment = rd_kafka_buf_read_topic_partitions( - reply, rd_true /* use topic_id */, - rd_true /* use topic name*/, 0, fields); - - rd_kafka_buf_skip_tags(reply); - - member_id = RD_KAFKAP_STR_DUP(&MemberId); - instance_id = RD_KAFKAP_STR_DUP(&InstanceId); - rack_id = RD_KAFKAP_STR_DUP(&RackId); - client_id = RD_KAFKAP_STR_DUP(&ClientId); - client_host = RD_KAFKAP_STR_DUP(&ClientHost); - subscribed_topic_regex = RD_KAFKAP_STR_DUP(&SubscribedTopicRegex); - - member = rd_kafka_MemberDescription_new( - client_id, member_id, instance_id, client_host, assignment, target_assignment); - + + rd_list_init(&members, 0, rd_kafka_MemberDescription_free); + + for (int j = 0; j < member_cnt; j++) { + rd_kafkap_str_t MemberId, InstanceId, RackId, ClientId, + ClientHost, SubscribedTopicNames, + SubscribedTopicRegex; + int32_t MemberEpoch, idx; + char *member_id, *instance_id, *rack_id, *client_id, + *client_host, *subscribed_topic_names, + *subscribed_topic_regex = NULL; + rd_kafka_MemberDescription_t *member = NULL; + rd_kafka_topic_partition_list_t *assignment = NULL, + *target_assignment = + NULL; + int8_t are_assignments_present = 0, + are_target_assignments_present = 0; + char **subscribed_topic_names_array = NULL; + int32_t subscribed_topic_names_array_cnt; + + rd_kafka_buf_read_str(reply, &MemberId); + rd_kafka_buf_read_str(reply, &InstanceId); + rd_kafka_buf_read_str(reply, &RackId); + rd_kafka_buf_read_i32(reply, &MemberEpoch); + rd_kafka_buf_read_str(reply, &ClientId); + rd_kafka_buf_read_str(reply, &ClientHost); + rd_kafka_buf_read_arraycnt( + reply, &subscribed_topic_names_array_cnt, 100000); + + subscribed_topic_names_array = rd_calloc( + subscribed_topic_names_array_cnt, sizeof(char *)); + for (idx = 0; idx < subscribed_topic_names_array_cnt; + idx++) { + rd_kafkap_str_t SubscribedTopicName; + rd_kafka_buf_read_str(reply, + &SubscribedTopicName); + char *subscribed_topic_name = + RD_KAFKAP_STR_DUP(&SubscribedTopicName); + subscribed_topic_names_array[idx] = + subscribed_topic_name; + } + rd_kafka_buf_read_str(reply, &SubscribedTopicRegex); + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + + assignment = rd_kafka_buf_read_topic_partitions( + reply, rd_true /* use topic_id */, + rd_true /* use topic name*/, 0, fields); + rd_kafka_buf_skip_tags(reply); + + target_assignment = rd_kafka_buf_read_topic_partitions( + reply, rd_true /* use topic_id */, + rd_true /* use topic name*/, 0, fields); + + rd_kafka_buf_skip_tags(reply); + + member_id = RD_KAFKAP_STR_DUP(&MemberId); + instance_id = RD_KAFKAP_STR_DUP(&InstanceId); + rack_id = RD_KAFKAP_STR_DUP(&RackId); + client_id = RD_KAFKAP_STR_DUP(&ClientId); + client_host = RD_KAFKAP_STR_DUP(&ClientHost); + subscribed_topic_regex = + RD_KAFKAP_STR_DUP(&SubscribedTopicRegex); + + member = rd_kafka_MemberDescription_new( + client_id, member_id, instance_id, client_host, + assignment, target_assignment); + + rd_kafka_buf_skip_tags(reply); + rd_list_add(&members, member); + + if (assignment) + rd_kafka_topic_partition_list_destroy( + assignment); + if (target_assignment) + rd_kafka_topic_partition_list_destroy( + target_assignment); + rd_free(member_id); + rd_free(instance_id); + rd_free(rack_id); + rd_free(client_id); + rd_free(client_host); + rd_free(subscribed_topic_names); + rd_free(subscribed_topic_regex); + } + rd_kafka_buf_read_i32(reply, &authorized_operations); + operations = rd_kafka_AuthorizedOperations_parse( + authorized_operations, &operation_cnt); rd_kafka_buf_skip_tags(reply); - rd_list_add(&members, member); - - if(assignment) - rd_kafka_topic_partition_list_destroy(assignment); - if(target_assignment) - rd_kafka_topic_partition_list_destroy(target_assignment); - rd_free(member_id); - rd_free(instance_id); - rd_free(rack_id); - rd_free(client_id); - rd_free(client_host); - rd_free(subscribed_topic_names); - rd_free(subscribed_topic_regex); - } - rd_kafka_buf_read_i32(reply, &authorized_operations); - operations = rd_kafka_AuthorizedOperations_parse( - authorized_operations, &operation_cnt); - rd_kafka_buf_skip_tags(reply); - - if(error == NULL) { - grpdesc = rd_kafka_ConsumerGroupDescription_new( - group_id, rd_false, &members, assignor_name, operations, operation_cnt, - rd_kafka_consumer_group_state_code(group_state), - node, error, RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER); - } else { - grpdesc = rd_kafka_ConsumerGroupDescription_new_error( - group_id, error , RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER); - } - rd_list_add(&rko_result->rko_u.admin_result.results, grpdesc); - - rd_list_destroy(&members); - rd_free(group_id); - rd_free(group_state); - rd_free(assignor_name); - rd_free(error_str); - RD_IF_FREE(error, rd_kafka_error_destroy); - RD_IF_FREE(operations, rd_free); - - error = NULL; - group_id = NULL; - group_state = NULL; - assignor_name = NULL; - error_str = NULL; - operations = NULL; + + if (error == NULL) { + grpdesc = rd_kafka_ConsumerGroupDescription_new( + group_id, rd_false, &members, assignor_name, + operations, operation_cnt, + rd_kafka_consumer_group_state_code(group_state), + node, error, RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER); + } else { + grpdesc = rd_kafka_ConsumerGroupDescription_new_error( + group_id, error, + RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER); + } + rd_list_add(&rko_result->rko_u.admin_result.results, grpdesc); + + rd_list_destroy(&members); + rd_free(group_id); + rd_free(group_state); + rd_free(assignor_name); + rd_free(error_str); + RD_IF_FREE(error, rd_kafka_error_destroy); + RD_IF_FREE(operations, rd_free); + + error = NULL; + group_id = NULL; + group_state = NULL; + assignor_name = NULL; + error_str = NULL; + operations = NULL; } rd_kafka_buf_skip_tags(reply); *rko_resultp = rko_result; return RD_KAFKA_RESP_ERR_NO_ERROR; err_parse: - if(group_id) - rd_free(group_id); - if(group_state) - rd_free(group_state); - if(assignor_name) - rd_free(assignor_name); - if(error_str) - rd_free(error_str); - if(error) - rd_kafka_error_destroy(error); + if (group_id) + rd_free(group_id); + if (group_state) + rd_free(group_state); + if (assignor_name) + rd_free(assignor_name); + if (error_str) + rd_free(error_str); + if (error) + rd_kafka_error_destroy(error); RD_IF_FREE(operations, rd_free); - if(rko_result) - rd_kafka_op_destroy(rko_result); - rd_snprintf(errstr, errstr_size, - "DescribeConsumerGroups response protocol parse failure: %s", - rd_kafka_err2str(reply->rkbuf_err)); + if (rko_result) + rd_kafka_op_destroy(rko_result); + rd_snprintf( + errstr, errstr_size, + "DescribeConsumerGroups response protocol parse failure: %s", + rd_kafka_err2str(reply->rkbuf_err)); return reply->rkbuf_err; - } /** @brief Merge the DescribeConsumerGroups response from a single broker @@ -8520,7 +8542,9 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( int orig_pos; rd_assert(rko_partial->rko_evtype == - RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT || rko_partial->rko_evtype == RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT); + RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT || + rko_partial->rko_evtype == + RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT); if (!rko_partial->rko_err) { /* Proper results. @@ -8534,13 +8558,16 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( /* Op errored, e.g. timeout */ rd_kafka_error_t *error = rd_kafka_error_new(rko_partial->rko_err, NULL); - newgroupres = - rd_kafka_ConsumerGroupDescription_new_error(grp, error, RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN); + newgroupres = rd_kafka_ConsumerGroupDescription_new_error( + grp, error, RD_KAFKA_CONSUMER_GROUP_TYPE_UNKNOWN); rd_kafka_error_destroy(error); } - if(newgroupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && newgroupres->error && - (newgroupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || newgroupres->error->code == RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION)) { + if (newgroupres->type == RD_KAFKA_CONSUMER_GROUP_TYPE_CONSUMER && + newgroupres->error && + (newgroupres->error->code == RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND || + newgroupres->error->code == + RD_KAFKA_RESP_ERR_UNSUPPORTED_VERSION)) { rko_fanout->rko_u.admin_request.fanout.outstanding++; static const struct rd_kafka_admin_worker_cbs cbs = { rd_kafka_admin_DescribeConsumerGroupsRequest, @@ -8548,7 +8575,8 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( }; rd_kafka_op_t *rko = rd_kafka_admin_request_op_new( rko_fanout->rko_rk, RD_KAFKA_OP_CONSUMERGROUPDESCRIBE, - RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT, &cbs, &rko_fanout->rko_u.admin_request.options, + RD_KAFKA_EVENT_CONSUMERGROUPDESCRIBE_RESULT, &cbs, + &rko_fanout->rko_u.admin_request.options, rko_fanout->rko_rk->rk_ops); rko->rko_u.admin_request.fanout_parent = rko_fanout; @@ -8563,26 +8591,26 @@ static void rd_kafka_DescribeConsumerGroups_response_merge( * the fanned out op. */ rd_kafka_AdminOptions_set_opaque( &rko->rko_u.admin_request.options, grp); - + rd_list_init(&rko->rko_u.admin_request.args, 1, rd_free); - rd_list_add(&rko->rko_u.admin_request.args, - rd_strdup(grp)); + rd_list_add(&rko->rko_u.admin_request.args, rd_strdup(grp)); rd_kafka_q_enq(rko_fanout->rko_rk->rk_ops, rko); - } - else { + } else { /* As a convenience to the application we insert group result - * in the same order as they were requested. */ - orig_pos = rd_list_index(&rko_fanout->rko_u.admin_request.args, grp, - rd_kafka_DescribeConsumerGroups_cmp); + * in the same order as they were requested. */ + orig_pos = + rd_list_index(&rko_fanout->rko_u.admin_request.args, grp, + rd_kafka_DescribeConsumerGroups_cmp); rd_assert(orig_pos != -1); /* Make sure result is not already set */ - rd_assert(rd_list_elem(&rko_fanout->rko_u.admin_request.fanout.results, - orig_pos) == NULL); + rd_assert(rd_list_elem( + &rko_fanout->rko_u.admin_request.fanout.results, + orig_pos) == NULL); - rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results, orig_pos, - newgroupres); + rd_list_set(&rko_fanout->rko_u.admin_request.fanout.results, + orig_pos, newgroupres); } } @@ -8693,7 +8721,8 @@ rd_kafka_DescribeConsumerGroups_result_groups( const rd_kafka_op_t *rko = (const rd_kafka_op_t *)result; rd_kafka_op_type_t reqtype = rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK; - rd_assert(reqtype == RD_KAFKA_OP_DESCRIBECONSUMERGROUPS || reqtype == RD_KAFKA_OP_CONSUMERGROUPDESCRIBE); + rd_assert(reqtype == RD_KAFKA_OP_DESCRIBECONSUMERGROUPS || + reqtype == RD_KAFKA_OP_CONSUMERGROUPDESCRIBE); *cntp = rd_list_cnt(&rko->rko_u.admin_result.results); return (const rd_kafka_ConsumerGroupDescription_t **) diff --git a/src/rdkafka_admin.h b/src/rdkafka_admin.h index f215501fec..4cbbe7c227 100644 --- a/src/rdkafka_admin.h +++ b/src/rdkafka_admin.h @@ -490,13 +490,12 @@ struct rd_kafka_MemberAssignment_s { * */ struct rd_kafka_MemberDescription_s { - char *client_id; /**< Client id */ - char *consumer_id; /**< Consumer id */ - char *group_instance_id; /**< Group instance id */ - char *host; /**< Group member host */ - rd_kafka_MemberAssignment_t assignment; /**< Member assignment */ - rd_kafka_MemberAssignment_t - target_assignment; /**< Target assignment */ + char *client_id; /**< Client id */ + char *consumer_id; /**< Consumer id */ + char *group_instance_id; /**< Group instance id */ + char *host; /**< Group member host */ + rd_kafka_MemberAssignment_t assignment; /**< Member assignment */ + rd_kafka_MemberAssignment_t target_assignment; /**< Target assignment */ }; /** diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index c8c72a99d7..6edb28527b 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -122,9 +122,8 @@ const char *rd_kafka_op2str(rd_kafka_op_type_t type) { "REPLY:RD_KAFKA_OP_SET_TELEMETRY_BROKER", [RD_KAFKA_OP_TERMINATE_TELEMETRY] = "REPLY:RD_KAFKA_OP_TERMINATE_TELEMETRY", - [RD_KAFKA_OP_ELECTLEADERS] = "REPLY:ELECTLEADERS", - [RD_KAFKA_OP_CONSUMERGROUPDESCRIBE] = - "REPLY:CONSUMERGROUPDESCRIBE", + [RD_KAFKA_OP_ELECTLEADERS] = "REPLY:ELECTLEADERS", + [RD_KAFKA_OP_CONSUMERGROUPDESCRIBE] = "REPLY:CONSUMERGROUPDESCRIBE", }; if (type & RD_KAFKA_OP_REPLY) diff --git a/src/rdkafka_proto.h b/src/rdkafka_proto.h index a1fae6bd6a..161532e8fa 100644 --- a/src/rdkafka_proto.h +++ b/src/rdkafka_proto.h @@ -156,7 +156,7 @@ static RD_UNUSED const char *rd_kafka_ApiKey2str(int16_t ApiKey) { "DescribeUserScramCredentialsRequest", [RD_KAFKAP_AlterUserScramCredentials] = "AlterUserScramCredentialsRequest", - [RD_KAFKAP_ConsumerGroupDescribe] = "ConsumerGroupDescribeRequest", + [RD_KAFKAP_ConsumerGroupDescribe] = "ConsumerGroupDescribeRequest", [RD_KAFKAP_Vote] = "VoteRequest", [RD_KAFKAP_BeginQuorumEpoch] = "BeginQuorumEpochRequest", [RD_KAFKAP_EndQuorumEpoch] = "EndQuorumEpochRequest", diff --git a/tests/0081-admin.c b/tests/0081-admin.c index 80431be1ee..493d5b4a7a 100644 --- a/tests/0081-admin.c +++ b/tests/0081-admin.c @@ -3138,200 +3138,208 @@ static void do_test_DescribeConsumerGroups(const char *what, } static void do_test_DescribeConsumerGroups_Compatibility(const char *what, - rd_kafka_t *rk, - rd_kafka_queue_t *useq, - int request_timeout) { - rd_kafka_queue_t *q; - rd_kafka_AdminOptions_t *options = NULL; - rd_kafka_event_t *rkev = NULL; - rd_kafka_resp_err_t err; - char errstr[512]; - const char *errstr2; + rd_kafka_t *rk, + rd_kafka_queue_t *useq, + int request_timeout) { + rd_kafka_queue_t *q; + rd_kafka_AdminOptions_t *options = NULL; + rd_kafka_event_t *rkev = NULL; + rd_kafka_resp_err_t err; + char errstr[512]; + const char *errstr2; #define TEST_DESCRIBE_CONSUMER_GROUPS_CNT 4 - int i; - const int partitions_cnt = 1; - const int msgs_cnt = 100; - char *topic; - rd_kafka_metadata_topic_t exp_mdtopic = {0}; - int64_t testid = test_id_generate(); - test_timing_t timing; - rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; - const rd_kafka_ConsumerGroupDescription_t **results = NULL; - expected_DescribeConsumerGroups_result_t - expected[TEST_DESCRIBE_CONSUMER_GROUPS_CNT] = RD_ZERO_INIT; - const char *describe_groups[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; - char group_instance_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; - char client_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; - rd_kafka_t *rks[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; - const rd_kafka_DescribeConsumerGroups_result_t *res; - char *protocols[TEST_DESCRIBE_CONSUMER_GROUPS_CNT] = { - "classic", "classic", "consumer", "consumer"}; - size_t authorized_operation_cnt; - rd_bool_t has_group_instance_id = - test_broker_version >= TEST_BRKVER(2, 4, 0, 0); - - SUB_TEST_QUICK("%s DescribeConsumerGroups Compatibility Test with %s, request_timeout %d", - rd_kafka_name(rk), what, request_timeout); - - q = useq ? useq : rd_kafka_queue_new(rk); - - if (request_timeout != -1) { - options = rd_kafka_AdminOptions_new( - rk, RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS); + int i; + const int partitions_cnt = 1; + const int msgs_cnt = 100; + char *topic; + rd_kafka_metadata_topic_t exp_mdtopic = {0}; + int64_t testid = test_id_generate(); + test_timing_t timing; + rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR; + const rd_kafka_ConsumerGroupDescription_t **results = NULL; + expected_DescribeConsumerGroups_result_t + expected[TEST_DESCRIBE_CONSUMER_GROUPS_CNT] = RD_ZERO_INIT; + const char *describe_groups[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + char group_instance_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; + char client_ids[TEST_DESCRIBE_CONSUMER_GROUPS_CNT][512]; + rd_kafka_t *rks[TEST_DESCRIBE_CONSUMER_GROUPS_CNT]; + const rd_kafka_DescribeConsumerGroups_result_t *res; + char *protocols[TEST_DESCRIBE_CONSUMER_GROUPS_CNT] = { + "classic", "classic", "consumer", "consumer"}; + size_t authorized_operation_cnt; + rd_bool_t has_group_instance_id = + test_broker_version >= TEST_BRKVER(2, 4, 0, 0); - err = rd_kafka_AdminOptions_set_request_timeout( - options, request_timeout, errstr, sizeof(errstr)); - TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); - } + SUB_TEST_QUICK( + "%s DescribeConsumerGroups Compatibility Test with %s, " + "request_timeout %d", + rd_kafka_name(rk), what, request_timeout); + + q = useq ? useq : rd_kafka_queue_new(rk); + + if (request_timeout != -1) { + options = rd_kafka_AdminOptions_new( + rk, RD_KAFKA_ADMIN_OP_DESCRIBECONSUMERGROUPS); + + err = rd_kafka_AdminOptions_set_request_timeout( + options, request_timeout, errstr, sizeof(errstr)); + TEST_ASSERT(!err, "%s", rd_kafka_err2str(err)); + } - topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); - exp_mdtopic.topic = topic; + topic = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + exp_mdtopic.topic = topic; - /* Create the topics first. */ - test_CreateTopics_simple(rk, NULL, &topic, 1, partitions_cnt, NULL); + /* Create the topics first. */ + test_CreateTopics_simple(rk, NULL, &topic, 1, partitions_cnt, NULL); - /* Verify that topics are reported by metadata */ - test_wait_metadata_update(rk, &exp_mdtopic, 1, NULL, 0, 15 * 1000); + /* Verify that topics are reported by metadata */ + test_wait_metadata_update(rk, &exp_mdtopic, 1, NULL, 0, 15 * 1000); - /* Produce 100 msgs */ - test_produce_msgs_easy(topic, testid, 0, msgs_cnt); + /* Produce 100 msgs */ + test_produce_msgs_easy(topic, testid, 0, msgs_cnt); - for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { - rd_kafka_conf_t *conf; - char *group_id = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); - if (i < 2) { - /* Classic Protocol */ - test_conf_init(&conf, NULL, 0); - test_conf_set(conf, "group.protocol", "classic"); - } else { - /* Consumer Protocol */ - test_conf_init(&conf, NULL, 0); - test_conf_set(conf, "group.protocol", "consumer"); + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + rd_kafka_conf_t *conf; + char *group_id = rd_strdup(test_mk_topic_name(__FUNCTION__, 1)); + if (i < 2) { + /* Classic Protocol */ + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "group.protocol", "classic"); + } else { + /* Consumer Protocol */ + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "group.protocol", "consumer"); + } + + snprintf(client_ids[i], sizeof(client_ids[i]), + "client_id_%" PRId32, i); + + test_conf_set(conf, "client.id", client_ids[i]); + test_conf_set(conf, "session.timeout.ms", "5000"); + test_conf_set(conf, "auto.offset.reset", "earliest"); + rks[i] = test_create_consumer(group_id, NULL, conf, NULL); + test_consumer_subscribe(rks[i], topic); + /* Consume messages */ + test_consumer_poll("consumer", rks[i], testid, -1, -1, msgs_cnt, + NULL); + + expected[i].group_id = group_id; + expected[i].err = RD_KAFKA_RESP_ERR_NO_ERROR; + describe_groups[i] = group_id; } - snprintf(client_ids[i], sizeof(client_ids[i]), "client_id_%" PRId32, i); - - test_conf_set(conf, "client.id", client_ids[i]); - test_conf_set(conf, "session.timeout.ms", "5000"); - test_conf_set(conf, "auto.offset.reset", "earliest"); - rks[i] = test_create_consumer(group_id, NULL, conf, NULL); - test_consumer_subscribe(rks[i], topic); - /* Consume messages */ - test_consumer_poll("consumer", rks[i], testid, -1, -1, msgs_cnt, NULL); - - expected[i].group_id = group_id; - expected[i].err = RD_KAFKA_RESP_ERR_NO_ERROR; - describe_groups[i] = group_id; - } - - TIMING_START(&timing, "DescribeConsumerGroups"); - TEST_SAY("Call DescribeConsumerGroups\n"); - rd_kafka_DescribeConsumerGroups( - rk, describe_groups, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, options, q); - TIMING_ASSERT_LATER(&timing, 0, 50); - - TIMING_START(&timing, "DescribeConsumerGroups.queue_poll"); - - /* Poll result queue for DescribeConsumerGroups result. */ - while (1) { - rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); - TEST_SAY("DescribeConsumerGroups: got %s in %.3fms\n", - rd_kafka_event_name(rkev), - TIMING_DURATION(&timing) / 1000.0f); - if (rkev == NULL) - continue; - if (rd_kafka_event_error(rkev)) - TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), - rd_kafka_event_error_string(rkev)); - - if (rd_kafka_event_type(rkev) == - RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT) { - break; + TIMING_START(&timing, "DescribeConsumerGroups"); + TEST_SAY("Call DescribeConsumerGroups\n"); + rd_kafka_DescribeConsumerGroups( + rk, describe_groups, TEST_DESCRIBE_CONSUMER_GROUPS_CNT, options, q); + TIMING_ASSERT_LATER(&timing, 0, 50); + + TIMING_START(&timing, "DescribeConsumerGroups.queue_poll"); + + /* Poll result queue for DescribeConsumerGroups result. */ + while (1) { + rkev = rd_kafka_queue_poll(q, tmout_multip(20 * 1000)); + TEST_SAY("DescribeConsumerGroups: got %s in %.3fms\n", + rd_kafka_event_name(rkev), + TIMING_DURATION(&timing) / 1000.0f); + if (rkev == NULL) + continue; + if (rd_kafka_event_error(rkev)) + TEST_SAY("%s: %s\n", rd_kafka_event_name(rkev), + rd_kafka_event_error_string(rkev)); + + if (rd_kafka_event_type(rkev) == + RD_KAFKA_EVENT_DESCRIBECONSUMERGROUPS_RESULT) { + break; + } + + rd_kafka_event_destroy(rkev); } - rd_kafka_event_destroy(rkev); - } - - /* Convert event to proper result */ - res = rd_kafka_event_DescribeConsumerGroups_result(rkev); - TEST_ASSERT(res, "expected DescribeConsumerGroups_result, got %s", - rd_kafka_event_name(rkev)); - - err = rd_kafka_event_error(rkev); - errstr2 = rd_kafka_event_error_string(rkev); - TEST_ASSERT(err == exp_err, - "expected DescribeConsumerGroups to return %s, got %s (%s)", - rd_kafka_err2str(exp_err), rd_kafka_err2str(err), - err ? errstr2 : "n/a"); - - TEST_SAY("DescribeConsumerGroups: returned %s (%s)\n", - rd_kafka_err2str(err), err ? errstr2 : "n/a"); - - size_t cnt = 0; - results = rd_kafka_DescribeConsumerGroups_result_groups(res, &cnt); - - TEST_ASSERT( - TEST_DESCRIBE_CONSUMER_GROUPS_CNT == cnt, - "expected DescribeConsumerGroups_result_groups to return %d items, " - "got %" PRIusz, - TEST_DESCRIBE_CONSUMER_GROUPS_CNT, cnt); - - /* Verify results */ - for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { - expected_DescribeConsumerGroups_result_t *exp = &expected[i]; - rd_kafka_resp_err_t exp_err = exp->err; - const rd_kafka_ConsumerGroupDescription_t *act = results[i]; - rd_kafka_resp_err_t act_err = rd_kafka_error_code( - rd_kafka_ConsumerGroupDescription_error(act)); - rd_kafka_consumer_group_state_t state = - rd_kafka_ConsumerGroupDescription_state(act); + /* Convert event to proper result */ + res = rd_kafka_event_DescribeConsumerGroups_result(rkev); + TEST_ASSERT(res, "expected DescribeConsumerGroups_result, got %s", + rd_kafka_event_name(rkev)); + + err = rd_kafka_event_error(rkev); + errstr2 = rd_kafka_event_error_string(rkev); + TEST_ASSERT(err == exp_err, + "expected DescribeConsumerGroups to return %s, got %s (%s)", + rd_kafka_err2str(exp_err), rd_kafka_err2str(err), + err ? errstr2 : "n/a"); + + TEST_SAY("DescribeConsumerGroups: returned %s (%s)\n", + rd_kafka_err2str(err), err ? errstr2 : "n/a"); + + size_t cnt = 0; + results = rd_kafka_DescribeConsumerGroups_result_groups(res, &cnt); TEST_ASSERT( - strcmp(exp->group_id, - rd_kafka_ConsumerGroupDescription_group_id(act)) == 0, - "Result order mismatch at #%d: expected group id to be %s, got %s", - i, exp->group_id, - rd_kafka_ConsumerGroupDescription_group_id(act)); - - if (strcmp(protocols[i], "classic") == 0) { - TEST_ASSERT(state == RD_KAFKA_CONSUMER_GROUP_STATE_STABLE || - state == RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY, - "Expected Stable or Empty state for classic protocol, got %s.", - rd_kafka_consumer_group_state_name(state)); - } else if (strcmp(protocols[i], "consumer") == 0) { - TEST_ASSERT(state == RD_KAFKA_CONSUMER_GROUP_STATE_STABLE, - "Expected Stable state, got %s.", - rd_kafka_consumer_group_state_name(state)); - } + TEST_DESCRIBE_CONSUMER_GROUPS_CNT == cnt, + "expected DescribeConsumerGroups_result_groups to return %d items, " + "got %" PRIusz, + TEST_DESCRIBE_CONSUMER_GROUPS_CNT, cnt); + + /* Verify results */ + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + expected_DescribeConsumerGroups_result_t *exp = &expected[i]; + rd_kafka_resp_err_t exp_err = exp->err; + const rd_kafka_ConsumerGroupDescription_t *act = results[i]; + rd_kafka_resp_err_t act_err = rd_kafka_error_code( + rd_kafka_ConsumerGroupDescription_error(act)); + rd_kafka_consumer_group_state_t state = + rd_kafka_ConsumerGroupDescription_state(act); + + TEST_ASSERT(strcmp(exp->group_id, + rd_kafka_ConsumerGroupDescription_group_id( + act)) == 0, + "Result order mismatch at #%d: expected group id " + "to be %s, got %s", + i, exp->group_id, + rd_kafka_ConsumerGroupDescription_group_id(act)); - TEST_ASSERT(exp_err == act_err, - "expected err=%d for group %s, got %d (%s)", - exp_err, exp->group_id, act_err, - rd_kafka_err2str(act_err)); - } + if (strcmp(protocols[i], "classic") == 0) { + TEST_ASSERT( + state == RD_KAFKA_CONSUMER_GROUP_STATE_STABLE || + state == RD_KAFKA_CONSUMER_GROUP_STATE_EMPTY, + "Expected Stable or Empty state for classic " + "protocol, got %s.", + rd_kafka_consumer_group_state_name(state)); + } else if (strcmp(protocols[i], "consumer") == 0) { + TEST_ASSERT(state == + RD_KAFKA_CONSUMER_GROUP_STATE_STABLE, + "Expected Stable state, got %s.", + rd_kafka_consumer_group_state_name(state)); + } + + TEST_ASSERT(exp_err == act_err, + "expected err=%d for group %s, got %d (%s)", + exp_err, exp->group_id, act_err, + rd_kafka_err2str(act_err)); + } - rd_kafka_event_destroy(rkev); + rd_kafka_event_destroy(rkev); - for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { - test_consumer_close(rks[i]); - rd_kafka_destroy(rks[i]); - rd_free(expected[i].group_id); - } + for (i = 0; i < TEST_DESCRIBE_CONSUMER_GROUPS_CNT; i++) { + test_consumer_close(rks[i]); + rd_kafka_destroy(rks[i]); + rd_free(expected[i].group_id); + } - test_DeleteTopics_simple(rk, NULL, &topic, 1, NULL); - rd_free(topic); + test_DeleteTopics_simple(rk, NULL, &topic, 1, NULL); + rd_free(topic); - if (options) - rd_kafka_AdminOptions_destroy(options); + if (options) + rd_kafka_AdminOptions_destroy(options); - if (!useq) - rd_kafka_queue_destroy(q); + if (!useq) + rd_kafka_queue_destroy(q); - TEST_LATER_CHECK(); + TEST_LATER_CHECK(); #undef TEST_DESCRIBE_CONSUMER_GROUPS_CNT - SUB_TEST_PASS(); + SUB_TEST_PASS(); } /** @brief Helper function to check whether \p expected and \p actual contain @@ -5470,10 +5478,11 @@ static void do_test_apis(rd_kafka_type_t cltype) { /* Describe groups */ do_test_DescribeConsumerGroups("temp queue", rk, NULL, -1); do_test_DescribeConsumerGroups("main queue", rk, mainq, 1500); - } - else { - do_test_DescribeConsumerGroups_Compatibility("temp queue", rk, NULL, -1); - do_test_DescribeConsumerGroups_Compatibility("main queue", rk, mainq, 1500); + } else { + do_test_DescribeConsumerGroups_Compatibility("temp queue", rk, + NULL, -1); + do_test_DescribeConsumerGroups_Compatibility("main queue", rk, + mainq, 1500); } /* Describe topics */ From 261199106c1af7ca96ba3d6b214dbb80d9b46541 Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Wed, 4 Dec 2024 15:34:27 +0530 Subject: [PATCH 16/19] style fix --- src/rdkafka_op.h | 14 +++++++------- src/rdkafka_proto.h | 34 +++++++++++++++++----------------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index 521cb46484..a735d94bc3 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -182,13 +182,13 @@ typedef enum { u.admin_request >*/ RD_KAFKA_OP_LISTOFFSETS, /**< Admin: ListOffsets u.admin_request >*/ RD_KAFKA_OP_METADATA_UPDATE, /**< Metadata update (KIP 951) **/ - RD_KAFKA_OP_SET_TELEMETRY_BROKER, /**< Set preferred broker for - telemetry. */ - RD_KAFKA_OP_TERMINATE_TELEMETRY, /**< Start termination sequence for - telemetry. */ - RD_KAFKA_OP_ELECTLEADERS, /**< Admin: - * ElectLeaders - * u.admin_request */ + RD_KAFKA_OP_SET_TELEMETRY_BROKER, /**< Set preferred broker for + telemetry. */ + RD_KAFKA_OP_TERMINATE_TELEMETRY, /**< Start termination sequence for + telemetry. */ + RD_KAFKA_OP_ELECTLEADERS, /**< Admin: + * ElectLeaders + * u.admin_request */ RD_KAFKA_OP_CONSUMERGROUPDESCRIBE, /**< Admin: * ConsumerGroupDescribe * u.admin_request */ diff --git a/src/rdkafka_proto.h b/src/rdkafka_proto.h index 161532e8fa..96637ab466 100644 --- a/src/rdkafka_proto.h +++ b/src/rdkafka_proto.h @@ -156,23 +156,23 @@ static RD_UNUSED const char *rd_kafka_ApiKey2str(int16_t ApiKey) { "DescribeUserScramCredentialsRequest", [RD_KAFKAP_AlterUserScramCredentials] = "AlterUserScramCredentialsRequest", - [RD_KAFKAP_ConsumerGroupDescribe] = "ConsumerGroupDescribeRequest", - [RD_KAFKAP_Vote] = "VoteRequest", - [RD_KAFKAP_BeginQuorumEpoch] = "BeginQuorumEpochRequest", - [RD_KAFKAP_EndQuorumEpoch] = "EndQuorumEpochRequest", - [RD_KAFKAP_DescribeQuorum] = "DescribeQuorumRequest", - [RD_KAFKAP_AlterIsr] = "AlterIsrRequest", - [RD_KAFKAP_UpdateFeatures] = "UpdateFeaturesRequest", - [RD_KAFKAP_Envelope] = "EnvelopeRequest", - [RD_KAFKAP_FetchSnapshot] = "FetchSnapshot", - [RD_KAFKAP_DescribeCluster] = "DescribeCluster", - [RD_KAFKAP_DescribeProducers] = "DescribeProducers", - [RD_KAFKAP_BrokerHeartbeat] = "BrokerHeartbeat", - [RD_KAFKAP_UnregisterBroker] = "UnregisterBroker", - [RD_KAFKAP_DescribeTransactions] = "DescribeTransactions", - [RD_KAFKAP_ListTransactions] = "ListTransactions", - [RD_KAFKAP_AllocateProducerIds] = "AllocateProducerIds", - [RD_KAFKAP_ConsumerGroupHeartbeat] = "ConsumerGroupHeartbeat", + [RD_KAFKAP_ConsumerGroupDescribe] = "ConsumerGroupDescribeRequest", + [RD_KAFKAP_Vote] = "VoteRequest", + [RD_KAFKAP_BeginQuorumEpoch] = "BeginQuorumEpochRequest", + [RD_KAFKAP_EndQuorumEpoch] = "EndQuorumEpochRequest", + [RD_KAFKAP_DescribeQuorum] = "DescribeQuorumRequest", + [RD_KAFKAP_AlterIsr] = "AlterIsrRequest", + [RD_KAFKAP_UpdateFeatures] = "UpdateFeaturesRequest", + [RD_KAFKAP_Envelope] = "EnvelopeRequest", + [RD_KAFKAP_FetchSnapshot] = "FetchSnapshot", + [RD_KAFKAP_DescribeCluster] = "DescribeCluster", + [RD_KAFKAP_DescribeProducers] = "DescribeProducers", + [RD_KAFKAP_BrokerHeartbeat] = "BrokerHeartbeat", + [RD_KAFKAP_UnregisterBroker] = "UnregisterBroker", + [RD_KAFKAP_DescribeTransactions] = "DescribeTransactions", + [RD_KAFKAP_ListTransactions] = "ListTransactions", + [RD_KAFKAP_AllocateProducerIds] = "AllocateProducerIds", + [RD_KAFKAP_ConsumerGroupHeartbeat] = "ConsumerGroupHeartbeat", [RD_KAFKAP_GetTelemetrySubscriptions] = "GetTelemetrySubscriptions", [RD_KAFKAP_PushTelemetry] = "PushTelemetry", From ecf6ed58b149f7a41835681e2fab40b941e28f2b Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Wed, 4 Dec 2024 15:39:50 +0530 Subject: [PATCH 17/19] removing unused variables --- src/rdkafka_admin.c | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 6f7900480c..9512c49c64 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -8402,8 +8402,6 @@ rd_kafka_ConsumerGroupDescribeResponseParse(rd_kafka_op_t *rko_req, rd_kafka_topic_partition_list_t *assignment = NULL, *target_assignment = NULL; - int8_t are_assignments_present = 0, - are_target_assignments_present = 0; char **subscribed_topic_names_array = NULL; int32_t subscribed_topic_names_array_cnt; From c86b24444c2dbebfafb47b70916089dee3def7b8 Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Wed, 4 Dec 2024 16:49:11 +0530 Subject: [PATCH 18/19] trying to fix build --- src/rdkafka_admin.c | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/rdkafka_admin.c b/src/rdkafka_admin.c index 9512c49c64..5bffc30a72 100644 --- a/src/rdkafka_admin.c +++ b/src/rdkafka_admin.c @@ -8343,6 +8343,7 @@ rd_kafka_ConsumerGroupDescribeResponseParse(rd_kafka_op_t *rko_req, int32_t nodeid; uint16_t port; int operation_cnt = -1; + int32_t i; api_version = rd_kafka_buf_ApiVersion(reply); rd_kafka_buf_read_throttle_time(reply); @@ -8361,10 +8362,10 @@ rd_kafka_ConsumerGroupDescribeResponseParse(rd_kafka_op_t *rko_req, rd_list_init(&rko_result->rko_u.admin_result.results, cnt, rd_kafka_ConsumerGroupDescription_free); - for (int i = 0; i < cnt; i++) { + for (i = 0; i < cnt; i++) { int16_t error_code; int32_t authorized_operations = -1; - int32_t member_cnt; + int32_t member_cnt, j; rd_kafkap_str_t GroupId, GroupState, AssignorName, ErrorString; rd_list_t members; rd_kafka_ConsumerGroupDescription_t *grpdesc = NULL; @@ -8390,7 +8391,7 @@ rd_kafka_ConsumerGroupDescribeResponseParse(rd_kafka_op_t *rko_req, rd_list_init(&members, 0, rd_kafka_MemberDescription_free); - for (int j = 0; j < member_cnt; j++) { + for (j = 0; j < member_cnt; j++) { rd_kafkap_str_t MemberId, InstanceId, RackId, ClientId, ClientHost, SubscribedTopicNames, SubscribedTopicRegex; @@ -8402,7 +8403,7 @@ rd_kafka_ConsumerGroupDescribeResponseParse(rd_kafka_op_t *rko_req, rd_kafka_topic_partition_list_t *assignment = NULL, *target_assignment = NULL; - char **subscribed_topic_names_array = NULL; + char **subscribed_topic_names_array = NULL; int32_t subscribed_topic_names_array_cnt; rd_kafka_buf_read_str(reply, &MemberId); From 385e6ab61dfce02701d29b64d5fb78fd1932eb1d Mon Sep 17 00:00:00 2001 From: PratRanj07 Date: Tue, 17 Dec 2024 11:21:25 +0530 Subject: [PATCH 19/19] Removing trivup file --- tests/trivup/trivup-0.12.7.tar.gz | Bin 33994 -> 0 bytes 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 tests/trivup/trivup-0.12.7.tar.gz diff --git a/tests/trivup/trivup-0.12.7.tar.gz b/tests/trivup/trivup-0.12.7.tar.gz deleted file mode 100644 index 657d58aab1e0d329d417b963289411ab9a95e861..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 33994 zcmZ6SQ*foh_V#1j=1k0qZF^#KV%s~mZQItwwly&(w!QZ^=bZ2A-xsT@tKXZ|)zxo5 zYyBS5XgD}DLtG08kgKzmyPG2;3lr-PCN3sdV`s2Szc0YX_<|>Z5ipYWEuIV-|1&Kl z?IXn~$)^FWA14?*uX2<{`3C_zMtIg z`(~um#{fH3*+=;9U)96>n!eVJS;bf4{56}+r+3ozE7INpi?g>d#2L`O&_nZA)Qqo< zIp;-v+wNu0j-Tr8s^Qx6`Q~P*(BFIjDE>xgZAzlGY^bAeuHntxr&GZ**q?OKg&5cDkn2JzHHbpd!+*7cx-$OY%K{Ivunt z4s?)v>+5U(GWT%sxoI_+k^liJK7}sZ_@usO9aumw9v;nqLC@FE^a-HN-%rF^kk4(# zz?*>&p@xn291y7RzVC|RTJPdu(W2BT({AnjL;dwcA=K{&)3IE<8`bgWV$+rAR2x3_ zV}^})nU9^~qiCUpA_JR@#ca%52%|u_Wv*T>>QwYp3|2v;zOv}T65l&jtAptwHhR#yw}d&i7S3S zB~j1H^eZ5{}jKAEZFW%4701`JtpLe$_smWrDS_sIJ7ZF zE%elNBZRabemE4YSU;yD1$+BwIe*bBKuk0gY*LNN5e@CT!22HQ=FflT_U`AtWxB0I z9j*@sGw)MdEYiMI-*Bz+ar;={^VmBG2FAxr(lB&gehn6d97n%yNv<7B+-I84+w%;YH4df*ROsfJu^~4susGIEEZp?;+@M;JR*(+6ZBQ=Ie;u;BAo$HR< z=jwkcY@ySWPX}S%_W)8XMC)cCY7uPR5!;ouk%ll`6ufd1B%c)y2cB&L3;4K+UDrPj z4_BHhA7vu%$W zw_7pX1)Nah%MbklYVoFL*wKkv&JXvRyIPDL}n-l?qsa@sXx<>Hw3m|H;hAVH|u9X_(<%Kb6ol@YK(d(gB^$>0TIDNz2NSajTQBD`3>gaDRs&8vz|7)y-AT0k z`F;ZD39?<{qljL9BdJlZb#hX_e%OJ8d*jr23^MBWO(DPd$fjb}er>-E>t)=|Z81k! zuXv;(w_w1$i_U%SmL6ppJoVpy!no|0pS>A;zAt70@F7E z&L2wlpoynGP2C>Fi#=7Te1+appwz4&>&M*nb+sPo11H~NF-29sa27v0lu^;LN3`&^ zoNbe!GnxM9sqZr1tlULv2jg1DpDcVh7dcdLB=~^d48s&eW;ws59!a-xT?4&-7t4mv zCS^1loeD`syN6YRKjxu~wlrq*nuPHR)m&7-K-{nkG3MhiXkrMmLxk)g3X2`?YjxxE zj^$rFONofJ#=%qA`lD>~a{S9}z7?UcTPux$xF!gN@PQ|&VC{`5 z9*W<^U1vR=8p~sS(muP1dsu)zEJ4p@vj1E0~M@T4u613rasO*8_XODl{znZRi zo<`*7=p9-kRq!zDiW_@g9}DJ0-}CF+VuoxbKK^U$N3%S8Tqn;?bUbdV;$Cf$XQC)HYe1!<7bPkq!Wy)OKH<9(Oi4ZhiG$ow)5TSsT$G+Svy|<^f6|9u@}IP z>yJ}XuQ65^8KZfWYMC^IiX}RzZHnvPW!_J?QYkX@o06pfXCJ#dZvCOX=P>?ML`5JZ z&wE^yg%jt+5B?*zZ%}2Q9W1dy{3J2bWr)~`s>}{TB`A-Ai02XW*Z4=-#H=)e!+xKb zTf*bWwioS(ekRQLUL6NSfe~}#p*Sp7d`aRuXE1kXP;ASZ5vaLAxNOe&vH(&&owKni z?0WYd6x`a&RfqiSS0B6^FHbrivb!TvO@=^#L|t|A=U!WWnteT$Z=>2kVYCLywzoS} z+iF*xd*V25&QYR2^W3xjmjD>ip^a+^xUrn7>0fR^es&yR=lr-d7eJ?*Q5=KmEPrn} z>Zhf6IF(2~Js2HcT&B~nu-hs*2#K3K*0EYg;8RY3sEDSeEqv)Ho$|`rV%vykG2B?- z^PhsXul+}}bv{87c92rdbVF8Q1aqP4>h~IEnyG|4+lxsCN)k+$DA6DT9JMnqE4`r~ zr#|A+*P(xtF1sihDj`@)uJACg%O&gzFSL3m#$P~hZ%Y2aWPrcW2Vm||ML*lXcj^mJ zYXHFJ1O2Hl{9@h(fu8f5Gr7TdndRjjE_D#=igX&{?V9>Y)M#=tadE|L1l?v4d!1N` z7#V#azMqakJ?&CeAK7^K0#BYn0F@7+8qg-6gS)}azbWYU3E=ed_y9-+ja9#s)PjEh ze%kIKz6NP;&aQzR?lQY(W0*b}{HC5CSvNLn@Cg9;tYU!(^P0O(wo1J@Iu8@d8yud6 zh@1EmYp*@Z!aWd=L6IEKaz{apTK%Z8E;*9Sb9;Yy_=gDdctDU zqgFQ%*EVDVlv`%KVG9dBz1~=-o^OVL&DqPKHkYrP>{CE}@^ikXgS%m0-4RRyi2o}- z|M3&&0c3n<`0VcfeE-V)1{C`DF182de}Ineg+59^?J1z&AC0ffEFbMIpgxCtujlgG z(vzfB$$SxD}6Dqvj7o~H~U@fpVH?$#=nO~HfZGz-1W zG^rpzD6@a3<^u5FVDfe2^Q=k?Rih2M;PbhAb*O(nfBTx-oqA{Pxp@m>Uh{5++W6yU zl(9-dGYY`QN(IYH3F#43gDOEOKrWExYw)xI+JXb9jxD_wh&tp{85ZWo!HK|=H(M()mRhFk&m0rqFpy&;OQx-{DbZn_(H6x zc)CjnXB*>V@LUiTcxQEiXPLlLO>`#@J`9*wPWS%0WiOlU+$Y_u@`S6!@$jIOEF$oG zClcz)OVEbWQg%!jTh6L>%al?;zS_oUo7HqWNW*)xFZJI)9t84wH?#-MJ@qMr4q-3* z20j4=UjPRmu6EGL2S^Y2FMK}C9Dy7oJaWT_sDFUfM1xR%3iX0^cP9?}K$tI}IRxnL z?_I{0uNDj(+@H;ZY|l`4`}vLKF=+A90(J?n7MJM82TKc{z9pXG2WKWDlr2({(PMD} zzxw|zor#VLnJVG3lb{FB(i$J4f3CsA0Y>v!TtC^KLk14>8>x{tK(XUx85SLXxiTWY_3?S29s{6So6Aj)^4?d(^J4Z!olcbaz4 z-}`?Dt@CqE2!!>fBF*9a^!>~6`Cp;{`VUa`yFqOm$h$9X6Xf!i zp8~>kG3N8=@_74u54gde2EAYV69YGQ>u-WScPnp>fS}z;V0;@WabG`ag|s$|eX-`- zwQ2f@pDQ~hNB}MrKP*O~T*#j=GbpbJyG!2D|Mc;F`;hWDlMPa=#~YcQL09!zuXg~f z>)P30y-*B%XF61Z(;>!(#?*zxb-`?0o)96HdFBoqB2fi8H6i%8_UMg7A2LA!>M7`8 z*3~+eHKx9yMU{-wzWw?hW$7sJ1U45K&@R6Q(#8%hMyQxMc@hoKd3bXd!Lb+Ir6mFg zp7*CS|BkH^X>_?kygF3M?+QMC5CC&>9|aCq^);EyizepgHv>L{Q>X7rhoZ_!evNn< zAi|Ajd+(Mm4jt|^Je--`E;R_9HlmFhdgjb~cPex9`-AFy)~y7YXX<=i+GudO|60I3 zzR8)tUV6csa;8RK@;&Qep(2Fl!WRO{Q|RT>A6SV`Zoy{xv6gY~J1|-Dj8&1J({^t6 z(44h~SQ(wE1<2qMi~ZKEh>>^m+)+^@C_qn!@?Oh1(Q*WGH_k<1kw#Sgl?7VO6W37O z!-213+V7Wx98_&yl%R9U0@;#u3M_FJJzq5N*1o+)DZD|hjU&#%F*Y1q)5~e|`LY@U zwMep`G2N2p+fa|=9LI-K0ds6duxyNqrIpz3EU`xYar0O3OFUiy0Yz(on1f~A*^>wi zW(_RGE<201elQa}yVhGNKe@$sT*zXe2^yUwx99djBL^qag3Ca znO{W|ShA+==L3Vp<)(1+(})_P{Kq}aFG`oFMn%2kb=+d-`XVU}&?X!r9qx+#JxtTQ zY$k=YNRod}fJ)HFK10sPw~0!Xim&gZ!lPgfdi`$*#aq-=EDh6+wtOyZZWw)1VL|N? zf9FO{zU}xk-B)4zQjiU?&8A>=;pD-(Jf2Y8tc3=IG$uiVU|OT`fcRMr3U74Zso))Q z!Yxr`Gudi)Wt;q34Kg?W0i99O(5|kXraXI@4qMR&bHi|`HGmvQ$UC~ zzn0iovg4U#CTeiMJs`JucV$+P zu-uuOz`>cIv%(=lvmL@CEVU9wVyYb5W^P^cK-#t=78ep<)5?3f;?V-t-o1ScV?b znuR@!cx49bGMa`LtZ|%2y?n7zk7+C=)1njemjCF)F*4YZ+;1DTar7O;;0M5@2`~Kp zO?6X%jF^T?uFFaR_fwo{t=~Anv+rkTY-;y!e2$xHS@ljWFm{*!Xfv;hFX|W2^pz0( zrBIto<41I_k%?lpPMcYe2rTleNFie4YJ!H9##1(t}PqAd-hVku_{ zVycCH-#2cf5>_`kaB5hla=oXvpnWfJ)l@DV6mL^)LpHg^r#D8&@2uAXA|8!o;CrM9 zK*eg1n?1POGWjl7Di5_Vt`BDzw+G<_;0VFv%|`IcBcH?2dC0PwznUcnp;lcK)b4I= zRb+KtZl=NYYkdzX96cj);SQaUlJxk&x&YVE5=%(LNkB;8EW9^b`ihS|-}#yWiRET1 zY9jNuV9px+{xX#*`_is@_o`TR=j3`1(tEXXC+%;E=ez_!_?}!GwL5;hI2eJA(V2t8 z0XGG5q7aR;O(Jg?{6iXwCK|VFSUbb?5jo615-tW8#dgWESypi{kJ`@&Jj!l{!SJtz ztg9?DR!58r@xMwH6O(+q*Swd{JIU#R(mbYiH>|bEsYDEPP<%EEfwPfkAZNdaZS(iZ zQq3j1L#Ec}$o-G`U@`BWCMM2rgYi+aGtIckF3%x zTTeu2t`CUvJlQLeaCj9**%;6YYu9_KUyqf%nQB75TKbrqmV%(c_DAK#C6TY*-GkqD zz71XP#Ss}W2P!Qgr9)hs>HXfvYpU$f>%pg48bZ-sS}nX~O;+rRkWbR?GZ2q7-qCms zhDsNB1+WV|23*0TTh`wY%F%gHPE?>2Vee@Ul2(#jd+NX0Bh3d#5Y}gl=)G!SODstf zhTqdHmO*+ES*h?r3<|)zVK$IR?wLIy5rltNmB|Tt;HI+}@WLweTDG}ZD<^ZsSPh2D z{pnX)0ykBj$OpYqJe~`0!R8!n-Q#*va`T6C2+0$xDD+1~Q5aKJvy9KdMGaFOv&iPo zwr>q-hN`L6N3P8}o-bJIG|EcNcvlWg6fD}p=Hm$#k_wgB`(zXNU{Q2J&5a=PtAWJ{3evNYC{v~ySu#?<1n z>{(*z9%?;iUKf@hxhx6hA$6BH@a?46G6m^ zf=!4TyTCFu^I%+wrNczn;M7=eEDFEKb#}s}ZZ}})2sEk&Y zNJ&RMWF%K{KJCSw!S7yD%x;t_+wl^h(sF}>VHhUpO)uDDzae`yjRRWsoS9xO1( z{OMu{XCf34#CC{-cxZWz%km>IQqr4-#J7(bFWKfVHL7*x_rWo-QkHYJ1`AQR*%7qf z(7EpMp*QEUaQyXsY<|2j)Pg^yHxV77UDD6?vkRhec0<1lBdo@Ew{bmxxEZLWJ#Lpq z-Z44}71P7-xaI8;lhLAvfA0GYWGV_^vgKe_#8!J9d!7|xh%KOf4>jg}pIvXv{?Y(B zdOEKUBEATrAmr`;d7l<z7#gKJDvwwhxv&*oSQTqxl#bvRL*s zgZ*PqAA0KaD`3}V(0T$qzyFffBbqq9l2spayyMHybIRw!1V3-*|0qHXZ_KVNr zrrspf#Z2f8sZ@zY*PVc zK0&-A_#9IiZxgC5Da_B>!t?7#h7~*}rka@|7;jl7XS~sr4tT<&gb_Ild@sn(xMP$- zS|>37dC^ToXVN0F2uw%NI{C|NVo5=T?h@faJ~{9Q_~x;v0u6?5oFi>fHeBz})_0QM zI6?PcM-cExK^X)?!`R4AZ4+{t8CmMkGDi+zZQ@d6gj!v1SDFb=r}NzPv&P7dwN4uT zFz|ugA-ktqc)(ww%otp0V{0_pNT_}>^ha?Lmyi)Lw7@ zQpMU?J!qY0+l;EoFK#$a7s1)z;SyiKIOt|ooC{B4#?N6c05X=>(_vk6UBklI1xDc)sTGp_^1hm)T^v?kB zoHRRpexNXzfDRK@mhw9kNLcKjh?9wFAJvNN&SqtSxD9bar~d0I>OT!uC?&gfdy02+ z*@&@BHyg8Q!jsXCJip~&x?27eUCK-x|a8BA<-A6|zCKI9zQzjSzLyw0X)JiLX5 z8uGUjoKry>cA417@F)^Kx$7XQv=d7m*Kwt#{Fi{%Y%wAaPzZK<4|v4>JwfR|7?+l_l)?%QQrXoDRW*2csFV?h8LKQ+ zE{(=Ge$~cb{yuI4W^MP6u4g2dLH*YG30qx(A}a^+@i8WC2d{7PrRmuR)f4vu~WiSS=730sHqgn?-|_KA6f2E+QW+h6u^UubuR_1run zegk|zbV7wwil-C#`F(|Pf6|#=JpD!6qc|>=Rf;q1Se9YuI)5I1^u@#b)o|Jfam8PK z4mTWGapuXJwgPaf@uEfMtyog#7nE=sA@xqD2OH<)%!bvmK8?_-?zZurdCF^g!ow*F zrGcPrqW&fd^M28w)K>1Sg!qJliye<)&r_N&qHRg+teC93&rq~(61YF^E-6DEx*IT~ zY@LFI{t#{K*#rY9H!9luHWG=yiLXJk{mq`m>SD}Jc`Q6U*B~?#sHFss1IGlBz^F)z zu903?Pj}VJYnlN^=?WTL1xZ^(fULUNBav6@d&bsg3CGL-UCW4cPB&nvep4FE9g;Xc6(^>`Hh-Gqn#gqaTt~3kF>-m4?5}yYncO$ei2o@)?e)c5( zGZ6TGe?qH8w?uk-_w{q^k#D@9AF^U-Hjj|k;wU_uK^lU%$ZnAF$$ z9@m-NFM*~toZ%m?I=cc}^r_3Cw7=-h-aX+faJ&++Ze$lC-q#OsuvIz-^a#xHp4^di({>Npdk`U6QAoAxeSReR*jLwhx} z5)^YA5MUUEV$m5S`DPx1J$Yh9h-Hn`s&P&yU)lW5gHc6OCyRY1Vcg6UR-`KL=@yrm zOp@u%po}qIf;2hvx|oA6@BpFYcr)qrdyoyhf=-^(a%4Cej)37Wm-4RVoHSw3Y0|h2 zid8^1Dn93r`;tJ6->O%kYo4ryTU*YBf#`&Ep|ZkWrEkvUvB%D^ZY0q7f7~^ag4Hxq zM_FJ}3S9ap64hB-1WgGFaPd4DZEtbHeh)@*6toazwB3yY!oB{mS9 z=?+<^{zb8#1v@3KWi!8!nLtf+8BNEit_p%qDK^?NCbv+EpTEL zH950W`)NTPoke$krnlTxBxOVd0{wys8hmZ)sI~&+)ulWx)D}Y<3bh%G7xU=JmW&-S z@DJrR@XbH3-R*JXEM(#3+P6EfG(yVv8wr=%ij#iCa)JlLLBm>w5Eb2O@GJ!kPL%{6 zeWF*TVO-9^|D;8(q3v9Y`y{IX*G9))B=F9noZHUXEA; zzhqD^A~LV8tN{iu5S>q(I8PQ|M!zn#2QR)}Tp$9yGgB(DV0RmmN05h)*E;TWLnqG- z=Qu2*f-znVXB?85l2H6<-3k}kskQNpLq)xzu*N2HDPcRg+t{3|mVG~ItogVAD<5x^ zH4LTpoe8)Y+hBFTL#MLnXaGq!>T8p|DGtyKD3Nxurk+?hM-fU9;Xa{^atOKi)cDD# zv_^^nTeJX#=d){C9+LM+v?I-{v}W~IlEnRfrTknz@EW3MNVHhhHYkG~{!`1XAw|&p z?g_qR-t5#EkJ<%qr;8C?O*)YCCSnNmY;F*WiyFL5Z$~WrnmrhD-9|PJ*Rb_{Sw!SQE+(L2%0t*6*?y*mw1sKQZV}vrr&cl`72YO4vPANzW6G<Z)RZAzj>-npkG@N>@N2FOR zBr;Crm3ZvMcTFooOwhN>U&CsJ?u`oqOluGNI(zv8-`wP3`aYC((dK$}=Tb9>+;cw{ zRAaX)_}=&Cq2AI0KL|57{(`Riu1%?}wqYrKc#@HQW@Y2HGteI-J21LM|osJ&w<=JSMI=*CgtHMB$locnp!-Av^+cLvVIdBOavyi;flDZsK5x;E@*P!a(a{LuHxg_oc7HSB3mk%ZhBn>nRDZ zz1cs^U+ddBhKI)}oIiWoeZ6=wG=qvL;`cD7unPCXxsTn7HOoDrTRI`bP`5i1{zLj=D zSU_x8CXP||#WXIkW@obGkzn$oFi?^$4~?=S)955)Xe)h_slw>I<#AS0cX?`8g)B>E z9X4wrI)^&vgod`1?ypenM&b$dyqKg}eUSSjuQy^sV{3;wO=zp89x_D%8zxTRM41Wu zL39Zwy46b{?z}5?31Uyu0QlHqzt04v{JAT5ZKt~weqcE2Or-x-kcGq9AJj0rM3qj) zy=bsB8~k_Lt}?;bqho}_6?q(#l-2Y)#%1;aFwrohO7dAfS3ckvlWk&CL$X8Zr0)=RN=={Ef`)3?kue5M#-r{>K&GdS z=nUtK8OkK7%tTJ^w`KScE=xtidue8v3|p#gVr zYPrVjo!54e(?fSer{cLNoq&v?7j+*nPU88I(SE}2Oz zXlR8L9>OJI9JohH2!j)+cUZS?nb_?fNq{AlwJJ3aTHXH*EBfA+t9irYuxfO9$>Qxp z@Li_kRPtMR2jthkOK(Vyx_Pty*~fQu#|`kgY}IgM(GEG*XpCs*+$q-CIF63GQ4*7G z)~nL(*fAFlA^W^AK$s*qU3|oRLKb<+cUELkDr}TqR_%FN0io-fuqyCYoWK-!6bVVd z<=g&EPAc-1gfMoDK;h*%!@RyjtUF&Il8E8VDBXT$ZU^c)fr3v3=uJ*M#Bi%Zx)o!RMHWb~jPoP8)6i zXn%CgoQ*2OA6{U6A5_2INTn~SRk~PO`?B1su#6KY^0!JGlhw|)uD-%Aj;|Za5?nMR zR^8pvJ@n-*5s%ON*%N2&-qlzC1 zL8phX$|eWSJ1}yAWrc9s32GWK{nR8$Z(r3fH>g|8l)ag1x-c zUa>H|)^?k2Vmhns=_F9ONtd(vsl9)iKA=8~upA3RQnLh)<0|hCAKqMO0{8XP6)t_Z z?MaxC-QrO(HGqj%c7)#sBM=MtN)+0F42AotdVbHuVxYx!x4DdgTucp8lnHx&qBJ%KFq$=%Bx?Ki>Mb?M{ouGV&ziov zNNi19ED`rM614t|FikoF!={a{wI5`md6S0Mpt9*&-Cp?G5{|5n?gB+UXjj*!u(>H# zbYu)6XD)+8w1PT90{G>1iLlDJi+udfi_bo}G5y&`o3XKuW5a8elgTb8qE~0hTe@r~^6586^9I(?O$|T<*1@bL`J2u$Ghrzs8RgNe|*8gOs2WZh406YB|{4 zKB?UYXpun4oE@U#b|W!pDING#-|9<>(yJ@t6x(QllTCFrZW??_Sny6sp(gjNUtr&~ z6?)(jbVS{69pLGwkc!;|>cT~ikS^Jr{Gyl;uoJNS>lVd9GJSj{&aM)Jb!J;s(qU1F zn9Q#l`$Kx@Ei@K-@MtFBh=H>WqDQ*L&$Y^5KUahnu4QAD7S1)uQL(WT!`PgYdgvKT zqga1gH&x{{Ih7B_IPx8QB!{iZaGaKJoyq8S$TptOt+PHZw^Efw%utVJ$sHe=m?#T7 ziZfKBM%FSw3o0SI3^iSkmc~o=*IaNiE(8i;Yr#f9+i1aT9Ho5h5CJTSeZGvXMOTm!myW{nNFApzy1(=^e}~aQT8RnsBfSymO4hiQmTQ<^N7-%Sxq=9` zbjNOiFU}ftaKxbye5HfGtzu9%rdBk>cXT?|gB*IYS}$U}H=6IST;uZ`v>$HNu@LE)OaYdLn$8nA1+*NPN>L}Iqh|n@T1I@yhKy@K}+p(pqA$3X^`67gOFeH zr&4{|Fb!(| z-QJvg@*i@Rt?P(dblUjX_|rzg{O=y%R(sdZpArFulGaunn-=RHS*#@}?-g?QlN{rO^gZF$jiI3Lz+>uy=Qt9#J4i8`-3P-+2}raPH4XJZta;^71u zsk*!0(U2zR&+JGjM;|xyrwT6`hc>`*{_F{%^K|azAI_bA0eGWc0~NjJcZY)c2aUFYeHpyhvHOarOwb0|T3OIP7o|nS?!GyykZ$ApB5qq2|1z1o zriE?o7~<noxO6Oz`e{9yRyj0)LWg~o;=}o7TTm%F;TK#bkbnarIiz0w!1Y!~g6kKx*{H0j6 zlzE9*(2XaGW^qF3QeMKeBzhPqY%kM9-}$mfqrb*Kc5N@Cpf?Vsuw;_=2*dqDl|g&D zF=#d$SiiYEpht}bn9WoYZ!@l8XZwO7fBF%YCIxTLBlx#PCX-nY^Y9r{+Vrp9A-zm| zC!{r%!DC-b_aIy126$iv<3a8Q9Pj)$iQ$0+*E36D7X-#{1c=v#Z3vj6K}APGXvuqB z@oWY~9y|*f1tPVKc+j#}C{?8nT{x1rh^Goe;_9hPw{ggVt{*jkPy5@>l?#?v|3yAv zq0c;zkpmzoop@DH@U3#q4`kdgNJJXMyKYfi44J22|MBeVh=dwdE4a)+;V@9Wx}a3^ za41v1bklrDa^cU*a|yv6uxmq%+?DGe>K4BdUJkom{DKWbw!Ux|pyF;u!^vG|)C@+i zo%nyP9z{62|FL>(@LAUVU#kacpz#Sx*VnUg^+%;k0p%;DWxAXHDG{<3NbKU4rL}!M zSlV#c9c*QFQw&P>O2h%Rt4Gl%U>l*YH83pT$T;*#iT8@h|Lk);kg3&MNF8d~{{)mP zO702YkD`;Wb5j^!$}t>xXO;iVn~&yDAO(7~1I!cWpzvOP-Q9sF5d}Q)^^kU5DT0_U zZ#$G+`K~{TN&|m{wb(P-&X#9M| ziD6`kU;ITIq!9@s5I#N&A?JLgwTiS5;TK>hTVZ8c17PhSXklfQ{-~UM8Nc|QRBR8l zkHMY%*Xz;oU#|xta4Sedsq&Cpr<}d*QEdMJA(rl<*R(d6v@4>E2Bmxc2BDXHu~i7J z=^_+D|8(DmuG!qxFB(J_VHl@LgViBq#z1E(GmO%&?63uk#*7B-)h0yJvY@XafC+#9 zG>^^SrTq7H0vT~D^lZsrCA~oOMkiBA_jPHqW}{^ZbE4HtT*gAtu*f4T=mwyRSI3(C zbf)5S)`bTz%O8zw(l>rP3&FC7M|qYrl^?@{)b2?phneHT@Pu79pXd1tyf6m-Qd9f^)lll9-*`T zOLI`FkqkY`g|$qAc>lZEJ9froe>|EMQ<^sdY|tMmh>UNZ>y>CfOF9ik9@LsMG^K8} zS$yX}n%pM9;mqu(o9L)D^Cyi&i<|LeLrl!otmE9g42NEZPxg;Z_nu}OHTn&Z9qluK z1E~@%1QkuHQAct$^|qfwo9mwQ(7cyFTwzsXr}S^RIwW27*6%yV;VSgA^Ud^)sxlMX zUMJl<@8xwxJDiE14EENNHTGCDWkDJ8@~FsBlN_cJDJ5pq8aq_?gH#w!Q7gecBld#geQcIqVH@_FBFdrj!- zWu6mlS$BGb#87XG)RO()yqHX#`)+T2rV(UNZvc6?XLM;nydV#Kuuix?Cbf%P0?@va zkAGnYk}?pq{-k%Ltc1Msj51C358uF4t);4Q#){!fnyGhF-C)(1xH_t5-Kn2SNqq6L zbJ$^av)k#9lQS#O&^eWG!8u|wH0d0j6(Ob~W{2!3j`66^7eme*>tpyE8txX)@Iqx> zCA49L=W5kM)KKobh!ZGwvfpFbcIZe9NkhNI-|3tuc_39l{np4Wjt{{FV{e26ha)+@gN#tF9~ zRz@6O#|?aNk`j+|LyA(iB8hib&RySE)ZRDilW_HRnqj@$=CuV zKlSA;KkogI5`1&orrrrjh1X?5js&8MO5zlG7L;a6$0l2RbZSO-Ts8jvgZD@ zMW0MXJ84ScM@{3<6eOR7Xz?VXWE7A!jKO|WBH7uH{H_9x1I3t5%6T6c zUD?GI*@8}nc`xj>5+DR;qGCxma(^gEJi%|gLJ$0VM>(tFy_k4XQHtynKV|hxc1*%k zX2OeHl~p;j&&yJYd#R#UJL8_mKQ~ADFOAf!A&d!i%Nr>%04FK%Y1gxUEKHNn_NYuB(&Z%Th!3N z7jqIHsckA>R!;q1G4l)j8M%hU+3;XhbT(NjK9jld7Gu#yIUUyPQ+Ib;${hs*S|8E_s;r_db1IuKpv<9_d9|!-#kdb3hn&Q%f zgmovvkcA%|Z2{1k&HVZq-e88WR~~#&FJv%?FV=0mm9`dmrua2s)3AIq+LUSdXulTk!4=+g?W0hX>~= zS_UWVRLsXm6d1Qa-qjfD>xN^pY(^~?t94I)FKFqQ@WJd2Pwu8J$J0ucGS2`K2DG7< z+&~jl5t>`HS3rPM)CbB2m%V(;<$P2BmoQd-Jhv5u^g)*_+8&{1u9e9iz4VW!Q`K9? z3iOT6l9}6$010ceY~P+ScgC)4c5>aAe{IC?(P5k=?Q(UT9c6X~gc+6V{e(vW;DTSk zhXA8k)>l0=T>Wp4bQS)BvqX|1$C_F}L8?r5VOR}*9VS0z+ZmV3u?h4tT{=6u%;tj5 z)Q`!@)1wgxQ~#JnhPj!upgL(Ec!k?2itKj9#4(;)VO0~Kh89jQu!rgG3hePF9T|7w z?i4m4`pF`>1VaVf^|J){VlC{4D6yd6u=IS8`Rl7W-PrkzoRoSAByvNbdLfvV#9ElRS32rU@Ai60Nn z?FZgk90z1O5uu@3{Jtge;>NL>L@4j7rNe%NO+<(msBMW3M;&1~z@f5% z4dfVrD1<^94FbjeXotxmOb`&bEHmR4rP3n)m{I^$gz}Pb2C!G#U_B7EWoaf zzDDMS5V@3BvO4!*V(sN>a<)Z)r}xja>;q8F&>@r4`3X|i{~a_P)stB)lQ9BAZTC7&d#Q@A8+t-iQ^lVA%MgGnOw!8 zoC@jW@#^(*P(HT{y0@DDdRhlQUMTTx8&Y4*jqeoE=Ry9n-C_pAmFCJ*{ znMd1w&vmb7qoHQWY&Zx-bk3njW0!zwZbT#?G5_1pgP>GOKxZ<7wU$Zf>VOuBF|2jg zeNpO(WycGOB12Oe28mLqD2BNr{8Gp|y+TXRJd0p9Y8D=x&5;3*kYL$muGpN1>!qFe zc^hV=8Ptqw9<gDLi3U3z6*+dO|M%<~a((H=h*RV>F;I!bE_>6PtC$tX7zEt0NxukG$^ z?hY5yeLEreQ6{x12^p8etN?5$GAxc@Ev`ntL78;3QLK+{tV6l0;+duUe*lX>biV_{ ztmko-FN(BBFL;^z_%U%%GiA;wOS)J|)fudCtgKPG^6#2J#}@jCftugkuI;|A!1myx zxhZu)jcsb4bpa4yh?~Ixs}mqViz@|k_VMSXa`GfRs=6rRf#^5{P1C0&f6`q@{*LMn z!Q&Oq%p^xC_jGBuR$JX$ZRM)KcZHM#drh?asb$&|zQYOsXKtFMbu_j091bIsybOii8^`j;s@AGS@KETNgLg#KTA>C9H#HXRC) zJPr35*sU;2y3)M18bl^`^zV=y9yS^e@|YNo%nNjuCI)Ddn!X_^!gWm44d{B7fZRBb zxbjm49136`)nN3s)a`PC#T%%Ic0wAXXCqL4eMVRnr%foC>dhF4H$M5f0EF=uxn)z_ z9E0ytpyZph z6mjJwO)dF7ZTO_V9kmPbjn{5@qa7U;-gHYhGA4Z4UMs`DHAOZe$qw+1NUHPXqydc> zXT-lIVXF0~qli2ZNm>F^t$=b(PE{z^DGue7v^-f2Yz54_x~UaB{-EYKL#%EqlWEH9 zZj435;Vy8+Y41?_i}RhVnmvnOK#=E&F^8rPzQ5qdOeN(A(Ou4mXh|2B2ZMc@>nlCF zH7Sa|5jGi~C2NE;3*s@qW9 z1f>I6Y*8N0>Mi$zd^=9)4QAY;B$1f)TpRGiX@x2b$+Bt2v&cs=7>jAianaG)+Ym^U zv}#pc(k5*TbOWB`uPN?sXErR?U2fybEOu$yi>7u$$J4$z@=1EJt?gjAsE#ljyXX*} zgy0s3_ly2u&jMMaFbx!2!E5o-dYt7Ds!qI-#k$!6BB&spFQTGeq1*ZfwZ>xinO4Pq zc*39$`orQmBw{L%bAkA*9og8(ts3W&ywOzFS&NnM8jdq7!@j~ad7C$uZBuSun5^c! zu+GeOV|k|jz#t0xisQIs?;a9K;fW$1B;+rbSwq~FHCKdRNJW6JWw1}=cN38(8(Ul9 zR`tasSVWSE7ZV*xYav|1HfkkY8MRnzFE(pWE@PXu#!El4YrouFd8^yO^8;;D%ss=i zhfc_1Lw@qkp(DR(8`To~X=@@QHtam8o@c^NoHM0njK-VQ3VT~|0loylgEF}KmYXmKYb8;Goxo-pp*aI*Mxvw6aRI) zzPB5T|GK@qoyUL82~^muUSHUUbE=>|4BKJka1I~3<+zFR_ZqT@firE>YK)_4`#46ks4V^l|B z&Bt3sLy!ig=**9JmGU~N-WCa)O-$@54i-VZk>mJ)r6JEpC`u1jQ-i+{^9+p%y!WAr z$q@A*JZ|)2@oJOMo($0-7NKmr5IMy7TFZNiy`3g+Xmk%#feLc)BP%67CPrZuBe{?w z0(siD=F8g{V?Seurq2Qnk}qoUZH+{6xHt7Jh z5IvSo?byU{LE}yI?0C+O`#*-gpw97Kg76RZtdWflgcyyFLIE2w5HzMDZy5+F{V56Y zy7s!hk`N2_V$$F?&Qe-io5$)*!yv-b1Kd@JxaE?unTT%fAa@Y_+N0o?G;tJ8 zO7$3{rqh3-KXK_RkScGA5^NbWajGalmL4u>2)i1NVj9w`PF-RLE&=@`0n}hchg}w) z<70}sxpMO|J94MM9JPTL9s|Zl^#H$_tl$37`P3TrI`6xn3I81|LFeu1`zB`WlZNpXk-`)< zt`t_Rb&+K&cGDptz*j<2mVuS!_|jdDaDLx+7C~51j{4fzsdC`(p&-b5e~Oj8|Lx9p z9(1aMo>iq-T^>FFCS0?w*Vdfwd`G7{WQ&Nt_$%A_mpt`(g7Y4^2TJF~l)*h&+BVs5 z6kS=OMn9BF!8TB*j!>+El2%fydK8Qg@9b=Uw>2@Zx8?w!{exY&<^o^+w_axd_=jw5 zfiDFaxXLQpgjMp=v~7O5Eo1ioGmf_MPk#QFpa13Y|MJh{&i_37EAu{XcK)}sx3{|& zi~m>KhS&M|Uw;0Vpa13OfBE@ee*X9GcK)X%fRyKUmzax&SHbx$E8uql$h`G$Tq6KA zp@&dAF?r8$)_SV8OH2EnLN4^lby~Zl>lel*oJmd0}WDhii-s; z!IWnn;9}oNso>ES3`)O(?!=xN+A7FoqBaiuXMY?$9sD;v$NS*SVSxd;sWu5@TOc(| z^-S+2^E?|vNDOh)%&IhELHY#|8DFMvY%q+VxVQi6byA5_}+ z07zEy8tmvuF?FS!r4cWEUG7HRGkwuuNmILoXK51av`mDFipQr|1^(TrFqPe?=phz8 z+^A@OfMwy|jS5rQjf%Fh=+V(eWijp^6nDtQ8kDt?2C}Lr0i@?@@oyZEwKV6v3fBTC zDJcd*kT+bGpv(bdN;QE;no)qMWp7Z+!ux2TC!!p{YkDXhJnLdElmb0&smJr<8xLd* zg?q$UdNYhE8GGk+iy$~Vfh9vWbLp`_;_h(id^18)1F&_vj zdZg#fQBq;hKsEyTNGMaNK86PfHr~9pZ2;*N@f9T-ef12HN48^Fqei6Q#}w!a1`FsN;f%6`j#9r3DC>sbT`h)FQzR3lKxI zNjyy$h+T}liOfu>@KA=_YHS9kU%b?BST+dcSQ?mkG9PFGy6#V4BQl&$nlSe!<~5p0 zADwaMg$H<8zSN+(shTO~Yu=`--YuS8VQeee>3OWr!DKgB70WSs;ZlapPkcp}Gn;sVQ7yZNF-Mate*e;|JJ2k`4q{5XW~$Kv}jd~YAgDo14s4mF8c zh=We1OwMFygYV*~UC6d%Rx&wnbR8IQ0)kU~g+)ky^;}zimFp_~zE5os0Mm5j03y>7 zP<}y4gK+o&9m?+Yo15`z7Z z(ZpzwNO?iqS#e%4o?}`bUkH!*{lH*WYbZFNNj!F9Rg4^4z1^$5VO2l@gS_%gTH!a0 zF~{m?L}FFM(85br@V7y4@f(#5AJOzE^2}RnG8&0%qJ%XL?YY{6%=2|#M3OV86Fw-c zIs+8*9L6Z&&%+P*oH;tUFuro)Dg@g}Z7nr94S^J7Rlp#3RispenAri5N=mnRcaW9T zK@{Qv{v>8I`m6blYi>B5g&Og_aZQ1XsVZ@k)SBlh`3Ggeib&DSzat}71VH?r?m8AU z63chZ=QF+1f4TC+%u&utWkV?y!(aO{>2h2>-Ae8y08xI2>HZ?chO&*9r;hgwt(QF zZ%38xC}MnTAuVAyj_6fkkXXNU6eT+Q>t8#^?|z1-D_e*G1rbhPw=_K#maXo#+)4?~wvVcfxQLjV6bxSbwqyu(Zu}F;0%nE}3Owv(D!7xWu#=DN{6GaDr5tVn-oqAa&%573DSUW%AkBUux|A`rk23p5pd&6 zJXOYw9;PwDliFYKl$Te;f_kE1NpB*5A2?_OG(adbbe^Miu=i=Wvrsg7XEP)3O+? z+yDf+e(Fx3-4aTE@lnhqt%ZUt^j*aqGbhAJ7=0h1abfk12~aB0ejZp^71-ri%u+F7 zA4+wh3p|$ANsj)lslS$!lWhC&9oPfO?>(5=vBYmcbJUYUV*lpU%@i6jN@W}R>4qAq zrzH1&)Ulh~>i-x6z}gdl2ONo{Ol0v$o(`<(7FWT)vR|%e#JaU&_@8uTj6F;5^ibxD z0X?d|U5aZRE4Er`50mOgVdW7_h8`1c@+27L9vP^^>X=d;%j}d8 z|JmxUT6S~Yd#xUv1e>W9Cs(sgvE58le;}@_6!<8rsd{+?TB?$+XyS@@3FnfvCp+9h zv7{OXbKa;$^L-VI4^ZnPdRZeYu}r8BXDd68-vK;Ku9Y5$$3`0e#$A5Men12P zDM|~oJe|a0ev*Aa$jIvy(X5_CHUIdG3RIY=bKYmiy`r48pqCca@2SuQJSiLpfN^&IFbmTPXfjLonz^%fjoF}yRNvzAqoAZz$MU%D1Q|yt zSPs+lUmR;5-F=;DRau%9zuXstP+j9PKSaWVmldd-r_Uxl_QJU&&aqZGLZWGX15HQ_ zc$M=Yt6~ZlR1y@8$C@Wk@+nr#1_sp!Zu$x7hgeBh7gfdAs^k9Xpk@|L8g$E>g-=P% z-k2V#2<3D_p_z$mw_z&A}_FnJ40Wtf!wpZWY z&VxYwXUBi|yW?D$=7QoKZhiIqf3>%l5&wbD`TFjw+TQ-&tKIGTA7J*^UhVvW?d9C4sXcOD2RrX;=f)EX*i%| z`~%C+&80QIR0$q{6@*nEtxA7KuUU{dQcQykM)IG<}uCJ?Qh850UiO23+rh2*>` z>)yh);6XzxmnmspZJk>ME>s8fMj8sM&Vppu-#-c~qUF;aPgoFW@#I1onM9Rjd7hEn z{+HYTa{FKY`D**0RQi=|2EBF7-{l9bn?_B?n=>Ni)=WRXR{C|-Dcl{OY1aklH-2Rix z|8Hmic~#$oX8%vEee-#k|0Uxwqk-`o*JWlmV>cT@?2cX*!71Go#AraLlS$NM-L$0~S+ zXX`U_d}+)rZ`OR@@3&66T;L>QH2?&!>Y9c3>jnb9*ZP-fg^q2{WX z9UDvka%6%4caOUV?PMLtnv7)ys3ILva5XDo`qnacp%TI08xnJ1q90CAhP~tCfqeA$ zkHgl%0SOsaTFz|~)2~*qTx05&Wtq3V((6ZlgTZ zb)}$ZV*m{QlG0RC;1XH_FaUo9L=cfC2=WmDfk8!rI2u8xdT;mI?W5Kwm=XAW*gk%D z)O|00d^~6mNemoz2pDG;#xHVB`1tfq?j1#`bumNXa_leNAir+JGcxKr*_JM7fg2f! z^_XK>(y=n)At@lK&mQTDnXDC^L9$llX|h&wLWqiq7b$i5T$VGI+fpr)}F-+%GYrBisYs#}bbLz%#YU(mBu zJ|VdoBr*;P&QNwM^8CJXa~)tAl<&ti*yM%RXABX8=`JqZgAsaZ8p zsV01k8N_cQka(xp_0Rt*ME|0c^JxD*<}vmFcTR>f1Fn^X?FI&`=7zHpdGUbrK+`{;yiC=i8X*KL3G7iYSu(p(S zX>Ib-kx3Y@=%&>>Fh|P^>E*4&a)qa9#oM-N%+-=({s7A2lBUocm$@_utsqF1&;*2= zx8f@*yir9?bER&;TJRoVlk1|qyQ|NPTiu$#+?aVyOxLX;`|zk^_nE%qFJE5WV9MQ? zYoEvmgUb;4WlC*Gnie8%hPXmRrXt_;P@ZD+7B|r}p{F=7s~{5&%!(7InBN@Jq$`)JY%VJrXOqQvn;Yp@cPjP<+M*kg_D9qO*(m((ts#8oG)h>7S5XcrCaXx zeM{SHrHpjT)V1>YS!3s1$ry?0iP;YZ$fSyEWn#0M?@Tnb5tj$?M zEb;Bwy#Upx7{JN(A&FKpeRniGnB$=)Xt~gP!bDN~Fmt*;TU5CtZ&ot!C^(wP&5`5S z@$650idXmqEyt>fS)E!@&iD$=_+;kSA-|6=S0!8yOYyLApz8*2qOYQBL`v|%=o%6k zA`mSMH3Q}1NW_^sr86X@0wSca5f%U}lDbFqN7CY&Tc?8$Z#%G) z>qXt4DLX_;C781)Qp%XpJ5oE@Bv0|(Th4JEr0G-Ah-|K0>!N!orhuzx8-y)qQ;i7l zv-Twg)BB$_ZEe(8L3WBpmv&zP)sjTG=Zhi7s|t(>8Hdq{uS?{s5`q1iITx; zRGm02ikjG3nm{esK5#TZ$t;5kGg;1NcPJGtP-~pf0xc~vFiT><@U4ee+o;kOw-``o zM-J$lTxCXv1`^ERr5DwVR^FeAh;F`7%~1Tp4Fz-=6^7WvRyG{*t5u4^NcquR6I2Rr znIBFKG-H4sIkV=aor==T8h+A1rx=22=W~NKK-gVVA>37|y2+YKQ63Kxn?)aQ0z+;% z(w0*GZe53k!ks~Pd|6><=j))5F^O(tCSfV>#HFc7Qnfx!Bx#*s9}n6j`GvuzBwv(q zN=qh+t_Z0MEv%IsgqAYHhMws!7sD$7yOc7A6$+g3{b_hXPQ0V1g+g{UJOROFZAkgU z=?-6*PCTiA2IkXD38OUOB^5B7yqK5{)PVVgS*k_&K1zj=m^Qex(4$)nr3VS!C!v0S z2m1GR1ttlrohQ|iXX2798cSTf!TMiJ%u<9zrbb%P#by!o90%6@6fm%{A756{B2?uz zYST^L(+jV>0jd8Ch^4ZuU~L`oY4%#}kI%o51ia5!>!e)KCB{1G<;`?$Npsu_-5^Vh zPJFMUJqR>@38e?kwqUns_V|OofT}5LR45o>grdg2nEEnGj*`kEXQHpvL1aJvJ5}w1 z<#7`Wwu|HtlT zmV*m>mH36&0KuU|5T;WlHKO)yxao}I4a|0{)4cLfWh|xZVr4a?HjFg}xwcKw(*zo2 zoCM)StFWJ=NnF0%v%fO<$Gly@NMo*yIHij+;&+0jR*OJ4 zChWqc^es@ptzY7crLyx3&6DN(vobiXMucM6b9v>ZYo~7LZENICuEbH7MR)#F zl@N7FPTnI>*CiFYeM;X z^oVjW3e0fv(eYx#1HzfO)0}AKERrh+jxQ(-!6;m4fP$qF+I`15JV~;hI?b}#Z_NZw za-$0OObdom7el%6D97N4Rg|-lJ~d{ReaF?g>s>9Wi6-{;Xt)VC#cBFzq&*2hmjD!z z7l&PlZ>1h!#T}6;tr@GSm}VnO1r|Q5ozoq9?kh!4>gTY$)oW`zC5R{+nF4oWP!sJu zfxtRQ#=;7~h!kLaFeosm7Fh^5*U+g=y6jKiKCB<2h#5^8p9q*js{Wj^Qz7Qi(ADhc z!g?zr&DxSi2T&TB1*`Qm<@?k#+0=49vJ$u`IEWDVV~e5_B#e8203LGBSL`QIZBv!B zMyejXtgr>pWVw(fn_7CHnJn63V`%m!(k>rVHOS(PRTrsMN1z~F6*jWI()g^ly2^@V z*<2Y*WEI*@Xom*Q@-H3U0kYp}kA~#K%)&T+OnOC-B zOuCqm5+r$ZVqTkeduR7`f0YG31R$2~rhPR=swSsZ(yF+kPkd9LC%U$r(^6yVzua(= zqx$$%yg`z94aqbbrKUKznDp@%CI*|2t%YGXDK2xWtY0tcqvqjQ6d};3nn5`k3JK36 z@OTVc(8_BU2drr5WFW;A_dEhoCX3n#3T~Up-`I?T12m&ai#CA(g_d#B*t1=>eU0B~jkREK)hJp%f+l$siE5+FGhBO8$w}=;VbYhf6HJK=j z*h5i;hUj8P8w3z=9_D%CLOo0nhzI*p}Qe#%n6kyM)VC z`oT-+kyT-J<>=)Z$V@(wada<@Yn;zB$HnG>$^09KL-yhZw6MMJ#Qq_C@>jMK#S3kF zXIih*I{ftIUgx{#fbzHO&wx^B;tQ6W zbf+5!3M6qu;mRHDard0aLK9N?YBC=3hMMflZ6o0tn&LbXJwPkcd0tWNB#|Y0aKgEX z-ZY`f2KR1M9G_>nAD=6ZMf7qQkhoYWa`Wj~@ymVl%YEa^eY!33w!#u-UOX`@0luK5 zSHd2j-JU0~R?VqUFyrJx_-e~av1OTj2BI#k>&%S=Cu!8DsF7cIT*!ukI2F%K{->#= zgL>-E%sO$`#f)+((=WyGv7C7ZrzIR9AJ$O3dO2=_eHMzWxoJ+wn;UaNa|;n_WjLQ? zbW(Z#94KpE447~M_l*(vj17EWnK-pZEKn^F*A#dFOR=^^CYoT4z>*q!E(aBLl*u^# zq>i&)Q*hl`?L2PUCG96xs#-r!1;*KwT28c)4me&zZSOp83RLIz;Qhc;Ra;fQ#;gI+ zX@hpGp~AjH|HAi5h(W^iD2#5TV&I^yFrRa~XyB6n|J%Fv=C+L_e}D5+Ao8g|#3V$$ z9EZ2ui`SO3x;VDWQZ{#|s8|pQN?4Hq7XU5mT)NMG(><@A!2qOe*_FM3mB=J8m=}6_ zdiwEes^yg93>Vj^;RKpHoJ^eRSmDV3`sMMD&MLSR3KekM^m$z!#Q(pkBDVUidKA_6 z1dTc#h8a}lK)(!UktSz3aR`#KLC9;C!ZZ82`J<{`uHKNpq@LvmKcF-WVUM#sF0MSj z$h{|nf-T({Y}m+OaIG3#nvqQRmhj_FF*HqkG#c2E`k!hD|Y5V-Jxj0KgXZSqDyNc*zy!pB|wHH>;^CpKS@}X)uRRlPPw^>@3Kc zvB!uUBalDd?bJ!6PQu}`sBsD6+R(7pTmQH;#03_;jrle%UWM4xlc{U93AaIU_@Xnf z96G3j1~Nlcp$qVuKjj3x=F|Nu%dS^T?(4htOM0DVH>rAT>XG@=9S^Kt@z)yIXx$qQ z8#wPj?YJ!qJm#E_CS~$ImaKR&btD*IR31-O7)L^%@rtXM@f;L?ufR}?KTJPP5d#20 zV&3~GPeAZxhtRk*#ahYD&`%R;^$Rclkdz!2pgX`tU#un&*%pMR76G{IV(OkfkQmqo z{Kz{d-3A9wD!}kUTaVa|3=9U&_{}_~d9l{4$lBGXMEFUSIgwuSSOY2eh3LX&%CPUV zgq>D=cACEMy%LZbptCSqTFIGzdi3sRlwT?_8gQfM`R_|6lbB8>zvjt$9}}1=z+czz zb~3}k+fD#dHWrnI!t*jo$8ugQhRdZ2=!0QQ7u5&m_};zXE(@neC#NW0<6uQu*IR$? z8VkN;^@e3jf%ADDy@g@FG|IeUlJ^>Ert0<-Sel(#@_{cAHb>JaM--05vgEDFrjvA* zg`2I8*#|HSopF`Z@Z;SNAMegRrFyE{Z6u(*saajk=g<(V?;>Inc)y=dfEK(zbtjBJ z{O?W(m-ETLd5D_-oB#G?ceM5K?qAuzTfg~C98OfjtY1x*IWqJAm92b?HoZajH;X;3 zF8at!dZ^3weBda)=hCQdK@fxOUj#xVG9j6doj0_l<=g3FQ7?@1DLaRDm{X*E^y7}XbopHP&B~9^Y@DL+3 zj}5YpZ$`FQO*(ag+X~FQggfCvDn9w~Zs;|MNuq_Grw3unx}u2Q2iN%L{1`*rB!)v{ zgtn5b*Es2j=1hlU%D<>2T;XjZ6ANw~`n)#2m}eKREt)fEXP>&7Z3jTcBKHq6X4+p4 z0JLN>J8yOCW-9@z{&q3iE`Ccfkq6Wj1QqXVYs}5A21ezN zeoKw;Tw^wdeh@T1ZnX>a(@DTvk;f(CDQ7%hL~OGf`$PT)F$N#RJ{gZ6KFco=*XJ$X z=}zNfk|%gj82P6>^pZx+3iS}L2Q@B{sG#fj`!d8an(zu3r^@4T$yOJ;2u_6~83kF( zPK2pM9-`R_TaNsrl&2Qv{lO|O882w=3=b)l?aE)SuC48i5sFNMea}R_=k;~)5%9pB z1qz92rlXK(!26#^5c9qlK5ZTssD0G1#1GguMvN#vYbaKv=h`u$+miOk=^9`HCx3m` zPt&xgpay*JalKao@#G2CjGhi2zd!UFRZ)v;CqR%TMC3L1IoDmzzgUm+%{y$kh&T6L z91Ya~Na&X!JGLEuR_{rTYX|UJB@q`X*F^6chag=n)e6qCr zPCW7-sQaYGwIf)IN`i#dmy{Ja{nl#6nr3|+=*|b5W@CNnlNOByhT$I7i!A-kFSQyL z^ju6tEGMkd8SHuWd`q8Y3hZ;9-GXm+<2zkU>8q3P6|c%}V zW;ZE#?#1A`z+NGfX_}u^`>o;(Ot4s`*J1M%3E%=gJN!p&8xPIkvAIpTJ3&t1;0*uq z>#2xoT0@y!xnc5oe(fG4;QQETjfW#T84y9Jm?c53^YX z7K8=xhgCpFJz)`Nh_DIJgyr-Vt7rvm35&y3s~|xY;zepoL6^|YA-1^1e{b>Me`ombZT`O&|J}xaf0!JA>+=5{?jKnB|8@_)J8bda ze~n_uz1U5Ay#V?6>lNEC09he=GmD@_#G;|I_Aws^&~dF4Jhv)2zU< zVT)1RIgJ;~Ia_A@-fERhgX7=eCs`fG(e=T zQ^vG8eHGqckf8dUe5PJkY`WeO+z-=B^wVU_e{y9+R6G*K{hVzQ+2R+`Ay&=`Xv9`V zRXXh=Nq2+r;FUFt3uR6h6Q%rhNaW-R+Kv_J ziotLr-0v$%AJh`y!4>e0Lk`Fv(M5rg52|3mS5HuFB$K9U3Am zBW%a;ltwa!iwSM2ekg*f+23>BHu{MyWe_a;M%kXEH24#z(=Ob$Cc=0YId+e69W?K| z9K`u>AVwmBRAh2RBa?ycP*&n~#r_DK?sgdmW~A2fOf~{+tcWq{CGxsBz3(BB4dY#_ zrf>`b2H$S99GF}HCl%BX@#|7sj z2Z_ytCz0fbs;VQ2xvd2JXzTOvJ zj6j8-i-*nz=YsrUB=X|!`8~!WNE*`7*b7dl3Ze8_R0uRl#}F?Z*Hb7gIC+?&lv=n* zl@q8z41zzOo!^f<9_kgz0n&GbCqZ~8{LVKcRch0U-K@ZTCk5fW9Gcm|)PA_g6IMXr zJ43`K5AIH#a02UR4n}1HLDzv8RA0FI&ZBA_olzx3$chZgpUtzV+(Gj9wC{2e!{Hn8 zT5hU=;+uFF+2Pq3dKBNT+e_TP*&XmOQ_ zL{Ss|fuI`L#HqsMHw=%rgWr95R&WQp!$`NMC`dFSK@gw^SG@T>$Tg3_n@~U}aM*z= z;ga}%{2S1aFpkQ=`$aJ+Km`UtvWQJ#N;PU0^=ZQ;6MS-iD^L+57idJjN}14Cl;g=_ z>H=m{-`}l39Jpm|gI{@^gH{io^v;N)VsAE?^eS@!4n+6f#3IpZ!`S2}M&f0+e>B95aT?HWPB#@gB=cb==$) z#JSQkf8tM?eP`4mdk6cI4YnabHYlLRHxy9A<`00aL8?TFW-xTcL&E+5$Li{d2PAZ# z#=LJ&jmZ$wznlAY?H<%3cCpnWkk{b!NV6o8$ItB;K%CHG&L|u*a9iU+9nqyi$Oaf3jI(+p2zXseKfi+Hfb<>I5x7C<>xg^eBRB6`3;|*%NFGST4-))q(?zj7TmqdC82dNLl=p7Ppk(wn5KLkhEcKIqH1Om}uy;y#m&N zP|Q{+8zD;!!k&UYpC?(X11k`QMTiDM<$HlTMo{P+zZC?H=gcTc-Ld58y9t>^)aP*h&(1S(49n$p7hzU(a5Sk6$01_T=4@H_v|> zpParsdiHZw1S-M{NHNoe41LgJhj=Cig4P7661ZG9INLpUR0i)jwj?c)=v^&#cUqMV z(y2IW-~8Pf<1tp$TIH`|S)CI2>p5yQFiMjGsC!+_BDRabt4HMbwisbEdiJ8Sj2FwY*kWf3{aQ4`2Fq;&oB9C` zTFHdx#nGG`ITyrKPe%Js95yc^Mw|_?+ExGOrPt zsduxA_tL^eam7wB9%v}BWwo^~mm6=YJx|%*KmxM& z6K&XK4Qh0J6AK^b#yKJUwN%Vf>vm_eb)`?%DQ-~tUKkK^6@>As3x#oBeB>QDVII-= z7ZWT67mzmkYbalsE*z_>SlQ!fI_>(eo{yisdIhJ)=RQr+@CwXDAAL}S`a>@aSpgB5 z9fo(@8i}rkJipBph3cXw>;n#05WKH#{7xSeoduKnnmqE}vI*|v{9#5(1e36=^7bLV z#+g8rL8Z)~dQ0kS28M#)-ddFTnRHp3(J6>u%_6a$)wvKV;x#vj*JGFxYr*S`o6j{C zJ}MW`K)_dkMv#<}AhDL42+P7CT5VRLa990MY6dF{OT@O1*TeS)-Wi2BgT~*}ZSBr} zgEE8PQX~e`9KcmhC4f__-dRexGLVw&3NKElMd2`T%O**3S2J#cxC0Dv5g{%a1)zMl zN-ZB>faa`Kuww}H?HpSG_kYU@|;)fI+OqG z7s&0kBLGH8Nai%H8^iiR4Bsx4u0;|t?ak&6H+|)EV~u*RsU3^ymTxHMhX0RpS5r;OEl)Om z7gVt>=cWqDt-F&IjC>GWeN&ld4M>*th9iszV6r58rU&$G3}C(D42bs&sLuFnlW)m- zIn4?-!VMag=?IZk4v~!j56qPp?U)Doc*9(%B3i2ejcMRg|K`H4y2uDL)ym6W zKV^*JMnbHwpFL5%N;j(B#wW}nn;K}d{>&`>*`QRtFB{e|!hb^H0PIJ_BuT8SjuQS9 zYtsX&=o*@q3=DM)y5(^x9O@hAbc1bO)`JHfG5wqqj`a)Kpq^rNQIsONNd4=cTf;%Jcc-6Lwz5K{uwz$1ksF%|Iu$hpy#Pn^ zYzb#5CY)Wc=NF+X!-X+bCLNSwEgD70+LVi_yMZahSr-+zPV-wCnV=#KJaN#~26NZO z08%obVb+{-*Su%mvBzvvkX<7b4ihO>g6JEbV&GEO=Mk3x##doZ>bZt9&G8$alfi$E z+Jfyu4|}`NA`E&^+ih8%?LarS_?1r4D`%vFRJyWL%5`6?=ODxil5!;$UdREw>R%?Q zy{^)Z>5e8>UOih@Xn8o=N{ulw=73^vBfb2|CM#GU-Ayds)a)FdK?m*?J1cP zJ1RIZarO=>_>{Uedt6bZ-aD>ahsf#((Fvo1kJ#kxj)HiuQWO+;J04#!-j3Vq?{Ht< zk>e!Tp}YH9;Yub+DqLuE4+`#*0ozeJQqJvWj7{NQ;&E3lRyJ_v0I6^>Sitg>3R)rI z@L2JXQJK%+iE|`0p*X-EuGRUeCr9D|D-FzK;Y_zAsKJx)pz|b+xtA1rXKHeAx&)9z zwra&E%2|Ols*|w5`D2=`%JFO}c0Tqy&U5*@#-mT>X%TuS>8>F1s;KZXwBgPP=N5-A z)h20f=*0kBNxi2XVZDINJ?eJR2kmH4aVCC<{ly+K#huR z9d_HavRT<Spp_h}>Z(icFp_lq8q@wcFzr=ep{_=26u7g*HQAoI@72o) zwg`0liA&FDd59?)ntZQ(E;$vq1>2paveyr!RXYxvIzMo{v6nAAyv<%0ZmpuUbLK0a zWzI=glUl5q=N?-Eyyl*2)@kbd8!#6X4%W>sncTvj-0sJ_-p9KD=Gbq{3}$z#=f#w8 zOVHgjVZN%=mM00d_T9$`b;2S%!jJ6i@bVtW6u4)lnPsY!9g#YXnX@OH^{np+FX!$F zzjv>mRL_(C&NP0%ld|0(tPVMii!7%}TH8QV<-ml^Br0#QEzb6JY8X~9MkY5)M@&{` zv{57*a9YU0g4%AK-Kpn&`>kSU>ucbzaa7q2Mta6KsUt~8a+1sd&>1?}iW#zAt~zs8 z4xMbS3?Fx=>)Xl{%{b%1!A9#dzh zu8x!fS;!Qoy+oCBpDik0jYK1@ml$6KL|fzYyTDa^de(*Cfwf9xZEBe~Yh!meuT2uR zw5_50*{ z4|>=*q$ivYy_fi1wILe7Qvqb@utF~zBQiUc?G&9%SXDfA=WI2n4fODk&bK)?@B7Ty z8a!Ke$@|IX^KKyDu19o|n;4g|3X+LVRAbR7iBlMahg6P{TO-y3)X{aEMz=6>_2Zpl zH0*oNpd~XIUPC%bNs>KJIQ7~i2pp@fW{|_0*pQ+GQj6xv``E8!a6>9<*p7TCfIh66 zCVa>KQ-s&7mE|Su6zur9ycDZfjSy_$i0>1g9RK*!@vB!gOx!i|Ze5YdgyrIp!L4c) zy5d@$)ljB2kNr+#=ufBaQ66wr?+QeDU8CGXLT-_?U_j!~;pR&w->;8776Gn~GrP(Q z(OqJ~;=oKZs$L~IL)o&-JqZD_*%Adf-;Lj#)a|?ab&Ey<4C02ZUhZ5|_V?et?smZr zl*b_}z-F;geDolI{5s0|4aDp3wvArA7@2(v}88{je z(KfPl(WBgv>(tF%bX?<*@^~Th0gdH$P+SA!!ngMsRj=VaRQADAh980vhFkT#k*=q2 zvm(J%TGQ2HaXS>@3Lv>8i{(59jV$A97q=ioqdnsx0)C0ORL$)D#R7XNZB{~Phn zL1jHgDDUuLLqBP#WjI(DY*~*rB0)yTjAtDAD?}tj0EI;C!=o3b(F6|_V0#z9~rhaeBD*;low z`6qAJc{ZkOuy38k8(k6fW}+LZR6xQnSl>nTMRj-aKW-O4urY5JAgSGHafP_ZDxc|% zbVbywR_}Og5<~Nq#rCY@V{n@xW9}pSuyi*I%cD#zx%N1+g8?x(|^9H`{-?RLlQT|mj ziPHj$yg4}rHBHV=z*G4Bmy;Lrw(-atu0CT8oN%v1eAYWor!n|z#cA0QkM&=&Lo{pS z`1#T6lcSEza^HWE6eTEA|FMdTvaqlJ{brKiF3Zfm&eQyi|9(@ptp1PvY309G{%hsG zR{nF#e&(B^TzdSlQ4U@F6J9di6PTiTK(?E=s zds1GS+>Yl-dR;gRh zP8@gNL`*`xh@(8t-yFYq?kI6Gxnkn$J9OyHZ(Rk7x#Q}8XBk+RoLzEEw__kX++rFQ zS51ZH+2z6CLy;XNfw_};${HTR43PQyr;MrFxt>mVDAPuT^NYiV(sX)u{zv|3>XA5@ zh_O(rw(gVJZRNjK{%hsG_VfQ%{&Q`s4=(==s`|hEy(jyv{P#6J?NV>`KkfeC?*Hxn z-|qj_{hywgznK1KFxao?f1d2N`~O$@D9e#D6}7gZcK>hp|91axKkfc+D8;|r{y+H6 q-v7U2#oPV=tG)kr!MC6G(|+1d`)NPzr~UjRz=)avAOisZ*z=bF