Carl-Zhou-CN commented on code in PR #5100:
URL: https://github.com/apache/seatunnel/pull/5100#discussion_r1411643617


##########
seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/split/FileSourceSplitEnumerator.java:
##########
@@ -32,8 +35,11 @@
 @Slf4j
 public class FileSourceSplitEnumerator
         implements SourceSplitEnumerator<FileSourceSplit, FileSourceState> {
+
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(FileSourceSplitEnumerator.class);
+

Review Comment:
   Is it equivalent to @Slf4j?



##########
seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml:
##########
@@ -72,39 +72,46 @@
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-java</artifactId>
-            <version>${flink.1.13.6.version}</version>
+            <version>${flink.1.15.3.version}</version>
             <scope>${flink.scope}</scope>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-streaming-java</artifactId>
+            <version>${flink.1.15.3.version}</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            
<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-api-java</artifactId>
+            <version>${flink.1.15.3.version}</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-api-java-bridge</artifactId>
+            <version>${flink.1.15.3.version}</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-clients_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-planner-loader</artifactId>
+            <version>1.15.3</version>

Review Comment:
   ```suggestion
               <version>${flink.1.15.3.version}</version>
   ```



##########
seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml:
##########
@@ -72,39 +72,46 @@
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-java</artifactId>
-            <version>${flink.1.13.6.version}</version>
+            <version>${flink.1.15.3.version}</version>
             <scope>${flink.scope}</scope>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-streaming-java</artifactId>
+            <version>${flink.1.15.3.version}</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            
<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-api-java</artifactId>
+            <version>${flink.1.15.3.version}</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-api-java-bridge</artifactId>
+            <version>${flink.1.15.3.version}</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-clients_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-planner-loader</artifactId>
+            <version>1.15.3</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-runtime-web_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-runtime</artifactId>
+            <version>1.15.3</version>

Review Comment:
   Same as above



##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceSplitEnumeratorContext.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.seatunnel.translation.flink.source;
+
+import org.apache.seatunnel.api.common.metrics.MetricsContext;
+import org.apache.seatunnel.api.source.SourceEvent;
+import org.apache.seatunnel.api.source.SourceSplit;
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+
+import java.util.List;
+import java.util.Set;
+
+/**
+ * The implementation of {@link 
org.apache.seatunnel.api.source.SourceSplitEnumerator.Context} for
+ * flink engine.
+ *
+ * @param <SplitT>
+ */
+public class FlinkSourceSplitEnumeratorContext<SplitT extends SourceSplit>
+        implements SourceSplitEnumerator.Context<SplitT> {
+
+    private final SplitEnumeratorContext<SplitWrapper<SplitT>> enumContext;
+
+    public FlinkSourceSplitEnumeratorContext(
+            SplitEnumeratorContext<SplitWrapper<SplitT>> enumContext) {
+        this.enumContext = enumContext;
+    }
+
+    @Override
+    public int currentParallelism() {
+        return enumContext.currentParallelism();
+    }
+
+    @Override
+    public Set<Integer> registeredReaders() {
+        return enumContext.registeredReaders().keySet();
+    }
+
+    @Override
+    public void assignSplit(int subtaskId, List<SplitT> splits) {
+        splits.forEach(
+                split -> {
+                    enumContext.assignSplit(new SplitWrapper<>(split), 
subtaskId);
+                });
+    }
+
+    @Override
+    public void signalNoMoreSplits(int subtask) {
+        enumContext.signalNoMoreSplits(subtask);
+    }
+
+    @Override
+    public void sendEventToSourceReader(int subtaskId, SourceEvent event) {
+        enumContext.sendEventToSourceReader(subtaskId, new 
SourceEventWrapper(event));
+    }
+
+    @Override
+    public MetricsContext getMetricsContext() {
+        return null;

Review Comment:
   ```suggestion
           return new AbstractMetricsContext() {};
   ```
   Would this be better?



##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSource.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.seatunnel.translation.flink.source;
+
+import org.apache.seatunnel.api.serialization.Serializer;
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.source.SourceSplit;
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import 
org.apache.seatunnel.translation.flink.serialization.FlinkSimpleVersionedSerializer;
+import org.apache.seatunnel.translation.flink.utils.TypeConverterUtils;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.types.Row;
+
+import java.io.Serializable;
+
+/**
+ * The source implementation of {@link Source}, used for proxy all {@link 
SeaTunnelSource} in flink.
+ *
+ * @param <SplitT> The generic type of source split
+ * @param <EnumStateT> The generic type of enumerator state
+ */
+public class FlinkSource<SplitT extends SourceSplit, EnumStateT extends 
Serializable>
+        implements Source<Row, SplitWrapper<SplitT>, EnumStateT>, 
ResultTypeQueryable<Row> {
+
+    private final SeaTunnelSource<SeaTunnelRow, SplitT, EnumStateT> source;
+
+    public FlinkSource(SeaTunnelSource<SeaTunnelRow, SplitT, EnumStateT> 
source) {
+        this.source = source;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        org.apache.seatunnel.api.source.Boundedness boundedness = 
source.getBoundedness();
+        return boundedness == 
org.apache.seatunnel.api.source.Boundedness.BOUNDED
+                ? Boundedness.BOUNDED
+                : Boundedness.CONTINUOUS_UNBOUNDED;
+    }
+
+    @Override
+    public SourceReader<Row, SplitWrapper<SplitT>> 
createReader(SourceReaderContext readerContext)
+            throws Exception {
+        org.apache.seatunnel.api.source.SourceReader.Context context =
+                new FlinkSourceReaderContext(readerContext, source);
+        org.apache.seatunnel.api.source.SourceReader<SeaTunnelRow, SplitT> 
reader =
+                source.createReader(context);
+        return new FlinkSourceReader<>(reader, (SeaTunnelRowType) 
source.getProducedType());

Review Comment:
   source.getProducedType() ,have been marked as @Deprecated



##########
seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml:
##########
@@ -72,39 +72,46 @@
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-java</artifactId>
-            <version>${flink.1.13.6.version}</version>
+            <version>${flink.1.15.3.version}</version>
             <scope>${flink.scope}</scope>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-streaming-java</artifactId>
+            <version>${flink.1.15.3.version}</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            
<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-api-java</artifactId>
+            <version>${flink.1.15.3.version}</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-api-java-bridge</artifactId>
+            <version>${flink.1.15.3.version}</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-clients_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-planner-loader</artifactId>
+            <version>1.15.3</version>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-runtime-web_${scala.binary.version}</artifactId>
-            <version>${flink.1.13.6.version}</version>
-            <scope>${flink.scope}</scope>
+            <artifactId>flink-table-runtime</artifactId>
+            <version>1.15.3</version>
         </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-clients</artifactId>
+            <version>1.15.3</version>

Review Comment:
   Same as above



##########
seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/split/FileSourceSplitEnumerator.java:
##########
@@ -32,8 +35,11 @@
 @Slf4j
 public class FileSourceSplitEnumerator
         implements SourceSplitEnumerator<FileSourceSplit, FileSourceState> {
+
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(FileSourceSplitEnumerator.class);
+

Review Comment:
   Is it equivalent to @Slf4j?



##########
seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSource.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.seatunnel.translation.flink.source;
+
+import org.apache.seatunnel.api.serialization.Serializer;
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.source.SourceSplit;
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import 
org.apache.seatunnel.translation.flink.serialization.FlinkSimpleVersionedSerializer;
+import org.apache.seatunnel.translation.flink.utils.TypeConverterUtils;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.types.Row;
+
+import java.io.Serializable;
+
+/**
+ * The source implementation of {@link Source}, used for proxy all {@link 
SeaTunnelSource} in flink.
+ *
+ * @param <SplitT> The generic type of source split
+ * @param <EnumStateT> The generic type of enumerator state
+ */
+public class FlinkSource<SplitT extends SourceSplit, EnumStateT extends 
Serializable>
+        implements Source<Row, SplitWrapper<SplitT>, EnumStateT>, 
ResultTypeQueryable<Row> {
+
+    private final SeaTunnelSource<SeaTunnelRow, SplitT, EnumStateT> source;
+
+    public FlinkSource(SeaTunnelSource<SeaTunnelRow, SplitT, EnumStateT> 
source) {
+        this.source = source;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        org.apache.seatunnel.api.source.Boundedness boundedness = 
source.getBoundedness();
+        return boundedness == 
org.apache.seatunnel.api.source.Boundedness.BOUNDED
+                ? Boundedness.BOUNDED
+                : Boundedness.CONTINUOUS_UNBOUNDED;
+    }
+
+    @Override
+    public SourceReader<Row, SplitWrapper<SplitT>> 
createReader(SourceReaderContext readerContext)
+            throws Exception {
+        org.apache.seatunnel.api.source.SourceReader.Context context =
+                new FlinkSourceReaderContext(readerContext, source);
+        org.apache.seatunnel.api.source.SourceReader<SeaTunnelRow, SplitT> 
reader =
+                source.createReader(context);
+        return new FlinkSourceReader<>(reader, (SeaTunnelRowType) 
source.getProducedType());

Review Comment:
   source.getProducedType() ,have been marked as @Deprecated



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