imaffe commented on a change in pull request #17452:
URL: https://github.com/apache/flink/pull/17452#discussion_r793289032



##########
File path: 
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/serializer/PulsarSchemaWrapper.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.serializer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.pulsar.common.schema.PulsarSchema;
+import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext;
+import org.apache.flink.connector.pulsar.sink.writer.message.RawMessage;
+
+import org.apache.pulsar.client.api.Schema;
+
+/** Wrap the Pulsar's Schema into PulsarSerializationSchema. */
+@Internal
+public class PulsarSchemaWrapper<IN> implements PulsarSerializationSchema<IN> {
+    private static final long serialVersionUID = -2567052498398184194L;
+
+    private static final byte[] EMPTY_BYTES = new byte[0];
+    private final PulsarSchema<IN> pulsarSchema;
+
+    public PulsarSchemaWrapper(PulsarSchema<IN> pulsarSchema) {
+        this.pulsarSchema = pulsarSchema;
+    }
+
+    @Override
+    public RawMessage<byte[]> serialize(IN element, PulsarSinkContext 
sinkContext) {
+        RawMessage<byte[]> message;
+
+        if (sinkContext.isEnableSchemaEvolution()) {
+            // We don't need to serialize incoming records in schema evolution.
+            message = new RawMessage<>(EMPTY_BYTES);
+        } else {
+            Schema<IN> schema = this.pulsarSchema.getPulsarSchema();
+            byte[] bytes = schema.encode(element);
+            message = new RawMessage<>(bytes);
+        }
+
+        Long eventTime = sinkContext.timestamp();

Review comment:
       Here we put the event time in when we do the serialization,  wondering 
what is the design considerations here?  Asking because I feel like adding 
event time can happen in a different step

##########
File path: 
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/committer/PulsarCommitter.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.committer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.pulsar.sink.PulsarSink;
+import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration;
+
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient;
+import 
org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException;
+import 
org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException.CoordinatorNotFoundException;
+import 
org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException.TransactionNotFoundException;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.PulsarClientImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.util.Collections.emptyList;
+import static 
org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.createClient;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Committer implementation for {@link PulsarSink}.
+ *
+ * <p>The committer is responsible to finalize the Pulsar transactions by 
committing them.
+ */
+@Internal
+public class PulsarCommitter implements Committer<PulsarCommittable>, 
Closeable {
+    private static final Logger LOG = 
LoggerFactory.getLogger(PulsarCommitter.class);
+
+    private final SinkConfiguration sinkConfiguration;
+
+    private PulsarClient pulsarClient;
+    private TransactionCoordinatorClient coordinatorClient;
+
+    public PulsarCommitter(SinkConfiguration sinkConfiguration) {
+        this.sinkConfiguration = sinkConfiguration;
+    }
+
+    @Override
+    public List<PulsarCommittable> commit(List<PulsarCommittable> committables)
+            throws IOException, InterruptedException {
+        if (committables == null || committables.isEmpty()) {
+            return emptyList();
+        }
+
+        List<PulsarCommittable> retryableCommittables = new ArrayList<>();
+        for (PulsarCommittable committable : committables) {
+            TxnID txnID = committable.getTxnID();
+            String topic = committable.getTopic();
+
+            LOG.debug("Start committing the transaction {} for topic {}", 
txnID, topic);
+            try {
+                coordinatorClient.commit(txnID);
+            } catch (TransactionNotFoundException | 
CoordinatorNotFoundException e) {

Review comment:
       For these 2 exceptions, why do we rethrow it ? Which class is the final 
handlers of these two exceptions ? Curious because I saw this is the only place 
we catch these 2 exceptions. I guess these 2 are nonretrayble transaction 
failures and should be treated to stop pipeline ?

##########
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:
       Just out of curiousity, we evaluate availableTopics when we call it 
(evaluate: retrieving data from topicMetadata) in this method instead of in 
updateTopicMetadata(), is there any special reason to do so ?

##########
File path: 
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/committer/PulsarCommitter.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.committer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.pulsar.sink.PulsarSink;
+import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration;
+
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient;
+import 
org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException;
+import 
org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException.CoordinatorNotFoundException;
+import 
org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException.TransactionNotFoundException;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.PulsarClientImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.util.Collections.emptyList;
+import static 
org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.createClient;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Committer implementation for {@link PulsarSink}.
+ *
+ * <p>The committer is responsible to finalize the Pulsar transactions by 
committing them.
+ */
+@Internal
+public class PulsarCommitter implements Committer<PulsarCommittable>, 
Closeable {
+    private static final Logger LOG = 
LoggerFactory.getLogger(PulsarCommitter.class);
+
+    private final SinkConfiguration sinkConfiguration;
+
+    private PulsarClient pulsarClient;
+    private TransactionCoordinatorClient coordinatorClient;
+
+    public PulsarCommitter(SinkConfiguration sinkConfiguration) {
+        this.sinkConfiguration = sinkConfiguration;
+    }
+
+    @Override
+    public List<PulsarCommittable> commit(List<PulsarCommittable> committables)
+            throws IOException, InterruptedException {
+        if (committables == null || committables.isEmpty()) {
+            return emptyList();
+        }
+
+        List<PulsarCommittable> retryableCommittables = new ArrayList<>();
+        for (PulsarCommittable committable : committables) {
+            TxnID txnID = committable.getTxnID();
+            String topic = committable.getTopic();
+
+            LOG.debug("Start committing the transaction {} for topic {}", 
txnID, topic);
+            try {
+                coordinatorClient.commit(txnID);
+            } catch (TransactionNotFoundException | 
CoordinatorNotFoundException e) {
+                throw e;
+            } catch (TransactionCoordinatorClientException e) {
+                LOG.error("Unable to commit transaction {} for topic {}", 
txnID, topic);
+                retryableCommittables.add(committable);
+            }
+        }
+        return retryableCommittables;
+    }
+
+    /**
+     * Lazy initialize this backend Pulsar client. This committer may not be 
used in {@link
+     * DeliveryGuarantee#NONE} and {@link DeliveryGuarantee#AT_LEAST_ONCE}. So 
we couldn't create
+     * the Pulsar client immediately.
+     */
+    private TransactionCoordinatorClient transactionCoordinatorClient() {

Review comment:
       This method is not used, will it be used in the future ? 

##########
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

Review comment:
       nit: typo hsa  in the comment




-- 
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]


Reply via email to