syhily commented on a change in pull request #17452: URL: https://github.com/apache/flink/pull/17452#discussion_r794656612
########## File path: flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java ########## @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink.Sink.ProcessingTimeService; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; + +import org.apache.flink.shaded.guava30.com.google.common.base.Objects; +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.emptyList; +import static org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.createAdmin; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.isPartitioned; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; + +/** + * We need the latest topic metadata for making sure the newly created topic partitions would be + * used by Pulsar sink. This routing policy would be different comparing with Pulsar Client built-in + * logic. We use flink's ProcessingTimer as the executor. + */ +@Internal +public class TopicMetadataListener implements Serializable, Closeable { + private static final long serialVersionUID = 6186948471557507522L; + + private static final Logger LOG = LoggerFactory.getLogger(TopicMetadataListener.class); + + private final List<String> partitionedTopics; + private final Map<String, Integer> topicMetadata; + private volatile ImmutableList<String> availableTopics; + + // Dynamic fields. + private transient PulsarAdmin pulsarAdmin; + private transient Long topicMetadataRefreshInterval; + private transient ProcessingTimeService timeService; + + public TopicMetadataListener() { + this(emptyList()); + } + + public TopicMetadataListener(List<String> topics) { + this.partitionedTopics = new ArrayList<>(topics.size()); + this.topicMetadata = new HashMap<>(topics.size()); + this.availableTopics = ImmutableList.of(); + + for (String topic : topics) { + if (isPartitioned(topic)) { + partitionedTopics.add(topic); + } else { + // This would be updated when open writing. + topicMetadata.put(topic, -1); + } + } + } + + /** Register the topic metadata update in process time service. */ + public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService) { + if (topicMetadata.isEmpty()) { + LOG.info("No topics have been provided, skip listener initialize."); + return; + } + + // Initialize listener properties. + this.pulsarAdmin = createAdmin(sinkConfiguration); + this.topicMetadataRefreshInterval = sinkConfiguration.getTopicMetadataRefreshInterval(); + this.timeService = timeService; + + // Initialize the topic metadata. Quit if fail to connect to Pulsar. + sneakyAdmin(this::updateTopicMetadata); + + // Register time service. + triggerNextTopicMetadataUpdate(true); + } + + /** + * Return all the available topic partitions. We would recalculate the partitions if the topic + * metadata hsa been changed. Otherwise, we would return the cached result for better + * performance. + */ + public List<String> availableTopics() { + if (availableTopics.isEmpty() + && (!partitionedTopics.isEmpty() || !topicMetadata.isEmpty())) { Review comment: I should point out your mistake. We only evaluate the `availableTopics` when it needs updating. This is a cached result, which shouldn't be created every time we call this method. Putting the evaluate logic here is just my design flavor. XD -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org