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-35333] Fix JdbcXaSinkTestBase fails in weekly workflows #120

Closed
wants to merge 2 commits into from
Closed
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
2 changes: 2 additions & 0 deletions .github/workflows/push_pr.yml
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@ jobs:
include:
- flink: 1.18-SNAPSHOT
jdk: '8, 11, 17'
- flink: 1.19-SNAPSHOT
jdk: '8, 11, 17, 21'
uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils
with:
flink_version: ${{ matrix.flink }}
Expand Down
2 changes: 2 additions & 0 deletions .github/workflows/weekly.yml
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,8 @@ jobs:
include:
- flink: 1.18-SNAPSHOT
jdk: '8, 11, 17'
- flink: 1.19-SNAPSHOT
jdk: '8, 11, 17, 21'
uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils
with:
flink_version: ${{ matrix.flink }}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,16 @@

package org.apache.flink.connector.jdbc;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.connector.jdbc.databases.derby.DerbyTestBase;

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.mockito.Mockito;

import static org.mockito.Mockito.doReturn;

/**
* Base class for JDBC test using DDL from {@link JdbcTestFixture}. It uses create tables before
Expand All @@ -37,4 +43,23 @@ public void before() throws Exception {
public void after() throws Exception {
JdbcTestFixture.cleanUpDatabasesStatic(getMetadata());
}

public static RuntimeContext getRuntimeContext(Boolean reused) {
ExecutionConfig config = getExecutionConfig(reused);
RuntimeContext context = Mockito.mock(RuntimeContext.class);
doReturn(config).when(context).getExecutionConfig();
return context;
}

public static ExecutionConfig getExecutionConfig(Boolean reused) {
ExecutionConfig config = Mockito.mock(ExecutionConfig.class);
doReturn(reused).when(config).isObjectReuseEnabled();
return config;
}

public static RuntimeContext getRuntimeContext(JobID jobId) {
RuntimeContext context = getRuntimeContext(false);
doReturn(jobId).when(context).getJobId();
return context;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,25 +19,7 @@

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.common.accumulators.DoubleCounter;
import org.apache.flink.api.common.accumulators.Histogram;
import org.apache.flink.api.common.accumulators.IntCounter;
import org.apache.flink.api.common.accumulators.LongCounter;
import org.apache.flink.api.common.cache.DistributedCache;
import org.apache.flink.api.common.externalresource.ExternalResourceInfo;
import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.common.state.AggregatingState;
import org.apache.flink.api.common.state.AggregatingStateDescriptor;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.MapState;
import org.apache.flink.api.common.state.MapStateDescriptor;
import org.apache.flink.api.common.state.ReducingState;
import org.apache.flink.api.common.state.ReducingStateDescriptor;
import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.jdbc.JdbcExactlyOnceOptions;
import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
Expand All @@ -46,7 +28,6 @@
import org.apache.flink.connector.jdbc.internal.JdbcOutputFormat;
import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
import org.apache.flink.core.fs.CloseableRegistry;
import org.apache.flink.metrics.groups.OperatorMetricGroup;
import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.StateInitializationContextImpl;
Expand All @@ -57,12 +38,10 @@

import javax.transaction.xa.Xid;

import java.io.Serializable;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;

Expand Down Expand Up @@ -160,150 +139,6 @@ static JdbcXaSinkFunction<TestEntry> buildSink(

static final RuntimeContext TEST_RUNTIME_CONTEXT = getRuntimeContext(new JobID());

static RuntimeContext getRuntimeContext(final JobID jobID) {
return new RuntimeContext() {

@Override
public JobID getJobId() {
return jobID;
}

@Override
public String getTaskName() {
return "test";
}

@Override
public OperatorMetricGroup getMetricGroup() {
return null;
}

@Override
public int getNumberOfParallelSubtasks() {
return 1;
}

@Override
public int getMaxNumberOfParallelSubtasks() {
return 1;
}

@Override
public int getIndexOfThisSubtask() {
return 0;
}

@Override
public int getAttemptNumber() {
return 0;
}

@Override
public String getTaskNameWithSubtasks() {
return "test";
}

@Override
public ExecutionConfig getExecutionConfig() {
return null;
}

@Override
public ClassLoader getUserCodeClassLoader() {
return null;
}

@Override
public <V, A extends Serializable> void addAccumulator(
String name, Accumulator<V, A> accumulator) {}

@Override
public <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name) {
return null;
}

@Override
public void registerUserCodeClassLoaderReleaseHookIfAbsent(
String releaseHookName, Runnable releaseHook) {
throw new UnsupportedOperationException();
}

@Override
public IntCounter getIntCounter(String name) {
return null;
}

@Override
public LongCounter getLongCounter(String name) {
return null;
}

@Override
public DoubleCounter getDoubleCounter(String name) {
return null;
}

@Override
public Histogram getHistogram(String name) {
return null;
}

@Override
public boolean hasBroadcastVariable(String name) {
return false;
}

@Override
public <RT> List<RT> getBroadcastVariable(String name) {
return null;
}

@Override
public <T, C> C getBroadcastVariableWithInitializer(
String name, BroadcastVariableInitializer<T, C> initializer) {
return null;
}

@Override
public DistributedCache getDistributedCache() {
return null;
}

@Override
public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
return null;
}

@Override
public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
return null;
}

@Override
public <T> ReducingState<T> getReducingState(
ReducingStateDescriptor<T> stateProperties) {
return null;
}

@Override
public <IN, ACC, OUT> AggregatingState<IN, OUT> getAggregatingState(
AggregatingStateDescriptor<IN, ACC, OUT> stateProperties) {
return null;
}

@Override
public Set<ExternalResourceInfo> getExternalResourceInfos(String resourceName) {
throw new UnsupportedOperationException();
}

@Override
public <UK, UV> MapState<UK, UV> getMapState(
MapStateDescriptor<UK, UV> stateProperties) {
return null;
}
};
}

static final SinkFunction.Context TEST_SINK_CONTEXT =
new SinkFunction.Context() {
@Override
Expand Down
Loading