AHeise commented on a change in pull request #15304: URL: https://github.com/apache/flink/pull/15304#discussion_r688394786
########## File path: flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java ########## @@ -0,0 +1,243 @@ +/* + * 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.source; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.common.config.ConfigurationDataCustomizer; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumStateSerializer; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumerator; +import org.apache.flink.connector.pulsar.source.enumerator.SplitsAssignmentState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import org.apache.flink.connector.pulsar.source.reader.PulsarSourceReaderFactory; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchemaInitializationContext; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; + +import static org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.createAdmin; +import static org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.createClient; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; + +/** + * The Source implementation of Pulsar. Please use a {@link PulsarSourceBuilder} to construct a + * {@link PulsarSource}. The following example shows how to create a PulsarSource emitting records + * of <code>String</code> type. + * + * <pre>{@code + * PulsarSource<String> source = PulsarSource + * .<String>builder() Review comment: We don't need `<String>` anymore. ########## File path: flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java ########## @@ -0,0 +1,265 @@ +/* + * 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.source; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.ClosureCleaner; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.common.config.ConfigurationDataCustomizer; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumStateSerializer; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumerator; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import org.apache.flink.connector.pulsar.source.reader.PulsarSourceReaderFactory; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarSchemaInitializationContext; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.function.SerializableSupplier; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; + +import static org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.createAdmin; +import static org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.createClient; +import static org.apache.flink.connector.pulsar.common.exception.PulsarExceptionUtils.sneaky; + +/** + * The Source implementation of Pulsar. Please use a {@link PulsarSourceBuilder} to construct a + * {@link PulsarSource}. The following example shows how to create a PulsarSource emitting records + * of <code>String</code> type. + * + * <pre>{@code + * PulsarSource<byte[], String> source = PulsarSource.<byte[], String>builder() + * .setTopics(TOPIC1, TOPIC2) + * .setServiceUrl(getServiceUrl()) + * .setAdminUrl(getAdminUrl()) + * .setSubscriptionName("test") + * .setDeserializationSchema(flinkSchema(new SimpleStringSchema())) + * .setBounded(StopCursor::defaultStopCursor) + * .build(); + * }</pre> + * + * <p>See {@link PulsarSourceBuilder} for more details. + * + * @param <IN> The input type of the pulsar {@link Message}. + * @param <OUT> The output type of the source. + */ +@PublicEvolving +public final class PulsarSource<IN, OUT> + implements Source<OUT, PulsarPartitionSplit, PulsarSourceEnumState>, + ResultTypeQueryable<OUT> { + private static final long serialVersionUID = 7773108631275567433L; + + /** + * The common configuration for pulsar source, we don't support the pulsar's configuration class + * directly. + */ + private final Configuration configuration; + + private final SourceConfiguration sourceConfiguration; + + private final PulsarSubscriber subscriber; + + private final RangeGenerator rangeGenerator; + + private final SerializableSupplier<StartCursor> startCursorSupplier; + + private final SerializableSupplier<StopCursor> stopCursorSupplier; + + /** + * Boundedness for source, we only support {@link Boundedness#CONTINUOUS_UNBOUNDED} currently. + */ + private final Boundedness boundedness; + + /** The pulsar deserialization schema used for deserialize message. */ + private final PulsarDeserializationSchema<IN, OUT> deserializationSchema; + + /** Modify the flink generated {@link ClientConfigurationData}. */ + private final ConfigurationDataCustomizer<ClientConfigurationData> + clientConfigurationCustomizer; + + /** Modify the flink generated {@link ConsumerConfigurationData}. */ + private final ConfigurationDataCustomizer<ConsumerConfigurationData<IN>> + consumerConfigurationCustomizer; + + /** A lazy evaluated pulsar client for split reader. */ + private transient volatile PulsarClient pulsarClient; + + /** A lazy evaluated pulsar admin for source enumerator. */ + private transient volatile PulsarAdmin pulsarAdmin; Review comment: This is still open. Next to being too complicated it also brings some potential issues with lifecycle. Just assuming for a second that we would create multiple SourceReader and enumerators from the same Source instance (we don't), what happens when you close one reader/enumerator? You would also close the (admin) client leaving the other other readers in some limbo state. So please, just remove the fields and directly pass to them into the constructor or even just move the creation entirely into the Reader/Enumerator. -- 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]
