You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/04/14 12:19:28 UTC

[GitHub] [flink] syhily opened a new pull request, #19473: [FLINK-27231][Connector/Pulsar] Bump pulsar to 2.10.0

syhily opened a new pull request, #19473:
URL: https://github.com/apache/flink/pull/19473

   ## What is the purpose of the change
   
   Bump the Pulsar to 2.10.0 release which fixes a lot of issues on Pulsar transaction, especially the long waiting in Unit tests.
   
   ## Brief change log
   
     - Bump pulsar dependencies to 2.10.0
     - Fix the Pulsar mock instance which drops the zookeeper.
   
   ## Verifying this change
   
   This change is already covered by existing tests, such as All the Pulsar connector's unit tests.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): **(yes)**
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on a diff in pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on code in PR #19473:
URL: https://github.com/apache/flink/pull/19473#discussion_r913650218


##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java:
##########
@@ -128,6 +127,16 @@ private PulsarSinkOptions() {
                             "The allowed transaction recommit times if we meet some retryable exception."
                                     + " This is used in Pulsar Transaction.");
 
+    public static final ConfigOption<Integer> PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM =
+            ConfigOptions.key(SINK_CONFIG_PREFIX + "maxPendingMessages")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "The maximum number of pending messages in on sink parallelism.")

Review Comment:
   Yep. You are right. I'll change this typo.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] PatrickRen commented on a diff in pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
PatrickRen commented on code in PR #19473:
URL: https://github.com/apache/flink/pull/19473#discussion_r913448420


##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java:
##########
@@ -128,6 +127,16 @@ private PulsarSinkOptions() {
                             "The allowed transaction recommit times if we meet some retryable exception."
                                     + " This is used in Pulsar Transaction.");
 
+    public static final ConfigOption<Integer> PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM =
+            ConfigOptions.key(SINK_CONFIG_PREFIX + "maxPendingMessages")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "The maximum number of pending messages in on sink parallelism.")

Review Comment:
   "in **one** sink parallelism" I guess?



##########
docs/layouts/shortcodes/generated/pulsar_sink_configuration.html:
##########
@@ -20,6 +20,12 @@
             <td>Boolean</td>
             <td>If you enable this option and use PulsarSerializationSchema.pulsarSchema(), we would consume and deserialize the message by using Pulsar's <code class="highlighter-rouge">Schema</code>.</td>
         </tr>
+        <tr>
+            <td><h5>pulsar.sink.maxPendingMessages</h5></td>
+            <td style="word-wrap: break-word;">1000</td>
+            <td>Integer</td>
+            <td>The maximum number of pending messages in on sink parallelism.</td>

Review Comment:
   Here also "in **one** sink parallelism" I think



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on pull request #19473: [FLINK-27231][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1100619926

   @MartijnVisser This PR is ready and gets CI passed. Can you invite someone to review it?


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1174078355

   @PatrickRen I have splitted this PR into multiple commits.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] codelipenghui commented on a diff in pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #19473:
URL: https://github.com/apache/flink/pull/19473#discussion_r851822332


##########
flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh:
##########
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+# ----------------------------------------------------------------------------
+# 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.
+# ----------------------------------------------------------------------------
+
+# Enable the transaction in standalone config.
+sed -i 's/transactionCoordinatorEnabled=false/transactionCoordinatorEnabled=true/g' /pulsar/conf/standalone.conf
+sed -i 's/acknowledgmentAtBatchIndexLevelEnabled=false/acknowledgmentAtBatchIndexLevelEnabled=true/g' /pulsar/conf/standalone.conf
+sed -i 's/systemTopicEnabled=false/systemTopicEnabled=true/g' /pulsar/conf/standalone.conf

Review Comment:
   For enabling the transaction, it needs to enable `brokerDeduplicationEnabled=true` as well. The transaction depends on deduplication to avoid duplicated messages while the producer resends the messages due to connection issues or broker crashes.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] PatrickRen commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
PatrickRen commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1161370334

   Thanks for the patch @syhily. I notice that this PR touches a lot of components including Pulsar client version, docs even default value of sink options. What about breaking these changes into different commits or PRs? I'm afraid squashing them into one commit is way too ambiguous. 


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1169400631

   > Thanks for the patch @syhily. I notice that this PR touches a lot of components including Pulsar client version, docs even default value of sink options. What about breaking these changes into different commits or PRs? I'm afraid squashing them into one commit is way too ambiguous.
   
   OK, I'll split this PRs.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1165259847

   @syhily How quickly is it possible to improve this PR? If I'm not mistaken, this is the most important PR to get in since it should resolve the current test instabilities. In the current state that the Pulsar connector is in, it's threatening the stability of the Flink 1.16 release.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on a diff in pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on code in PR #19473:
URL: https://github.com/apache/flink/pull/19473#discussion_r851854474


##########
flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh:
##########
@@ -0,0 +1,27 @@
+#!/usr/bin/env bash
+# ----------------------------------------------------------------------------
+# 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.
+# ----------------------------------------------------------------------------
+
+# Enable the transaction in standalone config.
+sed -i 's/transactionCoordinatorEnabled=false/transactionCoordinatorEnabled=true/g' /pulsar/conf/standalone.conf
+sed -i 's/acknowledgmentAtBatchIndexLevelEnabled=false/acknowledgmentAtBatchIndexLevelEnabled=true/g' /pulsar/conf/standalone.conf
+sed -i 's/systemTopicEnabled=false/systemTopicEnabled=true/g' /pulsar/conf/standalone.conf

Review Comment:
   OK, I'll add it.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on pull request #19473: [FLINK-27231][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1099526008

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fapaul commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
fapaul commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1111056246

   @syhily, what is the current state of this PR? I know you have investigated some other failures in the mean, but the version bump also seems to be quite critical for the stability.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1174909574

   > The patch overall looks good to me except 2 typos I guess.
   
   Are you OK with merging this when these typos have been addressed? Would be great if we can get it merged today. 


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1174910440

   @PatrickRen The typo has been fixed.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] fapaul commented on a diff in pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
fapaul commented on code in PR #19473:
URL: https://github.com/apache/flink/pull/19473#discussion_r853812193


##########
docs/layouts/shortcodes/generated/pulsar_producer_configuration.html:
##########
@@ -56,18 +56,6 @@
             <td>Long</td>
             <td>The sequence id for avoiding the duplication, it's used when Pulsar doesn't have transaction.</td>
         </tr>
-        <tr>
-            <td><h5>pulsar.producer.maxPendingMessages</h5></td>

Review Comment:
   Please also use a separate commit for the restructuring of the docs if they are unrelated to the version bump.



##########
docs/layouts/shortcodes/generated/pulsar_client_configuration.html:
##########
@@ -100,7 +100,7 @@
         </tr>
         <tr>
             <td><h5>pulsar.client.memoryLimitBytes</h5></td>
-            <td style="word-wrap: break-word;">0</td>

Review Comment:
   I think this change deserves at least a separate commit. Is this changing the behavior or correcting the docs?



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java:
##########
@@ -128,6 +127,16 @@ private PulsarSinkOptions() {
                             "The allowed transaction recommit times if we meet some retryable exception."
                                     + " This is used in Pulsar Transaction.");
 
+    public static final ConfigOption<Integer> PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM =
+            ConfigOptions.key(SINK_CONFIG_PREFIX + "maxPendingMessages")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "The maximum number of pending messages in on sink parallelism.")

Review Comment:
   Can you explain this configuration value? The current description is hard to understand.



##########
flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java:
##########
@@ -50,22 +49,10 @@ static PulsarRuntime mock() {
         return new PulsarMockRuntime();
     }
 
-    /**
-     * Create a standalone Pulsar instance in test thread. We would start an embedded zookeeper and
-     * bookkeeper. The stream storage for bookkeeper is disabled. The function worker is disabled on
-     * Pulsar broker.
-     *
-     * <p>This runtime would be faster than {@link #container()} and behaves the same as the {@link
-     * #container()}.
-     */
-    static PulsarRuntime embedded() {

Review Comment:
   Has this something to do with the version bump? If not, please use a separate commit to remove the embedded test environment.



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java:
##########
@@ -128,6 +127,16 @@ private PulsarSinkOptions() {
                             "The allowed transaction recommit times if we meet some retryable exception."
                                     + " This is used in Pulsar Transaction.");
 
+    public static final ConfigOption<Integer> PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM =

Review Comment:
   Can you add this configuration on a separate commit?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on a diff in pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on code in PR #19473:
URL: https://github.com/apache/flink/pull/19473#discussion_r853829901


##########
docs/layouts/shortcodes/generated/pulsar_client_configuration.html:
##########
@@ -100,7 +100,7 @@
         </tr>
         <tr>
             <td><h5>pulsar.client.memoryLimitBytes</h5></td>
-            <td style="word-wrap: break-word;">0</td>

Review Comment:
   Yep



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] flinkbot commented on pull request #19473: [FLINK-27231][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1099132596

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "72aab7f0e930848e28b9a7ba9154b610f3141df9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "72aab7f0e930848e28b9a7ba9154b610f3141df9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 72aab7f0e930848e28b9a7ba9154b610f3141df9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1156706620

   @fapaul This is PR ready, I think.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on a diff in pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on code in PR #19473:
URL: https://github.com/apache/flink/pull/19473#discussion_r853830134


##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java:
##########
@@ -128,6 +127,16 @@ private PulsarSinkOptions() {
                             "The allowed transaction recommit times if we meet some retryable exception."
                                     + " This is used in Pulsar Transaction.");
 
+    public static final ConfigOption<Integer> PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM =
+            ConfigOptions.key(SINK_CONFIG_PREFIX + "maxPendingMessages")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "The maximum number of pending messages in on sink parallelism.")

Review Comment:
   OK



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1102354385

   @syhily The CI hasn't passed for this PR, please have a look


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] syhily commented on pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
syhily commented on PR #19473:
URL: https://github.com/apache/flink/pull/19473#issuecomment-1100673940

   @codelipenghui Can you help me review this PR to see if I'm correctly using the memory meta store and testing on the latest 2.10.0 Pulsar?


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink] MartijnVisser merged pull request #19473: [FLINK-27199][Connector/Pulsar] Bump pulsar to 2.10.0

Posted by GitBox <gi...@apache.org>.
MartijnVisser merged PR #19473:
URL: https://github.com/apache/flink/pull/19473


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org