@@ -474,12 +474,14 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
474
474
rd_kafka_metadata_internal_t * mdi = NULL ;
475
475
rd_kafka_metadata_t * md = NULL ;
476
476
size_t rkb_namelen ;
477
- const int log_decode_errors = LOG_ERR ;
478
- rd_list_t * missing_topics = NULL ;
479
-
480
- const rd_list_t * requested_topics = request_topics ;
481
- rd_bool_t all_topics = rd_false ;
482
- rd_bool_t cgrp_update = rd_false ;
477
+ const int log_decode_errors = LOG_ERR ;
478
+ rd_list_t * missing_topics = NULL ;
479
+ rd_list_t * missing_topic_ids = NULL ;
480
+
481
+ const rd_list_t * requested_topics = request_topics ;
482
+ const rd_list_t * requested_topic_ids = NULL ;
483
+ rd_bool_t all_topics = rd_false ;
484
+ rd_bool_t cgrp_update = rd_false ;
483
485
rd_bool_t has_reliable_leader_epochs =
484
486
rd_kafka_has_reliable_leader_epochs (rkb );
485
487
int ApiVersion = rkbuf -> rkbuf_reqhdr .ApiVersion ;
@@ -496,8 +498,9 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
496
498
rd_bool_t compute_racks = has_client_rack ;
497
499
498
500
if (request ) {
499
- requested_topics = request -> rkbuf_u .Metadata .topics ;
500
- all_topics = request -> rkbuf_u .Metadata .all_topics ;
501
+ requested_topics = request -> rkbuf_u .Metadata .topics ;
502
+ requested_topic_ids = request -> rkbuf_u .Metadata .topic_ids ;
503
+ all_topics = request -> rkbuf_u .Metadata .all_topics ;
501
504
cgrp_update =
502
505
request -> rkbuf_u .Metadata .cgrp_update && rk -> rk_cgrp ;
503
506
compute_racks |= request -> rkbuf_u .Metadata .force_racks ;
@@ -519,6 +522,9 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
519
522
if (requested_topics )
520
523
missing_topics =
521
524
rd_list_copy (requested_topics , rd_list_string_copy , NULL );
525
+ if (requested_topic_ids )
526
+ missing_topic_ids =
527
+ rd_list_copy (requested_topic_ids , rd_list_Uuid_copy , NULL );
522
528
523
529
rd_kafka_broker_lock (rkb );
524
530
rkb_namelen = strlen (rkb -> rkb_name ) + 1 ;
@@ -833,34 +839,37 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
833
839
rd_kafka_parse_Metadata_update_topic (rkb , & md -> topics [i ],
834
840
& mdi -> topics [i ]);
835
841
836
- // TODO: Should be done for requested_topic_ids as well.
837
- if (requested_topics ) {
842
+ if (requested_topics )
838
843
rd_list_free_cb (missing_topics ,
839
844
rd_list_remove_cmp (missing_topics ,
840
845
md -> topics [i ].topic ,
841
846
(void * )strcmp ));
842
- if (!all_topics ) {
843
- /* Only update cache when not asking
844
- * for all topics. */
845
-
846
- rd_kafka_wrlock (rk );
847
- rd_kafka_metadata_cache_topic_update (
848
- rk , & md -> topics [i ], & mdi -> topics [i ],
849
- rd_false /*propagate later*/ ,
850
- /* use has_client_rack rather than
851
- compute_racks. We need cached rack ids
852
- only in case we need to rejoin the group
853
- if they change and client.rack is set
854
- (KIP-881). */
855
- has_client_rack , mdi -> brokers ,
856
- md -> broker_cnt );
857
- cache_changes ++ ;
858
- rd_kafka_wrunlock (rk );
859
- }
860
- }
847
+ if (requested_topic_ids )
848
+ rd_list_free_cb (
849
+ missing_topic_ids ,
850
+ rd_list_remove_cmp (missing_topic_ids ,
851
+ & mdi -> topics [i ].topic_id ,
852
+ (void * )rd_kafka_Uuid_ptr_cmp ));
853
+ /* Only update cache when not asking
854
+ * for all topics or cache entry
855
+ * already exists. */
856
+ rd_kafka_wrlock (rk );
857
+ cache_changes +=
858
+ rd_kafka_metadata_cache_topic_update (
859
+ rk , & md -> topics [i ], & mdi -> topics [i ],
860
+ rd_false /*propagate later*/ ,
861
+ /* use has_client_rack rather than
862
+ compute_racks. We need cached rack ids
863
+ only in case we need to rejoin the group
864
+ if they change and client.rack is set
865
+ (KIP-881). */
866
+ has_client_rack , mdi -> brokers ,
867
+ md -> broker_cnt ,
868
+ all_topics /*cache entry needs to exist
869
+ *if all_topics*/ );
870
+ rd_kafka_wrunlock (rk );
861
871
}
862
872
863
- // TODO: Should be done for missing_topic_ids as well.
864
873
/* Requested topics not seen in metadata? Propogate to topic code. */
865
874
if (missing_topics ) {
866
875
char * topic ;
@@ -892,6 +901,41 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
892
901
}
893
902
}
894
903
}
904
+ if (missing_topic_ids ) {
905
+ rd_kafka_Uuid_t * topic_id ;
906
+ rd_rkb_dbg (rkb , TOPIC , "METADATA" ,
907
+ "%d/%d requested topic(s) seen in metadata" ,
908
+ rd_list_cnt (requested_topic_ids ) -
909
+ rd_list_cnt (missing_topic_ids ),
910
+ rd_list_cnt (requested_topic_ids ));
911
+ for (i = 0 ; i < rd_list_cnt (missing_topic_ids ); i ++ ) {
912
+ rd_kafka_Uuid_t * missing_topic_id =
913
+ missing_topic_ids -> rl_elems [i ];
914
+ rd_rkb_dbg (rkb , TOPIC , "METADATA" , "wanted %s" ,
915
+ rd_kafka_Uuid_base64str (missing_topic_id ));
916
+ }
917
+ RD_LIST_FOREACH (topic_id , missing_topic_ids , i ) {
918
+ rd_kafka_topic_t * rkt ;
919
+
920
+ rd_kafka_rdlock (rk );
921
+ rkt = rd_kafka_topic_find_by_topic_id (rkb -> rkb_rk ,
922
+ * topic_id );
923
+ rd_kafka_rdunlock (rk );
924
+ if (rkt ) {
925
+ /* Received metadata response contained no
926
+ * information about topic 'rkt' and thus
927
+ * indicates the topic is not available in the
928
+ * cluster.
929
+ * Mark the topic as non-existent */
930
+ rd_kafka_topic_wrlock (rkt );
931
+ rd_kafka_topic_set_notexists (
932
+ rkt , RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC );
933
+ rd_kafka_topic_wrunlock (rkt );
934
+
935
+ rd_kafka_topic_destroy0 (rkt );
936
+ }
937
+ }
938
+ }
895
939
896
940
897
941
rd_kafka_wrlock (rkb -> rkb_rk );
@@ -956,17 +1000,18 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
956
1000
"Caching full metadata with "
957
1001
"%d broker(s) and %d topic(s): %s" ,
958
1002
md -> broker_cnt , md -> topic_cnt , reason );
959
- } else {
960
- if (cache_changes )
961
- rd_kafka_metadata_cache_propagate_changes (rk );
962
- rd_kafka_metadata_cache_expiry_start (rk );
963
1003
}
964
1004
1005
+ if (cache_changes ) {
1006
+ rd_kafka_metadata_cache_propagate_changes (rk );
1007
+ }
1008
+ rd_kafka_metadata_cache_expiry_start (rk );
965
1009
966
- // TODO: Should be done for requested_topic_ids as well.
967
1010
/* Remove cache hints for the originally requested topics. */
968
1011
if (requested_topics )
969
1012
rd_kafka_metadata_cache_purge_hints (rk , requested_topics );
1013
+ if (requested_topic_ids )
1014
+ rd_kafka_metadata_cache_purge_hints (rk , requested_topic_ids );
970
1015
971
1016
rd_kafka_wrunlock (rkb -> rkb_rk );
972
1017
@@ -982,7 +1027,8 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
982
1027
* which may contain only a sub-set of the subscribed topics (namely
983
1028
* the effective subscription of available topics) as to not
984
1029
* propagate non-included topics as non-existent. */
985
- if (cgrp_update && (requested_topics || all_topics ))
1030
+ if (cgrp_update &&
1031
+ (requested_topics || requested_topic_ids || all_topics ))
986
1032
rd_kafka_cgrp_metadata_update_check (rkb -> rkb_rk -> rk_cgrp ,
987
1033
rd_true /*do join*/ );
988
1034
@@ -995,10 +1041,10 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
995
1041
}
996
1042
997
1043
done :
998
-
999
- // TODO: Should be done for requested_topic_ids as well.
1000
1044
if (missing_topics )
1001
1045
rd_list_destroy (missing_topics );
1046
+ if (missing_topic_ids )
1047
+ rd_list_destroy (missing_topic_ids );
1002
1048
1003
1049
/* This metadata request was triggered by someone wanting
1004
1050
* the metadata information back as a reply, so send that reply now.
@@ -1013,18 +1059,26 @@ rd_kafka_parse_Metadata0(rd_kafka_broker_t *rkb,
1013
1059
err_parse :
1014
1060
err = rkbuf -> rkbuf_err ;
1015
1061
err :
1016
- // TODO: Should be done for requested_topic_ids as well.
1017
1062
if (requested_topics ) {
1018
1063
/* Failed requests shall purge cache hints for
1019
1064
* the requested topics. */
1020
1065
rd_kafka_wrlock (rkb -> rkb_rk );
1021
1066
rd_kafka_metadata_cache_purge_hints (rk , requested_topics );
1022
1067
rd_kafka_wrunlock (rkb -> rkb_rk );
1023
1068
}
1069
+ if (requested_topic_ids ) {
1070
+ /* Failed requests shall purge cache hints for
1071
+ * the requested topics. */
1072
+ rd_kafka_wrlock (rkb -> rkb_rk );
1073
+ rd_kafka_metadata_cache_purge_hints_by_id (rk ,
1074
+ requested_topic_ids );
1075
+ rd_kafka_wrunlock (rkb -> rkb_rk );
1076
+ }
1024
1077
1025
- // TODO: Should be done for requested_topic_ids as well.
1026
1078
if (missing_topics )
1027
1079
rd_list_destroy (missing_topics );
1080
+ if (missing_topic_ids )
1081
+ rd_list_destroy (missing_topic_ids );
1028
1082
rd_tmpabuf_destroy (& tbuf );
1029
1083
1030
1084
return err ;
0 commit comments