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 2021/10/20 03:51:45 UTC

[GitHub] [flink] JingsongLi commented on a change in pull request #17520: [FLINK-24565][avro] Port avro file format factory to BulkReaderFormatFactory

JingsongLi commented on a change in pull request #17520:
URL: https://github.com/apache/flink/pull/17520#discussion_r732391776



##########
File path: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AbstractAvroBulkFormat.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.formats.avro;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.IteratorResultIterator;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.utils.FSDataInputStreamWrapper;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableInput;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.io.DatumReader;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/** Provides a {@link BulkFormat} for Avro records. */
+public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSplit>
+        implements BulkFormat<T, SplitT> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public AvroReader createReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public AvroReader restoreReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public boolean isSplittable() {
+        return true;
+    }
+
+    private AvroReader createReader(SplitT split) throws IOException {
+        long end = split.offset() + split.length();
+        if (split.getReaderPosition().isPresent()) {
+            CheckpointedPosition position = split.getReaderPosition().get();
+            return new AvroReader(
+                    split.path(),
+                    split.offset(),
+                    end,
+                    position.getOffset(),
+                    position.getRecordsAfterOffset());
+        } else {
+            return new AvroReader(split.path(), split.offset(), end, -1, 0);
+        }
+    }
+
+    protected void open(SplitT split) {}
+
+    abstract T convert(A record);

Review comment:
       protected

##########
File path: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AbstractAvroBulkFormat.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.formats.avro;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.IteratorResultIterator;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.utils.FSDataInputStreamWrapper;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableInput;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.io.DatumReader;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/** Provides a {@link BulkFormat} for Avro records. */
+public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSplit>
+        implements BulkFormat<T, SplitT> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public AvroReader createReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public AvroReader restoreReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public boolean isSplittable() {
+        return true;
+    }
+
+    private AvroReader createReader(SplitT split) throws IOException {
+        long end = split.offset() + split.length();
+        if (split.getReaderPosition().isPresent()) {
+            CheckpointedPosition position = split.getReaderPosition().get();
+            return new AvroReader(
+                    split.path(),
+                    split.offset(),
+                    end,
+                    position.getOffset(),
+                    position.getRecordsAfterOffset());
+        } else {
+            return new AvroReader(split.path(), split.offset(), end, -1, 0);
+        }
+    }
+
+    protected void open(SplitT split) {}
+
+    abstract T convert(A record);
+
+    abstract A getReusedAvroObject();

Review comment:
       protected

##########
File path: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AbstractAvroBulkFormat.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.formats.avro;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.IteratorResultIterator;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.utils.FSDataInputStreamWrapper;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableInput;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.io.DatumReader;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/** Provides a {@link BulkFormat} for Avro records. */
+public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSplit>
+        implements BulkFormat<T, SplitT> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public AvroReader createReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public AvroReader restoreReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public boolean isSplittable() {
+        return true;
+    }
+
+    private AvroReader createReader(SplitT split) throws IOException {
+        long end = split.offset() + split.length();
+        if (split.getReaderPosition().isPresent()) {
+            CheckpointedPosition position = split.getReaderPosition().get();
+            return new AvroReader(
+                    split.path(),
+                    split.offset(),
+                    end,
+                    position.getOffset(),
+                    position.getRecordsAfterOffset());
+        } else {
+            return new AvroReader(split.path(), split.offset(), end, -1, 0);
+        }
+    }
+
+    protected void open(SplitT split) {}
+
+    abstract T convert(A record);
+
+    abstract A getReusedAvroObject();
+
+    private class AvroReader implements BulkFormat.Reader<T> {
+
+        private final DataFileReader<A> reader;
+        private final A reuse;
+
+        private final long end;
+        private final BlockingQueue<Boolean> blockingQueue;

Review comment:
       Use flink-connectors-files `Pool` instead? The record can be reused avro record.

##########
File path: flink-formats/flink-avro/pom.xml
##########
@@ -47,13 +47,11 @@ under the License.
 
 		<!-- Table ecosystem -->
 
-		<!-- Projects depending on this project won't depend on flink-table-*. -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-common</artifactId>
+			<artifactId>flink-table-runtime_${scala.binary.version}</artifactId>

Review comment:
       Avro could be scala free.
   Can we move `FileSystemConnectorOptions` to common?

##########
File path: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AbstractAvroBulkFormat.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.formats.avro;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.IteratorResultIterator;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.utils.FSDataInputStreamWrapper;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableInput;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.io.DatumReader;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/** Provides a {@link BulkFormat} for Avro records. */
+public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSplit>
+        implements BulkFormat<T, SplitT> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public AvroReader createReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public AvroReader restoreReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public boolean isSplittable() {
+        return true;
+    }
+
+    private AvroReader createReader(SplitT split) throws IOException {
+        long end = split.offset() + split.length();
+        if (split.getReaderPosition().isPresent()) {
+            CheckpointedPosition position = split.getReaderPosition().get();
+            return new AvroReader(
+                    split.path(),
+                    split.offset(),
+                    end,
+                    position.getOffset(),
+                    position.getRecordsAfterOffset());
+        } else {
+            return new AvroReader(split.path(), split.offset(), end, -1, 0);
+        }
+    }
+
+    protected void open(SplitT split) {}
+
+    abstract T convert(A record);
+
+    abstract A getReusedAvroObject();

Review comment:
       createAvroRecord?

##########
File path: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroFileFormatFactory.java
##########
@@ -90,6 +186,94 @@ public String factoryIdentifier() {
         return options;
     }
 
+    private static class AvroGenericRecordBulkFormat
+            extends AbstractAvroBulkFormat<GenericRecord, RowData, FileSourceSplit> {
+
+        private static final long serialVersionUID = 1L;
+
+        private final RowType physicalRowType;
+        private final TypeInformation<RowData> typeInfo;
+
+        private final String[] physicalFieldNames;
+        private final DataType[] physicalFieldTypes;
+        private final int[] selectFieldIndices;
+        private final List<String> partitionKeys;
+        private final String defaultPartitionValue;
+
+        private final int[] nonPartitionProjectIndexInProducedRecord;
+        private final int[] nonPartitionProjectIndexInAvroRecord;
+
+        private transient AvroToRowDataConverters.AvroToRowDataConverter converter;
+        private transient GenericRecord reusedAvroRecord;
+        private transient GenericRowData reusedRowData;
+
+        public AvroGenericRecordBulkFormat(
+                DynamicTableSource.Context context,
+                RowType physicalRowType,
+                String[] physicalFieldNames,
+                DataType[] physicalFieldTypes,

Review comment:
       Obtain `physicalFieldTypes` from `physicalRowType`?

##########
File path: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AbstractAvroBulkFormat.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.formats.avro;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.IteratorResultIterator;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.utils.FSDataInputStreamWrapper;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableInput;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.io.DatumReader;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/** Provides a {@link BulkFormat} for Avro records. */
+public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSplit>
+        implements BulkFormat<T, SplitT> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public AvroReader createReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public AvroReader restoreReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public boolean isSplittable() {
+        return true;
+    }
+
+    private AvroReader createReader(SplitT split) throws IOException {
+        long end = split.offset() + split.length();
+        if (split.getReaderPosition().isPresent()) {
+            CheckpointedPosition position = split.getReaderPosition().get();
+            return new AvroReader(
+                    split.path(),
+                    split.offset(),
+                    end,
+                    position.getOffset(),
+                    position.getRecordsAfterOffset());
+        } else {
+            return new AvroReader(split.path(), split.offset(), end, -1, 0);
+        }
+    }
+
+    protected void open(SplitT split) {}
+
+    abstract T convert(A record);
+
+    abstract A getReusedAvroObject();
+
+    private class AvroReader implements BulkFormat.Reader<T> {
+
+        private final DataFileReader<A> reader;
+        private final A reuse;
+
+        private final long end;
+        private final BlockingQueue<Boolean> blockingQueue;
+
+        private long currentBlockStart;
+        private long currentRecordsToSkip;
+
+        private AvroReader(Path path, long offset, long end, long blockStart, long recordsToSkip)
+                throws IOException {
+            this.reader = createReaderFromPath(path);
+            this.reuse = getReusedAvroObject();
+            if (blockStart >= 0) {
+                reader.seek(blockStart);
+            } else {
+                reader.sync(offset);
+            }
+            for (int i = 0; i < recordsToSkip; i++) {
+                reader.next(reuse);
+            }
+
+            this.end = end;
+            this.blockingQueue = new LinkedBlockingQueue<>(1);
+
+            this.currentBlockStart = reader.previousSync();
+            this.currentRecordsToSkip = recordsToSkip;
+        }
+
+        private DataFileReader<A> createReaderFromPath(Path path) throws IOException {
+            FileSystem fileSystem = path.getFileSystem();
+            DatumReader<A> datumReader = new GenericDatumReader<>();
+            SeekableInput in =
+                    new FSDataInputStreamWrapper(
+                            fileSystem.open(path), fileSystem.getFileStatus(path).getLen());
+            return (DataFileReader<A>) DataFileReader.openReader(in, datumReader);
+        }
+
+        @Nullable
+        @Override
+        public RecordIterator<T> readBatch() throws IOException {
+            if (reachEnd()) {

Review comment:
       Pool.pollEntry first.
   I don't know there is thread safety bug in Avro `reachEnd`.

##########
File path: flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AbstractAvroBulkFormat.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.formats.avro;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.IteratorResultIterator;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.utils.FSDataInputStreamWrapper;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.SeekableInput;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.io.DatumReader;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/** Provides a {@link BulkFormat} for Avro records. */
+public abstract class AbstractAvroBulkFormat<A, T, SplitT extends FileSourceSplit>
+        implements BulkFormat<T, SplitT> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public AvroReader createReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public AvroReader restoreReader(Configuration config, SplitT split) throws IOException {
+        open(split);
+        return createReader(split);
+    }
+
+    @Override
+    public boolean isSplittable() {
+        return true;
+    }
+
+    private AvroReader createReader(SplitT split) throws IOException {
+        long end = split.offset() + split.length();
+        if (split.getReaderPosition().isPresent()) {
+            CheckpointedPosition position = split.getReaderPosition().get();
+            return new AvroReader(
+                    split.path(),
+                    split.offset(),
+                    end,
+                    position.getOffset(),
+                    position.getRecordsAfterOffset());
+        } else {
+            return new AvroReader(split.path(), split.offset(), end, -1, 0);
+        }
+    }
+
+    protected void open(SplitT split) {}
+
+    abstract T convert(A record);
+
+    abstract A getReusedAvroObject();
+
+    private class AvroReader implements BulkFormat.Reader<T> {
+
+        private final DataFileReader<A> reader;
+        private final A reuse;
+
+        private final long end;
+        private final BlockingQueue<Boolean> blockingQueue;
+
+        private long currentBlockStart;
+        private long currentRecordsToSkip;
+
+        private AvroReader(Path path, long offset, long end, long blockStart, long recordsToSkip)
+                throws IOException {
+            this.reader = createReaderFromPath(path);
+            this.reuse = getReusedAvroObject();
+            if (blockStart >= 0) {
+                reader.seek(blockStart);
+            } else {
+                reader.sync(offset);
+            }
+            for (int i = 0; i < recordsToSkip; i++) {
+                reader.next(reuse);
+            }
+
+            this.end = end;
+            this.blockingQueue = new LinkedBlockingQueue<>(1);
+
+            this.currentBlockStart = reader.previousSync();
+            this.currentRecordsToSkip = recordsToSkip;
+        }
+
+        private DataFileReader<A> createReaderFromPath(Path path) throws IOException {
+            FileSystem fileSystem = path.getFileSystem();
+            DatumReader<A> datumReader = new GenericDatumReader<>();
+            SeekableInput in =
+                    new FSDataInputStreamWrapper(
+                            fileSystem.open(path), fileSystem.getFileStatus(path).getLen());
+            return (DataFileReader<A>) DataFileReader.openReader(in, datumReader);
+        }
+
+        @Nullable
+        @Override
+        public RecordIterator<T> readBatch() throws IOException {
+            if (reachEnd()) {
+                return null;
+            }
+
+            try {
+                blockingQueue.put(true);
+            } catch (InterruptedException e) {
+                throw new RuntimeException(

Review comment:
       For `InterruptedException`.
   If you catch it, you shoud `Thread.currentThread().interrupt();`




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