This is an automated email from the ASF dual-hosted git repository.

yuxia pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/fluss.git


The following commit(s) were added to refs/heads/main by this push:
     new 60363bded [flink] lake tiering support flink 1.18 (#1742)
60363bded is described below

commit 60363bdedacbc5bcb49046fde3ce103e7e3ca30e
Author: Junbo Wang <[email protected]>
AuthorDate: Wed Sep 24 09:23:55 2025 +0800

    [flink] lake tiering support flink 1.18 (#1742)
---
 .../api/functions/sink/v2/DiscardingSink.java      | 58 ++++++++++++++++++++++
 fluss-test-coverage/pom.xml                        |  3 +-
 2 files changed, 60 insertions(+), 1 deletion(-)

diff --git 
a/fluss-flink/fluss-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java
 
b/fluss-flink/fluss-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java
new file mode 100644
index 000000000..44ee583fe
--- /dev/null
+++ 
b/fluss-flink/fluss-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java
@@ -0,0 +1,58 @@
+/*
+ * 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.streaming.api.functions.sink.v2;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+
+import java.io.IOException;
+
+/**
+ * A special sink that ignores all elements.
+ *
+ * @param <IN> The type of elements received by the sink.
+ */
+@PublicEvolving
+public class DiscardingSink<IN> implements Sink<IN>, 
SupportsConcurrentExecutionAttempts {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public SinkWriter<IN> createWriter(InitContext context) throws IOException 
{
+        return new DiscardingElementWriter();
+    }
+
+    private class DiscardingElementWriter implements SinkWriter<IN> {
+
+        @Override
+        public void write(IN element, Context context) throws IOException, 
InterruptedException {
+            // discard it.
+        }
+
+        @Override
+        public void flush(boolean endOfInput) throws IOException, 
InterruptedException {
+            // this writer has no pending data.
+        }
+
+        @Override
+        public void close() throws Exception {
+            // do nothing.
+        }
+    }
+}
diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml
index df13bb696..7c3a658f0 100644
--- a/fluss-test-coverage/pom.xml
+++ b/fluss-test-coverage/pom.xml
@@ -354,6 +354,8 @@
                                             org.apache.fluss.flink.metrics.*
                                         </exclude>
                                         <!-- end exclude for flink-connector 
-->
+                                        <!-- exclude flink 1.18 compatibility 
class for sink -->
+                                        
<exclude>org.apache.flink.streaming.api.functions.sink.v2.*</exclude>
                                         
<exclude>org.apache.fluss.fs.hdfs.HadoopFsPlugin
                                         </exclude>
                                         
<exclude>org.apache.fluss.fs.hdfs.HadoopSecurityTokenReceiver</exclude>
@@ -388,7 +390,6 @@
                                         
<exclude>org.apache.fluss.lake.iceberg.*</exclude>
                                         
<exclude>org.apache.fluss.row.encode.iceberg.*</exclude>
                                         
<exclude>org.apache.fluss.bucketing.IcebergBucketingFunction</exclude>
-                                        
<exclude>org.apache.iceberg.transforms.TransformUtils</exclude>
                                         <!-- start exclude for flink tiering 
service -->
                                         
<exclude>org.apache.fluss.flink.tiering.source.TieringSourceOptions</exclude>
                                         
<exclude>org.apache.fluss.flink.tiering.source.TieringSource.Builder</exclude>

Reply via email to