Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-36633] Add support for Flink 1.20 in Flink OpenSearch connector #48

Merged
merged 2 commits into from
Oct 30, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 6 additions & 1 deletion .github/workflows/weekly.yml
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,12 @@ jobs:
jdk: '8, 11, 17, 21',
branch: main
}, {
flink: 1.20-SNAPSHOT,
flink: 1.20-SNAPSHOT,
jdk: '8, 11, 17, 21',
branch: main
},
{
flink: 1.20,
jdk: '8, 11, 17, 21',
branch: main
}, {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,6 @@
* integration tests.
*/
public class DockerImageVersions {
public static final String OPENSEARCH_1 = "opensearchproject/opensearch:1.3.15";
public static final String OPENSEARCH_2 = "opensearchproject/opensearch:2.13.0";
public static final String OPENSEARCH_1 = "opensearchproject/opensearch:1.3.19";
public static final String OPENSEARCH_2 = "opensearchproject/opensearch:2.17.1";
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,17 +17,15 @@

package org.apache.flink.connector.opensearch.sink;

import org.apache.flink.api.common.operators.MailboxExecutor;
import org.apache.flink.api.connector.sink2.Sink.InitContext;
import org.apache.flink.connector.base.DeliveryGuarantee;
import org.apache.flink.connector.opensearch.sink.BulkResponseInspector.BulkResponseInspectorFactory;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.SimpleUserCodeClassLoader;
import org.apache.flink.util.TestLoggerExtension;
import org.apache.flink.util.function.ThrowingRunnable;

import org.apache.http.HttpHost;
import org.junit.jupiter.api.DynamicTest;
Expand Down Expand Up @@ -163,8 +161,7 @@ void testOverrideBulkResponseInspectorFactory() {
TestingSinkWriterMetricGroup.getSinkWriterMetricGroup(
new UnregisteredMetricsGroup()));

Mockito.when(sinkInitContext.getMailboxExecutor())
.thenReturn(new OpensearchSinkBuilderTest.DummyMailboxExecutor());
Mockito.when(sinkInitContext.getMailboxExecutor()).thenReturn(new SyncMailboxExecutor());
Mockito.when(sinkInitContext.getProcessingTimeService())
.thenReturn(new TestProcessingTimeService());
Mockito.when(sinkInitContext.getUserCodeClassLoader())
Expand All @@ -176,21 +173,6 @@ void testOverrideBulkResponseInspectorFactory() {
assertThat(called).isTrue();
}

private static class DummyMailboxExecutor implements MailboxExecutor {
private DummyMailboxExecutor() {}

public void execute(
ThrowingRunnable<? extends Exception> command,
String descriptionFormat,
Object... descriptionArgs) {}

public void yield() throws InterruptedException, FlinkRuntimeException {}

public boolean tryYield() throws FlinkRuntimeException {
return false;
}
}

private OpensearchSinkBuilder<Object> createEmptyBuilder() {
return new OpensearchSinkBuilder<>();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.flink.connector.opensearch.sink;

import org.apache.flink.api.common.operators.MailboxExecutor;
import org.apache.flink.api.connector.sink2.SinkWriter;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.connector.opensearch.OpensearchUtil;
Expand All @@ -27,11 +26,10 @@
import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
import org.apache.flink.metrics.testutils.MetricListener;
import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
import org.apache.flink.runtime.metrics.MetricNames;
import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.TestLoggerExtension;
import org.apache.flink.util.function.ThrowingRunnable;

import org.apache.http.HttpHost;
import org.junit.jupiter.api.AfterEach;
Expand Down Expand Up @@ -331,7 +329,7 @@ private OpensearchWriter<Tuple2<Integer, String>> createWriter(
null,
true),
metricGroup,
new TestMailbox(),
new SyncMailboxExecutor(),
new DefaultRestClientFactory(),
new DefaultBulkResponseInspector(failureHandler));
}
Expand Down Expand Up @@ -376,29 +374,4 @@ public void emit(
}
}
}

private static class TestMailbox implements MailboxExecutor {

@Override
public void execute(
ThrowingRunnable<? extends Exception> command,
String descriptionFormat,
Object... descriptionArgs) {
try {
command.run();
} catch (Exception e) {
throw new RuntimeException("Unexpected error", e);
}
}

@Override
public void yield() throws InterruptedException, FlinkRuntimeException {
Thread.sleep(100);
}

@Override
public boolean tryYield() throws FlinkRuntimeException {
return false;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.flink.connector.opensearch.sink;

import org.apache.flink.api.common.operators.MailboxExecutor;
import org.apache.flink.api.connector.sink2.SinkWriter;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.connector.opensearch.OpensearchUtil;
Expand All @@ -27,11 +26,10 @@
import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
import org.apache.flink.metrics.testutils.MetricListener;
import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
import org.apache.flink.runtime.metrics.MetricNames;
import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.TestLoggerExtension;
import org.apache.flink.util.function.ThrowingRunnable;

import org.apache.http.HttpHost;
import org.junit.jupiter.api.AfterEach;
Expand Down Expand Up @@ -331,7 +329,7 @@ private Opensearch2Writer<Tuple2<Integer, String>> createWriter(
null,
true),
metricGroup,
new TestMailbox(),
new SyncMailboxExecutor(),
new DefaultRestClientFactory(),
new DefaultBulkResponseInspector(failureHandler));
}
Expand Down Expand Up @@ -376,29 +374,4 @@ public void emit(
}
}
}

private static class TestMailbox implements MailboxExecutor {

@Override
public void execute(
ThrowingRunnable<? extends Exception> command,
String descriptionFormat,
Object... descriptionArgs) {
try {
command.run();
} catch (Exception e) {
throw new RuntimeException("Unexpected error", e);
}
}

@Override
public void yield() throws InterruptedException, FlinkRuntimeException {
Thread.sleep(100);
}

@Override
public boolean tryYield() throws FlinkRuntimeException {
return false;
}
}
}
35 changes: 23 additions & 12 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ under the License.
<parent>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-parent</artifactId>
<version>1.0.0</version>
<version>1.1.0</version>
</parent>

<modelVersion>4.0.0</modelVersion>
Expand Down Expand Up @@ -55,7 +55,7 @@ under the License.
</modules>

<properties>
<flink.version>1.18.0</flink.version>
<flink.version>1.20.0</flink.version>
<commons-compress.version>1.26.1</commons-compress.version>
<jackson-bom.version>2.16.2</jackson-bom.version>
<junit5.version>5.10.2</junit5.version>
Expand All @@ -64,7 +64,7 @@ under the License.
<mockito.version>3.12.4</mockito.version>

<japicmp.skip>false</japicmp.skip>
<japicmp.referenceVersion>1.0.0-1.16</japicmp.referenceVersion>
<japicmp.referenceVersion>1.0.0-1.19</japicmp.referenceVersion>

<slf4j.version>1.7.36</slf4j.version>
<log4j.version>2.17.2</log4j.version>
Expand Down Expand Up @@ -300,7 +300,7 @@ under the License.
<dependency>
<groupId>net.bytebuddy</groupId>
<artifactId>byte-buddy</artifactId>
<version>1.14.13</version>
<version>1.14.19</version>
</dependency>

<!-- For dependency convergence -->
Expand Down Expand Up @@ -370,18 +370,11 @@ under the License.
<scope>test</scope>
</dependency>

<dependency>
<!-- mockito/powermock mismatch -->
<groupId>net.bytebuddy</groupId>
<artifactId>byte-buddy</artifactId>
<version>1.14.10</version>
</dependency>

<dependency>
<!-- mockito/powermock mismatch -->
<groupId>net.bytebuddy</groupId>
<artifactId>byte-buddy-agent</artifactId>
<version>1.14.10</version>
<version>1.14.19</version>
</dependency>
</dependencies>
</dependencyManagement>
Expand Down Expand Up @@ -409,6 +402,24 @@ under the License.
<module>flink-sql-connector-opensearch2</module>
</modules>
</profile>
<!-- The spotless comes from flink-connector-parent and needs version bump to support JDK-21 -->
<profile>
<id>spotless</id>
<activation>
<jdk>[21,)</jdk>
</activation>
<build>
<plugins>
<plugin>
<groupId>com.diffplug.spotless</groupId>
<artifactId>spotless-maven-plugin</artifactId>
<configuration>
<skip>true</skip>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>

<build>
Expand Down