Skip to content

Commit

Permalink
[cdc-base] Close idle readers when snapshot finished (apache#2202)
Browse files Browse the repository at this point in the history
* [cdc-base] Close idle readers when snapshot finished
  • Loading branch information
Jiabao-Sun authored and 刘卓 committed Jun 13, 2023
1 parent a4029e1 commit 0e4cf57
Show file tree
Hide file tree
Showing 43 changed files with 628 additions and 31 deletions.
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 @@ -42,6 +42,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 +210,21 @@ 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 abstract JdbcSourceConfig create(int subtask);
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,8 @@ public interface SourceConfig extends Serializable {

boolean isIncludeSchemaChanges();

boolean isCloseIdleReaders();

/** Factory for the {@code SourceConfig}. */
@FunctionalInterface
interface Factory<C extends SourceConfig> extends Serializable {
Expand Down
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,14 @@ 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)
.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 when enabling this feature.");
}
Original file line number Diff line number Diff line change
Expand Up @@ -170,6 +170,11 @@ public void notifyCheckpointComplete(long checkpointId) {
snapshotSplitAssigner.notifyCheckpointComplete(checkpointId);
}

@Override
public boolean isStreamSplitAssigned() {
return isStreamSplitAssigned;
}

@Override
public void close() {
snapshotSplitAssigner.close();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,11 @@ public interface SplitAssigner {
*/
boolean waitingForFinishedSplits();

/** Whether the split assigner is finished stream split assigning. */
default boolean isStreamSplitAssigned() {
throw new UnsupportedOperationException("Not support to assigning StreamSplit.");
}

/**
* Gets the finished splits' information. This is useful metadata to generate a stream split
* that considering finished snapshot splits.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,11 @@ public void notifyCheckpointComplete(long checkpointId) {
// nothing to do
}

@Override
public boolean isStreamSplitAssigned() {
return isStreamSplitAssigned;
}

@Override
public void close() {}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,14 @@ private void assignSplits() {
continue;
}

if (splitAssigner.isStreamSplitAssigned() && sourceConfig.isCloseIdleReaders()) {
// close idle readers when snapshot phase finished.
context.signalNoMoreSplits(nextAwaiting);
awaitingReader.remove();
LOG.info("Close idle reader of subtask {}", nextAwaiting);
continue;
}

Optional<SourceSplitBase> split = splitAssigner.getNext();
if (split.isPresent()) {
final SourceSplitBase sourceSplit = split.get();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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.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");

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

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

public static void checkSupportCheckpointsAfterTasksFinished(boolean closeIdleReaders) {
if (closeIdleReaders && !supportCheckpointsAfterTasksFinished()) {
throw new UnsupportedOperationException(
"The flink version is required to be greater than or equal to 1.14 when 'execution.checkpointing.checkpoints-after-tasks-finish.enabled' is set to true. But the current version is "
+ runtimeFlinkVersion());
}
}
}
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;

private 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 @@ -24,6 +24,7 @@
import java.util.Properties;
import java.util.UUID;

import static com.ververica.cdc.connectors.base.utils.EnvironmentUtils.checkSupportCheckpointsAfterTasksFinished;
import static org.apache.flink.util.Preconditions.checkNotNull;

/** A factory to initialize {@link MySqlSourceConfig}. */
Expand All @@ -47,6 +48,7 @@ public MySqlSourceConfigFactory serverId(String serverId) {

/** Creates a new {@link MySqlSourceConfig} for the given subtask {@code subtaskId}. */
public MySqlSourceConfig create(int subtaskId) {
checkSupportCheckpointsAfterTasksFinished(closeIdleReaders);
Properties props = new Properties();
// hard code server name, because we don't need to distinguish it, docs:
// Logical name that identifies and provides a namespace for the particular
Expand Down Expand Up @@ -113,6 +115,7 @@ public MySqlSourceConfig create(int subtaskId) {
distributionFactorUpper,
distributionFactorLower,
includeSchemaChanges,
closeIdleReaders,
props,
dbzConfiguration,
driverClassName,
Expand Down
Loading

0 comments on commit 0e4cf57

Please sign in to comment.