twalthr commented on a change in pull request #17897:
URL: https://github.com/apache/flink/pull/17897#discussion_r758402241



##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ServiceLoaderUtil.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.table.factories;
+
+import java.util.Iterator;
+import java.util.ServiceLoader;
+
+class ServiceLoaderUtil {

Review comment:
       add JavaDoc and explain the need of this util

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ServiceLoaderUtil.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.table.factories;
+
+import java.util.Iterator;
+import java.util.ServiceLoader;
+
+class ServiceLoaderUtil {
+
+    static <T> Iterator<LoadResult<T>> load(Class<T> clazz, ClassLoader 
classLoader) {
+        return new SafeIterator<>(ServiceLoader.load(clazz, 
classLoader).iterator());
+    }
+
+    static class LoadResult<T> {
+        private final T service;
+        private final Throwable error;
+
+        private LoadResult(T service, Throwable error) {
+            this.service = service;
+            this.error = error;
+        }
+
+        private LoadResult(T service) {
+            this(service, null);
+        }
+
+        private LoadResult(Throwable error) {
+            this(null, error);
+        }
+
+        public boolean failed() {

Review comment:
       nit: `hasFailed`

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ServiceLoaderUtil.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.table.factories;
+
+import java.util.Iterator;
+import java.util.ServiceLoader;
+
+class ServiceLoaderUtil {
+
+    static <T> Iterator<LoadResult<T>> load(Class<T> clazz, ClassLoader 
classLoader) {
+        return new SafeIterator<>(ServiceLoader.load(clazz, 
classLoader).iterator());
+    }
+
+    static class LoadResult<T> {
+        private final T service;
+        private final Throwable error;
+
+        private LoadResult(T service, Throwable error) {
+            this.service = service;
+            this.error = error;
+        }
+
+        private LoadResult(T service) {
+            this(service, null);
+        }
+
+        private LoadResult(Throwable error) {
+            this(null, error);
+        }
+
+        public boolean failed() {
+            return error != null;
+        }
+
+        public Throwable getError() {
+            return error;
+        }
+
+        public T getService() {
+            return service;
+        }
+    }
+
+    /**
+     * This iterator wraps {@link Iterator#hasNext()} and {@link 
Iterator#next()} in try-catch, and
+     * returns {@link LoadResult} to handle such failures.
+     */
+    private static class SafeIterator<T> implements Iterator<LoadResult<T>> {
+
+        private final Iterator<T> iterator;
+
+        public SafeIterator(Iterator<T> iterator) {
+            this.iterator = iterator;
+        }
+
+        @Override
+        public boolean hasNext() {
+            try {
+                return iterator.hasNext();
+            } catch (Throwable t) {
+                return true; // It has the exception!

Review comment:
       this comment is not really useful, and it can even be an `Error` not an 
`Exception`

##########
File path: 
flink-table/flink-table-runtime/src/test/java/org/apache/flink/formats/testcsv/TestCsvSerializationSchema.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.testcsv;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.types.Row;
+
+import java.nio.charset.StandardCharsets;
+
+import static 
org.apache.flink.api.java.io.CsvOutputFormat.DEFAULT_FIELD_DELIMITER;
+
+class TestCsvSerializationSchema implements SerializationSchema<RowData> {

Review comment:
       Again: please add JavaDocs. What is this class good for?

##########
File path: 
flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/OrcNoHiveColumnarRowInputFormat.java
##########
@@ -58,7 +61,12 @@ private OrcNoHiveColumnarRowInputFormat() {}
                     PartitionFieldExtractor<SplitT> extractor,
                     int[] selectedFields,
                     List<OrcFilters.Predicate> conjunctPredicates,
-                    int batchSize) {
+                    int batchSize,
+                    Function<RowType, TypeInformation<RowData>> 
rowTypeInfoFactory) {
+        // TODO all this partition keys code should be pruned from the orc 
format,

Review comment:
       we should avoid TODOs that don't relate to a concrete JIRA issue

##########
File path: 
flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/stream/compact/CompactOperator.java
##########
@@ -119,10 +117,11 @@ public void initializeState(StateInitializationContext 
context) throws Exception
     }
 
     @Override
-    public void processElement(StreamRecord<CoordinatorOutput> element) throws 
Exception {
-        CoordinatorOutput value = element.getValue();
-        if (value instanceof CompactionUnit) {
-            CompactionUnit unit = (CompactionUnit) value;
+    public void processElement(StreamRecord<CompactMessages.CoordinatorOutput> 
element)
+            throws Exception {
+        CompactMessages.CoordinatorOutput value = element.getValue();

Review comment:
       nit: you could also add a static import here and at other locations 
(e.g. `CompactCoordinator`) to keep the changes in implementation minimal. this 
would help keeping the git blame helpful.

##########
File path: 
flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/stream/compact/CompactOperatorTest.java
##########
@@ -143,13 +149,13 @@ public void testEndInput() throws Exception {
 
     @Test
     public void testUnitSelection() throws Exception {
-        OneInputStreamOperatorTestHarness<CoordinatorOutput, 
PartitionCommitInfo> harness0 =
-                create(2, 0);
+        OneInputStreamOperatorTestHarness<CompactMessages.CoordinatorOutput, 
PartitionCommitInfo>

Review comment:
       same comment as above, try to keep the changes minimal with a static 
import

##########
File path: flink-table/flink-table-uber/pom.xml
##########
@@ -88,6 +88,11 @@ under the License.
                        <artifactId>flink-cep</artifactId>
                        <version>${project.version}</version>
                </dependency>
+               <dependency>
+                       <groupId>org.apache.flink</groupId>

Review comment:
       We should also add a comment in the docs that the filesystem connector 
is in `lib` by default.

##########
File path: flink-table/flink-table-uber/pom.xml
##########
@@ -88,6 +88,11 @@ under the License.
                        <artifactId>flink-cep</artifactId>
                        <version>${project.version}</version>
                </dependency>
+               <dependency>
+                       <groupId>org.apache.flink</groupId>

Review comment:
       I'm also skeptical about this change. It would be better to explicitly 
add `flink-connector-files` as a separate dependency to `lib`. It must not be 
part of the the table jars.




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