Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-26 Thread via GitHub


mayuehappy commented on code in PR #23765:
URL: https://github.com/apache/flink/pull/23765#discussion_r1405583287


##
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java:
##
@@ -350,6 +359,10 @@ private long uploadSnapshotFiles(
 ? CheckpointedStateScope.EXCLUSIVE
 : CheckpointedStateScope.SHARED;
 
+if (stateFileVerifier != null) {
+stateFileVerifier.verifySstFilesChecksum(sstFilePaths);
+}
+
 List sstFilesUploadResult =
 stateUploader.uploadFilesToCheckpointFs(

Review Comment:
   > > most files are damaged due to hardware failures on the machine where the 
file is written
   > 
   > Check with you first, if there is hardware failures after the file is 
written,, doesn't this issue happen?
   
   Well, just like I replied to you in the ticket, I think this possibility is 
very small. We can continue the discussion in the ticket and then come back to 
continue reviewing this PR.



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



Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-24 Thread via GitHub


1996fanrui commented on code in PR #23765:
URL: https://github.com/apache/flink/pull/23765#discussion_r1404180821


##
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java:
##
@@ -350,6 +359,10 @@ private long uploadSnapshotFiles(
 ? CheckpointedStateScope.EXCLUSIVE
 : CheckpointedStateScope.SHARED;
 
+if (stateFileVerifier != null) {
+stateFileVerifier.verifySstFilesChecksum(sstFilePaths);
+}
+
 List sstFilesUploadResult =
 stateUploader.uploadFilesToCheckpointFs(

Review Comment:
   > most files are damaged due to hardware failures on the machine where the 
file is written
   
   Check with you first, if there is hardware failures after the file is 
written,, doesn't this issue happen?



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



Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-24 Thread via GitHub


1996fanrui commented on code in PR #23765:
URL: https://github.com/apache/flink/pull/23765#discussion_r1404180821


##
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java:
##
@@ -350,6 +359,10 @@ private long uploadSnapshotFiles(
 ? CheckpointedStateScope.EXCLUSIVE
 : CheckpointedStateScope.SHARED;
 
+if (stateFileVerifier != null) {
+stateFileVerifier.verifySstFilesChecksum(sstFilePaths);
+}
+
 List sstFilesUploadResult =
 stateUploader.uploadFilesToCheckpointFs(

Review Comment:
   > most files are damaged due to hardware failures on the machine where the 
file is written
   
   Check with you first, if there is hardware failures after the file is 
written,, doesn't it happen?



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



Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-24 Thread via GitHub


mayuehappy commented on code in PR #23765:
URL: https://github.com/apache/flink/pull/23765#discussion_r1404175347


##
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java:
##
@@ -350,6 +359,10 @@ private long uploadSnapshotFiles(
 ? CheckpointedStateScope.EXCLUSIVE
 : CheckpointedStateScope.SHARED;
 
+if (stateFileVerifier != null) {
+stateFileVerifier.verifySstFilesChecksum(sstFilePaths);
+}
+
 List sstFilesUploadResult =
 stateUploader.uploadFilesToCheckpointFs(

Review Comment:
   
   
   
   > From the code order, we verify the file first, and then 
`uploadFilesToCheckpointFs`.
   > 
   > If the file is corrupted before calling `hdfs.write()`, this check still 
cannot cover, right?
   
   
   Yes ,  If the file is corrupted before calling `hdfs.write()`, this check 
still cannot cover
   
   But as I understand,  After the file is hardlinked to snapshotDir, no other 
operations will change the file. So there should be no file corruption during 
this process.
   
   I think there are only two places where corruption may occur, one is when 
the file is created, and the other is when it is re-downloaded from Checkpoint 
to the local disk.
   



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



Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-24 Thread via GitHub


1996fanrui commented on code in PR #23765:
URL: https://github.com/apache/flink/pull/23765#discussion_r1404136470


##
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java:
##
@@ -350,6 +359,10 @@ private long uploadSnapshotFiles(
 ? CheckpointedStateScope.EXCLUSIVE
 : CheckpointedStateScope.SHARED;
 
+if (stateFileVerifier != null) {
+stateFileVerifier.verifySstFilesChecksum(sstFilePaths);
+}
+
 List sstFilesUploadResult =
 stateUploader.uploadFilesToCheckpointFs(

Review Comment:
   From the code order, we verify the file first, and then 
`uploadFilesToCheckpointFs`.
   
   If the file is corrupted before calling `hdfs.write()` and after 
`verifySstFilesChecksum`, this check still doesn't work, right?



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



Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-24 Thread via GitHub


1996fanrui commented on code in PR #23765:
URL: https://github.com/apache/flink/pull/23765#discussion_r1404119092


##
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java:
##
@@ -0,0 +1,124 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.util.FileUtils;
+
+import org.junit.Assert;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.rocksdb.Checkpoint;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteOptions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX;
+import static org.junit.Assert.fail;
+
+/** {@link RocksDBStateFileVerifier} test. */
+public class RocksdbStateFileVerifierTest {
+
+@TempDir java.nio.file.Path folder;
+
+@Test
+public void rocksdbStateFileVerifierTest() throws Exception {
+List columnFamilyHandles = new ArrayList<>(1);

Review Comment:
   ```suggestion
   List columnFamilyHandles = new ArrayList<>(1);
   ```



##
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java:
##
@@ -0,0 +1,124 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.util.FileUtils;
+
+import org.junit.Assert;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import org.rocksdb.Checkpoint;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteOptions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX;
+import static org.junit.Assert.fail;
+
+/** {@link RocksDBStateFileVerifier} test. */
+public class RocksdbStateFileVerifierTest {
+
+@TempDir java.nio.file.Path folder;
+
+@Test
+public void rocksdbStateFileVerifierTest() throws Exception {

Review Comment:
   ```suggestion
   void rocksdbStateFileVerifierTest() throws Exception {
   ```
   
   public isn't needed for junit5.



##
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategyTest.java:
##
@@ -96,6 +105,46 @@ void testCheckpointIsIncremental() throws Exception {
 }
 }
 
+@Test
+void testCheckpointFailIfSstFileCorrupted() throws Exception {
+
+try (CloseableRegistry closeableRegistry = new CloseableRegistry();
+  

Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-24 Thread via GitHub


mayuehappy commented on PR #23765:
URL: https://github.com/apache/flink/pull/23765#issuecomment-1825308761

   > By the way, the commit message should include the [module name], such as : 
`[FLINK-27681][rocksdb] RocksdbStatebackend verify incremental sst file 
checksum during checkpoint`
   
   @1996fanrui Thanks for the review , I have updated the PR with your 
suggestions , pls take 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-unsubscr...@flink.apache.org

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



Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-23 Thread via GitHub


1996fanrui commented on code in PR #23765:
URL: https://github.com/apache/flink/pull/23765#discussion_r1404014532


##
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java:
##
@@ -86,6 +86,18 @@ public class RocksDBOptions {
 .withDescription(
 "The number of threads (per stateful operator) 
used to transfer (download and upload) files in RocksDBStateBackend.");
 
+/**
+ * Whether to verify the Checksum of the incremental sst file during 
Checkpoint in
+ * RocksDBStateBackend.
+ */
+@Documentation.Section(Documentation.Sections.EXPERT_ROCKSDB)
+public static final ConfigOption 
CHECKPOINT_VERIFY_CHECKSUM_ENABLE =
+
ConfigOptions.key("state.backend.rocksdb.checkpoint.verify.checksum.enable")

Review Comment:
   ```suggestion
   
ConfigOptions.key("state.backend.rocksdb.checkpoint.verify.checksum.enabled")
   ```



##
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java:
##
@@ -0,0 +1,125 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.util.FileUtils;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;

Review Comment:
   The new test should use `import org.junit.jupiter.api.Test;` instead of 
`org.junit.Test`.



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



Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-23 Thread via GitHub


mayuehappy commented on PR #23765:
URL: https://github.com/apache/flink/pull/23765#issuecomment-1825104163

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

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



Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-23 Thread via GitHub


mayuehappy commented on PR #23765:
URL: https://github.com/apache/flink/pull/23765#issuecomment-1825103757

   @masteryhx Thanks for the code review , I have updated the MR , PTAL


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



Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-23 Thread via GitHub


masteryhx commented on code in PR #23765:
URL: https://github.com/apache/flink/pull/23765#discussion_r1403408187


##
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksNativeFullSnapshotStrategy.java:
##
@@ -80,7 +86,8 @@ public RocksNativeFullSnapshotStrategy(
 @Nonnull LocalRecoveryConfig localRecoveryConfig,
 @Nonnull File instanceBasePath,
 @Nonnull UUID backendUID,
-@Nonnull RocksDBStateUploader rocksDBStateUploader) {
+@Nonnull RocksDBStateUploader rocksDBStateUploader,
+RocksDBStateFileVerifier stateFileVerifier) {

Review Comment:
   nit: Nullable Annotation



##
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java:
##
@@ -86,6 +86,18 @@ public class RocksDBOptions {
 .withDescription(
 "The number of threads (per stateful operator) 
used to transfer (download and upload) files in RocksDBStateBackend.");
 
+/**
+ * Whether to verify the Checksum of the incremental sst file during 
Checkpoint in
+ * RocksDBStateBackend.
+ */
+@Documentation.Section(Documentation.Sections.EXPERT_ROCKSDB)
+public static final ConfigOption 
CHECKPOINT_VERIFY_CHECKSUM_ENABLE =
+
ConfigOptions.key("state.backend.rocksdb.checkpoint.verify.checksum.enable")
+.booleanType()
+.defaultValue(false)
+.withDescription(
+"Whether to verify the Checksum of the incremental 
sst file during Checkpoint in RocksDBStateBackend");

Review Comment:
   Could we also add some important messages here:
   1. It may introduce some overhead for checkpoint procedure if enable.
   2. If checksum fail, we will fail the the checkpoint.



##
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksdbStateFileVerifierTest.java:
##
@@ -0,0 +1,125 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.util.FileUtils;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.Checkpoint;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteOptions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.SST_FILE_SUFFIX;
+import static org.junit.Assert.fail;
+
+/** {@link RocksDBStateFileVerifier} test. */
+public class RocksdbStateFileVerifierTest {
+
+@Rule public TemporaryFolder folder = new TemporaryFolder();
+
+@Test
+public void rocksdbStateFileVerifierTest() throws Exception {
+ArrayList columnFamilyHandles = new ArrayList<>(1);
+String rootPath = folder.newFolder().getAbsolutePath();
+File dbPath = new File(rootPath, "db");
+File cpPath = new File(rootPath, "cp");
+
+try (DBOptions dbOptions = new DBOptions().setCreateIfMissing(true);
+ColumnFamilyOptions colOptions = new ColumnFamilyOptions();
+Options sstFileReaderOptions = new Options(dbOptions, 
colOptions);
+WriteOptions writeOptions = new 
WriteOptions().setDisableWAL(true);
+RocksDB db =
+RocksDB.open(
+dbOptions,
+dbPath.toString(),
+Collections.singletonList(
+new ColumnFamilyDescriptor(
+"default".getBytes(), 
colOptions)),
+

Re: [PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-21 Thread via GitHub


flinkbot commented on PR #23765:
URL: https://github.com/apache/flink/pull/23765#issuecomment-1820555627

   
   ## CI report:
   
   * 31d21dd4a5808ddae5eb61b1580fb13868f6ab2a UNKNOWN
   
   
   Bot commands
 The @flinkbot bot supports the following commands:
   
- `@flinkbot run azure` re-run the last Azure build
   


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



[PR] [FLINK-27681] RocksdbStatebackend verify incremental sst file checksum during checkpoint [flink]

2023-11-21 Thread via GitHub


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

   
   ## What is the purpose of the change
   
   RocksdbStatebackend verify incremental sst file checksum during checkpoint
   
   ## Brief change log
   
   - Add the configuration 
**state.backend.rocksdb.checkpoint.verify.checksum.enable** in 
RocksDBStateBackend to determine whether the checksum of SST needs to be 
verified when making Checkpoint.
   
   - Introduce `RocksDBStateFileVerifier` into SnapshotStrategy to verify the 
Checksum of sst during snapshots
   
   ## Verifying this change
   
   - add unit test RocksdbStateFileVerifierTest#rocksdbStateFileVerifierTest
   - add unit test 
RocksIncrementalSnapshotStrategyTest#testCheckpointFailIfSstFileCorrupted
   
   ## Does this pull request potentially affect one of the following parts:
   
 - Dependencies (does it add or upgrade a dependency): ( no)
 - 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? (yes / no)
 - If yes, how is the feature documented? (not applicable / docs / JavaDocs 
/ not documented)
   


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