Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-22 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1401846952 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java: ## @@ -1185,8 +1295,10 @@ public DataStreamSource

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-22 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1401846952 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java: ## @@ -1185,8 +1295,10 @@ public DataStreamSource

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-22 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1401846952 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java: ## @@ -1185,8 +1295,10 @@ public DataStreamSource

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-22 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1401846952 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java: ## @@ -1185,8 +1295,10 @@ public DataStreamSource

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-22 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1401768536 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java: ## @@ -1185,8 +1295,10 @@ public DataStreamSource

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-20 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1820017623 @zentol CI is green again after the discussed API changes. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-18 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1397712701 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,200 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-18 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1397712701 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,200 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-17 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1397712701 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,200 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-17 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1397712701 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,200 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-17 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1816655632 @zentol some thoughts: - I like the idea of exposing `fromData`. There is no good reason for both `fromElements` and and `fromCollection` methods to exist - they do the same thing

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-15 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1394174857 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,200 @@ +/* + *

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-15 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1812451001 > Do you want to cleanup the git history a bit or shall I just squash everything? I'll do it, thanks! One thing to clarify before we merge - I just noticed yesterday that

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-14 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1810122266 @zentol CI is green -- 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.

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-14 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1810121992 @snuyanzin thanks for the pointer, is seems to be related, albite a somewhat different issue. The constructor for which ArchUnit previously added a violation was removed, but running it

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-13 Thread via GitHub
snuyanzin commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1809183659 >Using Java 11 the tests pass, using Java 8 they fail (as in CI). >Is this a known issue? not sure wether this is your issue or not some time ago we've faced this

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-13 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1809168702 @zentol I can confirm that the reason for the discrepancy in the local and the remote Architecture Tests execution is the Java version. Using Java 11 the tests pass, using Java 8 they

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-06 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1794701474 @zentol thanks for the feedback. The only remaining item currently is the [architecture tests

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-05 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1382648870 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-05 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1793845354 @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.

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-05 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1382645917 ## flink-formats/flink-avro/src/test/java/org/apache/flink/table/runtime/batch/AvroTypesITCase.java: ## @@ -240,7 +240,7 @@ public void testAvroToAvro() { }

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-05 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1382643573 ## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java: ## @@ -123,7

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-05 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1382643298 ## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java: ## @@ -1188,14 +1191,14 @@ void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-02 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1380507456 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* + *

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-02 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1380547823 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* + *

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-02 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1380524335 ## flink-formats/flink-avro/src/test/java/org/apache/flink/table/runtime/batch/AvroTypesITCase.java: ## @@ -240,7 +240,7 @@ public void testAvroToAvro() { }

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-02 Thread via GitHub
zentol commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1791157635 > What is the issue with adding the new method? By default you may not add new methods to classes because it can break downstream classes. Personally I'd say it's fine to do it here

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-11-02 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1790570036 @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.

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-31 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1787923597 @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.

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-26 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1781183357 @zentol I made two major changes as per our discussions above: - https://github.com/apache/flink/commit/2712c1813ca6420905e06b9e417de0eb61d586d9 - direct type passing without the

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1372403767 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370757585 ## flink-architecture-tests/flink-architecture-tests-production/archunit-violations/e5126cae-f3fe-48aa-b6fb-60ae6cc3fcd5: ## @@ -19,8 +19,8 @@ Method calls method

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371778188 ## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java: ## @@ -1188,14 +1191,14 @@ void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371775741 ## flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamExecutionEnvironmentITCase.java: ## @@ -67,7 +79,62 @@ public void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371769875 ## flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamExecutionEnvironmentITCase.java: ## @@ -67,7 +79,62 @@ public void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371721067 ## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java: ## @@ -123,7

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371721067 ## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java: ## @@ -123,7

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371718910 ## flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamExecutionEnvironmentITCase.java: ## @@ -67,7 +79,62 @@ public void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371715012 ## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java: ## @@ -1188,14 +1191,14 @@ void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371714068 ## flink-architecture-tests/flink-architecture-tests-production/archunit-violations/e5126cae-f3fe-48aa-b6fb-60ae6cc3fcd5: ## @@ -19,8 +19,8 @@ Method calls method

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371711944 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* + *

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371708873 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371577209 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* + *

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371571177 ## flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamExecutionEnvironmentITCase.java: ## @@ -67,7 +79,62 @@ public void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371506767 ## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java: ## @@ -123,7

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371398388 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* + *

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-25 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1371395994 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* + *

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1778137755 @zentol thanks a lot for the review!! I addressed all comments from your first pass, PTAL. -- This is an automated message from the Apache Git Service. To respond to the message,

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370875073 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370869459 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OutputTypeConfigurable.java: ## @@ -27,7 +27,10 @@ * at {@link

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370869459 ## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OutputTypeConfigurable.java: ## @@ -27,7 +27,10 @@ * at {@link

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370757489 ## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java: ## @@ -566,15 +567,15 @@ public void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370757728 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370845998 ## flink-tests/pom.xml: ## @@ -284,6 +284,13 @@ under the License. test + +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370845998 ## flink-tests/pom.xml: ## @@ -284,6 +284,13 @@ under the License. test + +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370841305 ## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java: ## @@ -123,7

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370841305 ## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessorTest.java: ## @@ -123,7

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370832543 ## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java: ## @@ -1188,14 +1191,14 @@ void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370824409 ## flink-formats/flink-parquet/src/test/resources/avro/user.avsc: ## @@ -1,9 +0,0 @@ -{ - "namespace": "org.apache.flink.connector.datagen.source.generated", -

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370765358 ## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java: ## @@ -1161,9 +1163,10 @@ void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370762519 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370762519 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370757864 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370757728 ## flink-connectors/flink-connector-datagen/src/main/java/org/apache/flink/connector/datagen/functions/FromElementsGeneratorFunction.java: ## @@ -0,0 +1,211 @@ +/* +

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
afedulov commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370757489 ## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java: ## @@ -566,15 +567,15 @@ public void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-24 Thread via GitHub
zentol commented on code in PR #23553: URL: https://github.com/apache/flink/pull/23553#discussion_r1370102899 ## flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java: ## @@ -566,15 +567,15 @@ public void

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-20 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1772756912 @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

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-19 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1771760579 @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.

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-19 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1771495374 @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.

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-19 Thread via GitHub
afedulov commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1771489496 The addition of the `OutputTypeConfigurable` was required to achieve feature parity and compatibility with the existing fromElements API

Re: [PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-19 Thread via GitHub
flinkbot commented on PR #23553: URL: https://github.com/apache/flink/pull/23553#issuecomment-1771102556 ## CI report: * 227741d560869d8610b9a80e65b9181ffd25bdb6 UNKNOWN Bot commands The @flinkbot bot supports the following commands: - `@flinkbot run azure`

[PR] [FLINK-28050][connectors] Migrate StreamExecutionEnvironment#fromElements() implementation to FLIP-27 Source API [flink]

2023-10-19 Thread via GitHub
afedulov opened a new pull request, #23553: URL: https://github.com/apache/flink/pull/23553 ## What is the purpose of the change The purpose is to change the internal implementation of `StreamExecutionEnvironment#fromElements()` that is currently based on SourceFunction API (deprecated,