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

[cdc-base] Close idle readers when snapshot finished #2162

Closed
wants to merge 1 commit 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
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ public abstract class BaseSourceConfig implements SourceConfig {
protected final double distributionFactorUpper;
protected final double distributionFactorLower;
protected final boolean includeSchemaChanges;
protected final boolean closeIdleReaders;

// --------------------------------------------------------------------------------------------
// Debezium Configurations
Expand All @@ -47,6 +48,7 @@ public BaseSourceConfig(
double distributionFactorUpper,
double distributionFactorLower,
boolean includeSchemaChanges,
boolean closeIdleReaders,
Properties dbzProperties,
Configuration dbzConfiguration) {
this.startupOptions = startupOptions;
Expand All @@ -55,6 +57,7 @@ public BaseSourceConfig(
this.distributionFactorUpper = distributionFactorUpper;
this.distributionFactorLower = distributionFactorLower;
this.includeSchemaChanges = includeSchemaChanges;
this.closeIdleReaders = closeIdleReaders;
this.dbzProperties = dbzProperties;
this.dbzConfiguration = dbzConfiguration;
}
Expand Down Expand Up @@ -87,6 +90,11 @@ public boolean isIncludeSchemaChanges() {
return includeSchemaChanges;
}

@Override
public boolean isCloseIdleReaders() {
return closeIdleReaders;
}

public Properties getDbzProperties() {
return dbzProperties;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ public JdbcSourceConfig(
double distributionFactorUpper,
double distributionFactorLower,
boolean includeSchemaChanges,
boolean closeIdleReaders,
Properties dbzProperties,
Configuration dbzConfiguration,
String driverClassName,
Expand All @@ -73,6 +74,7 @@ public JdbcSourceConfig(
distributionFactorUpper,
distributionFactorLower,
includeSchemaChanges,
closeIdleReaders,
dbzProperties,
dbzConfiguration);
this.driverClassName = driverClassName;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package com.ververica.cdc.connectors.base.config;

import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.Configuration;

import com.ververica.cdc.connectors.base.config.SourceConfig.Factory;
import com.ververica.cdc.connectors.base.options.JdbcSourceOptions;
Expand All @@ -28,6 +29,8 @@
import java.util.List;
import java.util.Properties;

import static com.ververica.cdc.connectors.base.utils.EnvironmentUtils.requireCheckpointsAfterTasksFinished;

/** A {@link Factory} to provide {@link SourceConfig} of JDBC data source. */
@Internal
public abstract class JdbcSourceConfigFactory implements Factory<JdbcSourceConfig> {
Expand All @@ -42,6 +45,7 @@ public abstract class JdbcSourceConfigFactory implements Factory<JdbcSourceConfi
protected List<String> tableList;
protected StartupOptions startupOptions = StartupOptions.initial();
protected boolean includeSchemaChanges = false;
protected boolean closeIdleReaders = false;
protected double distributionFactorUpper =
SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.defaultValue();
protected double distributionFactorLower =
Expand Down Expand Up @@ -209,6 +213,29 @@ public JdbcSourceConfigFactory startupOptions(StartupOptions startupOptions) {
return this;
}

/**
* Whether to close idle readers at the end of the snapshot phase. This feature depends on
* FLIP-147: Support Checkpoints After Tasks Finished. The flink version is required to be
* greater than or equal to 1.14, and the configuration <code>
* 'execution.checkpointing.checkpoints-after-tasks-finish.enabled'</code> needs to be set to
* true.
*
* <p>See more
* https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished.
*/
public JdbcSourceConfigFactory closeIdleReaders(boolean closeIdleReaders) {
this.closeIdleReaders = closeIdleReaders;
return this;
}

@Override
public JdbcSourceConfig create(int subtask, Configuration configuration) {
if (closeIdleReaders) {
requireCheckpointsAfterTasksFinished(configuration);
}
return create(subtask);
}

@Override
public abstract JdbcSourceConfig create(int subtask);
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package com.ververica.cdc.connectors.base.config;

import org.apache.flink.annotation.Experimental;
import org.apache.flink.configuration.Configuration;

import com.ververica.cdc.connectors.base.options.StartupOptions;

Expand All @@ -34,10 +35,13 @@ public interface SourceConfig extends Serializable {

boolean isIncludeSchemaChanges();

boolean isCloseIdleReaders();

/** Factory for the {@code SourceConfig}. */
@FunctionalInterface
interface Factory<C extends SourceConfig> extends Serializable {

C create(int subtask, Configuration configuration);

C create(int subtask);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@

package com.ververica.cdc.connectors.base.options;

import org.apache.flink.annotation.Experimental;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;

Expand Down Expand Up @@ -111,4 +112,15 @@ public class SourceOptions {
+ " The table chunks would use evenly calculation optimization when the data distribution is even,"
+ " and the query for splitting would happen when it is uneven."
+ " The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount.");

@Experimental
public static final ConfigOption<Boolean> SCAN_INCREMENTAL_CLOSE_IDLE_READER_ENABLED =
ConfigOptions.key("scan.incremental.close-idle-reader.enabled")
.booleanType()
.defaultValue(false)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this option's default value should be true , for the old flink version which don't support should be setted false by user.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Jiabao-Sun As you said , flink disabled by default. So ignore this comment. :)

.withDescription(
"Whether to close idle readers at the end of the snapshot phase. This feature depends on "
+ "FLIP-147: Support Checkpoints After Tasks Finished. The flink version is required to be "
+ "greater than or equal to 1.14, and the configuration 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' needs to be set to"
+ "true.");
}
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,7 @@ public IncrementalSourceReader<T, C> createReader(SourceReaderContext readerCont
throws Exception {
// create source config for the given subtask (e.g. unique server id)
C sourceConfig = configFactory.create(readerContext.getIndexOfSubtask());

FutureCompletingBlockingQueue<RecordsWithSplitIds<SourceRecords>> elementsQueue =
new FutureCompletingBlockingQueue<>();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,11 @@ private void assignSplits() {
context.assignSplit(sourceSplit, nextAwaiting);
awaitingReader.remove();
LOG.info("Assign split {} to subtask {}", sourceSplit, nextAwaiting);
} else if (sourceConfig.isCloseIdleReaders()) {
// close idle readers when snapshot phase finished.
context.signalNoMoreSplits(nextAwaiting);
awaitingReader.remove();
LOG.info("Close idle reader of subtask {}", nextAwaiting);
} else {
// there is no available splits by now, skip assigning
break;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.base.utils;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.util.EnvironmentInformation;

/** Utilities for environment information at runtime. */
public class EnvironmentUtils {

private EnvironmentUtils() {}

private static final VersionComparable FLINK_1_14 = VersionComparable.fromVersionString("1.14");

private static final String ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH =
"execution.checkpointing.checkpoints-after-tasks-finish.enabled";

public static VersionComparable runtimeFlinkVersion() {
return VersionComparable.fromVersionString(EnvironmentInformation.getVersion());
}

public static void requireCheckpointsAfterTasksFinished(Configuration configuration) {
if (!supportCheckpointsAfterTasksFinished()) {
throw new UnsupportedOperationException(
"To enabled checkpoints after tasks finished requires flink version greater than or equal to 1.14, current version is "
+ runtimeFlinkVersion());
}
if (!enableCheckPointsAfterTaskFinishes(configuration)) {
throw new IllegalArgumentException(
String.format(
"To enabled checkpoints after tasks finished requires '%s' set to true.",
ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH));
}
}

public static boolean supportCheckpointsAfterTasksFinished() {
return runtimeFlinkVersion().newerThanOrEqualTo(FLINK_1_14);
}

public static boolean enableCheckPointsAfterTaskFinishes(Configuration configuration) {
return configuration.getBoolean(ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, false);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.base.utils;

/** Used to compare version numbers at runtime. */
public class VersionComparable implements Comparable<VersionComparable> {

private int majorVersion;
private int minorVersion;
private int patchVersion;
private String versionString;

public VersionComparable(String versionString) {
this.versionString = versionString;
try {
int pos = versionString.indexOf('-');
String numberPart = versionString;
if (pos > 0) {
numberPart = versionString.substring(0, pos);
}

String[] versions = numberPart.split("\\.");

this.majorVersion = Integer.parseInt(versions[0]);
this.minorVersion = Integer.parseInt(versions[1]);
if (versions.length == 3) {
this.patchVersion = Integer.parseInt(versions[2]);
}
} catch (Exception e) {
throw new IllegalArgumentException(
String.format("Can not recognize version %s.", versionString));
}
}

public int getMajorVersion() {
return majorVersion;
}

public int getMinorVersion() {
return minorVersion;
}

public int getPatchVersion() {
return patchVersion;
}

public static VersionComparable fromVersionString(String versionString) {
return new VersionComparable(versionString);
}

@Override
public int compareTo(VersionComparable version) {
if (equalTo(version)) {
return 0;
} else if (newerThan(version)) {
return 1;
} else {
return -1;
}
}

public boolean equalTo(VersionComparable version) {
return majorVersion == version.majorVersion
&& minorVersion == version.minorVersion
&& patchVersion == version.patchVersion;
}

public boolean newerThan(VersionComparable version) {
if (majorVersion <= version.majorVersion) {
if (majorVersion < version.majorVersion) {
return false;
} else {
if (minorVersion <= version.minorVersion) {
if (minorVersion < version.patchVersion) {
return false;
} else {
return patchVersion > version.patchVersion;
}
}
}
}
return true;
}

public boolean newerThanOrEqualTo(VersionComparable version) {
return newerThan(version) || equalTo(version);
}

@Override
public String toString() {
return versionString;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -205,6 +205,21 @@ public MySqlSourceBuilder<T> debeziumProperties(Properties properties) {
return this;
}

/**
* Whether to close idle readers at the end of the snapshot phase. This feature depends on
* FLIP-147: Support Checkpoints After Tasks Finished. The flink version is required to be
* greater than or equal to 1.14, and the configuration <code>
* 'execution.checkpointing.checkpoints-after-tasks-finish.enabled'</code> needs to be set to
* true.
*
* <p>See more
* https://cwiki.apache.org/confluence/display/FLINK/FLIP-147%3A+Support+Checkpoints+After+Tasks+Finished.
*/
public MySqlSourceBuilder<T> closeIdleReaders(boolean closeIdleReaders) {
this.configFactory.closeIdleReaders(closeIdleReaders);
return this;
}

/**
* The deserializer used to convert from consumed {@link
* org.apache.kafka.connect.source.SourceRecord}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ public MySqlSourceConfig(
double distributionFactorUpper,
double distributionFactorLower,
boolean includeSchemaChanges,
boolean closeIdleReaders,
Properties dbzProperties,
Configuration dbzConfiguration,
String driverClassName,
Expand All @@ -66,6 +67,7 @@ public MySqlSourceConfig(
distributionFactorUpper,
distributionFactorLower,
includeSchemaChanges,
closeIdleReaders,
dbzProperties,
dbzConfiguration,
driverClassName,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,7 @@ public MySqlSourceConfig create(int subtaskId) {
distributionFactorUpper,
distributionFactorLower,
includeSchemaChanges,
closeIdleReaders,
props,
dbzConfiguration,
driverClassName,
Expand Down
Loading