je-ik commented on code in PR #32648:
URL: https://github.com/apache/beam/pull/32648#discussion_r1804194228
##########
.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.json:
##########
@@ -1,4 +1,5 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to
run",
- "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should
run this test"
+ "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should
run this test",
+ "https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19
support"
Review Comment:
Unrelated, but these edits seem to be accumulating over time. Can we
(probably in a different issue and PR) add a script that will just update some
timestamp field of the json?
##########
runners/flink/1.19/src/test/java/org/apache/beam/runners/flink/streaming/StreamSources.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.beam.runners.flink.streaming;
+
+import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.streaming.runtime.tasks.RegularOperatorChain;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
+
+/** {@link StreamSource} utilities, that bridge incompatibilities between
Flink releases. */
+public class StreamSources {
+
+ public static <OutT, SrcT extends SourceFunction<OutT>> void run(
+ StreamSource<OutT, SrcT> streamSource,
+ Object lockingObject,
+ Output<StreamRecord<OutT>> collector)
+ throws Exception {
+ streamSource.run(lockingObject, collector,
createOperatorChain(streamSource));
+ }
+
+ private static OperatorChain<?, ?>
createOperatorChain(AbstractStreamOperator<?> operator) {
+ return new RegularOperatorChain<>(
+ operator.getContainingTask(),
+ StreamTask.createRecordWriterDelegate(
+ operator.getOperatorConfig(), new
MockEnvironmentBuilder().build()));
+ }
+
+ /** The emitWatermarkStatus method was added in Flink 1.14, so we need to
wrap Output. */
+ public interface OutputWrapper<T> extends Output<T> {
+ @Override
+ default void emitWatermarkStatus(WatermarkStatus watermarkStatus) {}
+
+ /** In Flink 1.19 the {@code emitRecordAttributes} method was added. */
+ @Override
+ default void emitRecordAttributes(RecordAttributes recordAttributes) {
+ throw new UnsupportedOperationException("emitRecordAttributes not
implemented");
Review Comment:
Given the implementation, we could move the `StreamSources` to `src/main`
(shared across versions), if we just drop the `@Override` annotation.
##########
runners/flink/1.19/src/test/java/org/apache/beam/runners/flink/streaming/StreamSources.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.beam.runners.flink.streaming;
+
+import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorChain;
+import org.apache.flink.streaming.runtime.tasks.RegularOperatorChain;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
+
+/** {@link StreamSource} utilities, that bridge incompatibilities between
Flink releases. */
+public class StreamSources {
+
+ public static <OutT, SrcT extends SourceFunction<OutT>> void run(
+ StreamSource<OutT, SrcT> streamSource,
+ Object lockingObject,
+ Output<StreamRecord<OutT>> collector)
+ throws Exception {
+ streamSource.run(lockingObject, collector,
createOperatorChain(streamSource));
+ }
+
+ private static OperatorChain<?, ?>
createOperatorChain(AbstractStreamOperator<?> operator) {
+ return new RegularOperatorChain<>(
+ operator.getContainingTask(),
+ StreamTask.createRecordWriterDelegate(
+ operator.getOperatorConfig(), new
MockEnvironmentBuilder().build()));
+ }
+
+ /** The emitWatermarkStatus method was added in Flink 1.14, so we need to
wrap Output. */
Review Comment:
Flink 1.14 is no longer supported, seems that we currently need the wrapper
because of difference between Flink 1.15-1.18 and 1.19, am I correct?
--
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]