dianfu commented on code in PR #27409: URL: https://github.com/apache/flink/pull/27409#discussion_r2689998274
########## flink-python/pyflink/datastream/connectors/dynamic_kafka.py: ########## @@ -0,0 +1,237 @@ +################################################################################ +# 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. +################################################################################ +from typing import Dict, Set, Union + +from py4j.java_gateway import JavaObject + +from pyflink.common import DeserializationSchema +from pyflink.datastream.connectors import Source +from pyflink.datastream.connectors.kafka import KafkaOffsetsInitializer +from pyflink.java_gateway import get_gateway + +__all__ = [ + 'DynamicKafkaSource', + 'DynamicKafkaSourceBuilder', + 'KafkaMetadataService', + 'KafkaRecordDeserializationSchema', + 'KafkaStreamSubscriber', + 'SingleClusterTopicMetadataService' +] + + +class KafkaMetadataService(object): + """ + Base class for Kafka metadata service wrappers. + """ + + def __init__(self, j_metadata_service: JavaObject): + self._j_metadata_service = j_metadata_service + + +class SingleClusterTopicMetadataService(KafkaMetadataService): + """ + A KafkaMetadataService backed by a single Kafka cluster where stream ids map to topics. + """ + + def __init__(self, kafka_cluster_id: str, properties: Dict[str, str]): + gateway = get_gateway() + j_properties = gateway.jvm.java.util.Properties() + for key, value in properties.items(): + j_properties.setProperty(key, value) + j_service = gateway.jvm.org.apache.flink.connector.kafka.dynamic.metadata \ + .SingleClusterTopicMetadataService(kafka_cluster_id, j_properties) + super().__init__(j_service) + + +class KafkaStreamSubscriber(object): Review Comment: What about also introducing class `StreamPatternSubscriber` and `KafkaStreamSetSubscriber `? ########## flink-python/pyflink/datastream/connectors/dynamic_kafka.py: ########## @@ -0,0 +1,237 @@ +################################################################################ +# 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. +################################################################################ +from typing import Dict, Set, Union + +from py4j.java_gateway import JavaObject + +from pyflink.common import DeserializationSchema +from pyflink.datastream.connectors import Source +from pyflink.datastream.connectors.kafka import KafkaOffsetsInitializer +from pyflink.java_gateway import get_gateway + +__all__ = [ Review Comment: It would be great to also update the documentation here: https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/dynamic-kafka/ -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
