Skip to content

Commit 13c3b13

Browse files
authored
[yugabyte/yugabyte-db#26107] Parellel streaming changes (#172)
This PR introduces the changes to stream changes in parallel using multiple tasks for a table given the user provides the hash_code ranges for it to stream. The following changes have been introduced in this PR: 1. New configurations: a. `streaming.mode`: This values takes the input as `default` or `parallel` which is then used to decide whether or not parallel streaming mode is supposed to be used. b. `slot.names`: A list of comma separated values for all the slot names which should be used by each task. c. `publication.names`: A list of comma separated values for all the publication names which should be used by each task. d. `slot.ranges`: A list of **semi-colon** separated values for slot ranges in the format `a,b;b,c;c,d`. 2. Validations in the class `YBValidate` have been introduced: a. To validate that the complete hash range is provided by the user and nothing is missing. b. To validate that the number of slot names is equal to the publication names as well as the number of slot ranges. c. To ensure that there's only one table provided in the `table.include.list` as parallel streaming will not work with multiple tables. 3. Support for snapshot with `streaming.mode` parallel. a. This will require providing the hash part of the primary key columns to the configuration parameter `primary.key.hash.columns`. 4. The `PostgresPartition` object will now also use the slot name to uniquely identify the source partition. ### Usage example If the connector configuration contains the following properties: ``` { ... "streaming.mode":"parallel", "slot.names":"rs1,rs1", "publication.names":"pb1,pb2", "slot.ranges":"0,32768;32768,65536" ... } ``` then we will have 2 tasks created: 1. `task 0`: `slot=rs1 publication=pb1 hash_range=0,32768` 2. `task 1`: `slot=rs2 publication=pb2 hash_range=32768,65536` ### Note: It is currently the user's responsibility to provide full hash ranges and maintain the order given in the configs for `slot.names`, `publication.names` and `slot.ranges` as the values will be picked sequentially and divided into tasks. Thus, in order to ensure that the task with a slot gets the same hash_range every time, the user needs to be careful with the order provided. This closes yugabyte/yugabyte-db#26107.
1 parent 615451f commit 13c3b13

File tree

10 files changed

+340
-17
lines changed

10 files changed

+340
-17
lines changed

debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java

+89-1
Original file line numberDiff line numberDiff line change
@@ -381,6 +381,39 @@ public static SecureConnectionMode parse(String value, String defaultValue) {
381381
}
382382
}
383383

384+
public enum StreamingMode implements EnumeratedValue {
385+
DEFAULT("DEFAULT") {
386+
@Override
387+
public boolean isParallel() {
388+
return false;
389+
}
390+
},
391+
PARALLEL("PARALLEL") {
392+
@Override
393+
public boolean isParallel() {
394+
return true;
395+
}
396+
};
397+
398+
399+
private final String streamingMode;
400+
401+
StreamingMode(String streamingMode) {
402+
this.streamingMode = streamingMode;
403+
}
404+
405+
public static StreamingMode parse(String s) {
406+
return valueOf(s.trim().toUpperCase());
407+
}
408+
409+
@Override
410+
public String getValue() {
411+
return streamingMode;
412+
}
413+
414+
public abstract boolean isParallel();
415+
}
416+
384417
public enum LsnType implements EnumeratedValue {
385418
SEQUENCE("SEQUENCE") {
386419
@Override
@@ -696,6 +729,31 @@ public static SchemaRefreshMode parse(String value) {
696729
.withDescription("Whether or not to take a consistent snapshot of the tables." +
697730
"Disabling this option may result in duplication of some already snapshot data in the streaming phase.");
698731

732+
public static final Field STREAMING_MODE = Field.create("streaming.mode")
733+
.withDisplayName("Streaming mode")
734+
.withType(Type.STRING)
735+
.withImportance(Importance.LOW)
736+
.withEnum(StreamingMode.class, StreamingMode.DEFAULT)
737+
.withDescription("Streaming mode the connector should follow");
738+
739+
public static final Field SLOT_NAMES = Field.create("slot.names")
740+
.withDisplayName("Slot names for parallel consumption")
741+
.withImportance(Importance.LOW)
742+
.withDescription("Comma separated values for multiple slot names")
743+
.withValidation(PostgresConnectorConfig::validateUsageWithParallelStreamingMode);
744+
745+
public static final Field PUBLICATION_NAMES = Field.create("publication.names")
746+
.withDisplayName("Publication names for parallel consumption")
747+
.withImportance(Importance.LOW)
748+
.withDescription("Comma separated values for multiple publication names")
749+
.withValidation(PostgresConnectorConfig::validateUsageWithParallelStreamingMode);
750+
751+
public static final Field SLOT_RANGES = Field.create("slot.ranges")
752+
.withDisplayName("Ranges on which a slot is supposed to operate")
753+
.withImportance(Importance.LOW)
754+
.withDescription("Semi-colon separated values for hash ranges to be polled by tasks.")
755+
.withValidation(PostgresConnectorConfig::validateUsageWithParallelStreamingMode);
756+
699757
public static final Field YB_LOAD_BALANCE_CONNECTIONS = Field.create("yb.load.balance.connections")
700758
.withDisplayName("YB load balance connections")
701759
.withType(Type.BOOLEAN)
@@ -1153,6 +1211,10 @@ public LsnType slotLsnType() {
11531211
return LsnType.parse(getConfig().getString(SLOT_LSN_TYPE));
11541212
}
11551213

1214+
public StreamingMode streamingMode() {
1215+
return StreamingMode.parse(getConfig().getString(STREAMING_MODE));
1216+
}
1217+
11561218
protected boolean dropSlotOnStop() {
11571219
if (getConfig().hasKey(DROP_SLOT_ON_STOP.name())) {
11581220
return getConfig().getBoolean(DROP_SLOT_ON_STOP);
@@ -1242,6 +1304,18 @@ public String primaryKeyHashColumns() {
12421304
return getConfig().getString(PRIMARY_KEY_HASH_COLUMNS);
12431305
}
12441306

1307+
public List<String> getSlotNames() {
1308+
return List.of(getConfig().getString(SLOT_NAMES).trim().split(","));
1309+
}
1310+
1311+
public List<String> getPublicationNames() {
1312+
return List.of(getConfig().getString(PUBLICATION_NAMES).trim().split(","));
1313+
}
1314+
1315+
public List<String> getSlotRanges() {
1316+
return List.of(getConfig().getString(SLOT_RANGES).trim().split(";"));
1317+
}
1318+
12451319
@Override
12461320
public byte[] getUnavailableValuePlaceholder() {
12471321
String placeholder = getConfig().getString(UNAVAILABLE_VALUE_PLACEHOLDER);
@@ -1326,7 +1400,11 @@ protected SourceInfoStructMaker<? extends AbstractSourceInfo> getSourceInfoStruc
13261400
INCREMENTAL_SNAPSHOT_CHUNK_SIZE,
13271401
UNAVAILABLE_VALUE_PLACEHOLDER,
13281402
LOGICAL_DECODING_MESSAGE_PREFIX_INCLUDE_LIST,
1329-
LOGICAL_DECODING_MESSAGE_PREFIX_EXCLUDE_LIST)
1403+
LOGICAL_DECODING_MESSAGE_PREFIX_EXCLUDE_LIST,
1404+
STREAMING_MODE,
1405+
SLOT_NAMES,
1406+
PUBLICATION_NAMES,
1407+
SLOT_RANGES)
13301408
.excluding(INCLUDE_SCHEMA_CHANGES)
13311409
.create();
13321410

@@ -1503,5 +1581,15 @@ protected static int validateYBHostname(Configuration config, Field field, Field
15031581
return problemCount;
15041582
}
15051583

1584+
protected static int validateUsageWithParallelStreamingMode(Configuration config, Field field, Field.ValidationOutput problems) {
1585+
String mode = config.getString(STREAMING_MODE);
1586+
int problemCount = 0;
1587+
1588+
if (!StreamingMode.parse(mode).isParallel()) {
1589+
problems.accept(field, config.getString(field), "Configuration is only valid with parallel streaming mode");
1590+
++problemCount;
1591+
}
15061592

1593+
return problemCount;
1594+
}
15071595
}

debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/PostgresConnectorTask.java

+28-5
Original file line numberDiff line numberDiff line change
@@ -165,6 +165,26 @@ public ChangeEventSourceCoordinator<PostgresPartition, PostgresOffsetContext> st
165165
else {
166166
LOGGER.info("Found previous offset {}", previousOffset);
167167
snapshotter.init(connectorConfig, previousOffset.asOffsetState(), slotInfo);
168+
169+
// If previous offset is present that means that the connector is being restarted.
170+
if (snapshotter.shouldSnapshot() && connectorConfig.streamingMode().isParallel()) {
171+
// Drop existing slot so that a new slot can be created.
172+
LOGGER.info("Dropping existing replication slot '{}' since task restarted before snapshot was completed", connectorConfig.slotName());
173+
jdbcConnection.execute(String.format("SELECT * FROM pg_drop_replication_slot('%s')", connectorConfig.slotName()));
174+
175+
// Set slotInfo to null so that slot can be created again.
176+
slotInfo = null;
177+
}
178+
}
179+
180+
// TODO Vaibhav: Read more in https://issues.redhat.com/browse/DBZ-2118
181+
if (connectorConfig.streamingMode().isParallel()) {
182+
try {
183+
jdbcConnection.commit();
184+
}
185+
catch (SQLException e) {
186+
throw new DebeziumException(e);
187+
}
168188
}
169189

170190
SlotCreationResult slotCreatedInfo = null;
@@ -191,11 +211,14 @@ public ChangeEventSourceCoordinator<PostgresPartition, PostgresOffsetContext> st
191211
}
192212
}
193213

194-
try {
195-
jdbcConnection.commit();
196-
}
197-
catch (SQLException e) {
198-
throw new DebeziumException(e);
214+
// TODO Vaibhav: Read more in https://issues.redhat.com/browse/DBZ-2118
215+
if (!connectorConfig.streamingMode().isParallel()) {
216+
try {
217+
jdbcConnection.commit();
218+
}
219+
catch (SQLException e) {
220+
throw new DebeziumException(e);
221+
}
199222
}
200223

201224
final PostgresEventMetadataProvider metadataProvider = new PostgresEventMetadataProvider();

debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/PostgresPartition.java

+5-3
Original file line numberDiff line numberDiff line change
@@ -22,11 +22,13 @@ public class PostgresPartition extends AbstractPartition implements Partition {
2222

2323
private final String serverName;
2424
private final String taskId;
25+
private final String slotName;
2526

26-
public PostgresPartition(String serverName, String databaseName, String taskId) {
27+
public PostgresPartition(String serverName, String databaseName, String taskId, String slotName) {
2728
super(databaseName);
2829
this.serverName = serverName;
2930
this.taskId = taskId;
31+
this.slotName = slotName;
3032
}
3133

3234
@Override
@@ -57,7 +59,7 @@ public String toString() {
5759
}
5860

5961
public String getPartitionIdentificationKey() {
60-
return String.format("%s_%s", serverName, taskId);
62+
return String.format("%s_%s_%s", serverName, taskId, slotName);
6163
}
6264

6365
static class Provider implements Partition.Provider<PostgresPartition> {
@@ -73,7 +75,7 @@ static class Provider implements Partition.Provider<PostgresPartition> {
7375
public Set<PostgresPartition> getPartitions() {
7476
return Collections.singleton(new PostgresPartition(
7577
connectorConfig.getLogicalName(), taskConfig.getString(DATABASE_NAME.name()),
76-
connectorConfig.getTaskId()));
78+
connectorConfig.getTaskId(), connectorConfig.slotName()));
7779
}
7880
}
7981
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,66 @@
1+
package io.debezium.connector.postgresql;
2+
3+
import io.debezium.DebeziumException;
4+
import io.debezium.connector.postgresql.transforms.yugabytedb.Pair;
5+
import org.slf4j.Logger;
6+
import org.slf4j.LoggerFactory;
7+
8+
import java.util.Comparator;
9+
import java.util.List;
10+
import java.util.stream.Collectors;
11+
12+
/**
13+
* Class to store all the validation methods.
14+
*
15+
* @author Vaibhav Kushwaha ([email protected])
16+
*/
17+
public class YBValidate {
18+
private static final Logger LOGGER = LoggerFactory.getLogger(YBValidate.class);
19+
private static final String RANGE_BEGIN = "0";
20+
private static final String RANGE_END = "65536";
21+
22+
public static void completeRangesProvided(List<String> slotRanges) {
23+
List<Pair<Integer, Integer>> pairList = slotRanges.stream()
24+
.map(entry -> {
25+
String[] parts = entry.split(",");
26+
return new Pair<>(Integer.valueOf(parts[0]), Integer.valueOf(parts[1]));
27+
})
28+
.sorted(Comparator.comparing(Pair::getFirst))
29+
.collect(Collectors.toList());
30+
31+
int rangeBegin = Integer.valueOf(RANGE_BEGIN);
32+
33+
for (Pair<Integer, Integer> pair : pairList) {
34+
if (rangeBegin != pair.getFirst()) {
35+
LOGGER.error("Error while validating ranges: {}", pairList);
36+
throw new DebeziumException(
37+
String.format("Tablet range starting from hash_code %d is missing", rangeBegin));
38+
}
39+
40+
rangeBegin = pair.getSecond();
41+
}
42+
43+
// At this point, if the range is complete, rangeBegin will be pointing to the RANGE_END value.
44+
if (rangeBegin != Integer.valueOf(RANGE_END)) {
45+
LOGGER.error("Error while validating ranges: {}", pairList);
46+
throw new DebeziumException(
47+
String.format("Incomplete ranges provided. Range starting from hash_code %d is missing", rangeBegin));
48+
}
49+
}
50+
51+
public static void slotAndPublicationsAreEqual(List<String> slotNames, List<String> publicationNames) {
52+
if (slotNames.size() != publicationNames.size()) {
53+
throw new DebeziumException(
54+
String.format("Number of provided slots does not match the number of provided " +
55+
"publications. Slots: %s, Publications: %s", slotNames, publicationNames));
56+
}
57+
}
58+
59+
public static void slotRangesMatchSlotNames(List<String> slotNames, List<String> slotRanges) {
60+
if (slotNames.size() != slotRanges.size()) {
61+
throw new DebeziumException(
62+
String.format("Number of provided slots does not match the number of provided " +
63+
"slot ranges. Slots: %s, Slot ranges: %s", slotNames, slotRanges));
64+
}
65+
}
66+
}

debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/YugabyteDBConnector.java

+66-4
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@ public class YugabyteDBConnector extends RelationalBaseSourceConnector {
4141

4242
private static final Logger LOGGER = LoggerFactory.getLogger(YugabyteDBConnector.class);
4343
private Map<String, String> props;
44+
private PostgresConnectorConfig connectorConfig;
4445

4546
public YugabyteDBConnector() {
4647
}
@@ -58,6 +59,39 @@ public Class<? extends Task> taskClass() {
5859
@Override
5960
public void start(Map<String, String> props) {
6061
this.props = props;
62+
this.connectorConfig = new PostgresConnectorConfig(Configuration.from(props));
63+
}
64+
65+
protected List<Map<String, String>> getTaskConfigsForParallelStreaming(List<String> slotNames,
66+
List<String> publicationNames,
67+
List<String> slotRanges) {
68+
List<Map<String, String>> taskConfigs = new ArrayList<>();
69+
70+
if (connectorConfig.getSnapshotter().shouldSnapshot()) {
71+
props.put(PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE.name(), props.get(PostgresConnectorConfig.TABLE_INCLUDE_LIST.name()));
72+
}
73+
74+
for (int i = 0; i < slotNames.size(); ++i) {
75+
Map<String, String> taskProps = new HashMap<>(this.props);
76+
77+
taskProps.put(PostgresConnectorConfig.TASK_ID, String.valueOf(i));
78+
taskProps.put(PostgresConnectorConfig.SLOT_NAME.name(), slotNames.get(i));
79+
taskProps.put(PostgresConnectorConfig.PUBLICATION_NAME.name(), publicationNames.get(i));
80+
taskProps.put(PostgresConnectorConfig.STREAM_PARAMS.name(), "hash_range=" + slotRanges.get(i));
81+
82+
if (connectorConfig.getSnapshotter().shouldSnapshot()) {
83+
String[] splitRange = slotRanges.get(i).split(",");
84+
String query = getParallelSnapshotQuery(splitRange[0], splitRange[1]);
85+
taskProps.put(
86+
PostgresConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE.name() + "." + taskProps.get(PostgresConnectorConfig.TABLE_INCLUDE_LIST.name()),
87+
query
88+
);
89+
}
90+
91+
taskConfigs.add(taskProps);
92+
}
93+
94+
return taskConfigs;
6195
}
6296

6397
@Override
@@ -66,14 +100,33 @@ public List<Map<String, String>> taskConfigs(int maxTasks) {
66100
return Collections.emptyList();
67101
}
68102

103+
final String tableIncludeList = props.get(PostgresConnectorConfig.TABLE_INCLUDE_LIST.name());
104+
105+
if (connectorConfig.streamingMode().isParallel()) {
106+
LOGGER.info("Initialising parallel streaming mode");
107+
108+
// Validate for a single table.
109+
validateSingleTableProvided(tableIncludeList, false /* isSnapshot */);
110+
111+
List<String> slotNames = connectorConfig.getSlotNames();
112+
List<String> publicationNames = connectorConfig.getPublicationNames();
113+
List<String> slotRanges = connectorConfig.getSlotRanges();
114+
115+
YBValidate.slotAndPublicationsAreEqual(slotNames, publicationNames);
116+
YBValidate.slotRangesMatchSlotNames(slotNames, slotRanges);
117+
YBValidate.completeRangesProvided(slotRanges);
118+
119+
return getTaskConfigsForParallelStreaming(slotNames, publicationNames, slotRanges);
120+
}
121+
122+
// TODO Vaibhav (#26106): The following code block is not needed now, remove in a separate PR.
69123
if (props.containsKey(PostgresConnectorConfig.SNAPSHOT_MODE.name())
70124
&& props.get(PostgresConnectorConfig.SNAPSHOT_MODE.name())
71125
.equalsIgnoreCase(PostgresConnectorConfig.SnapshotMode.PARALLEL.getValue())) {
72126
LOGGER.info("Initialising parallel snapshot consumption");
73127

74-
final String tableIncludeList = props.get(PostgresConnectorConfig.TABLE_INCLUDE_LIST.name());
75128
// Perform basic validations.
76-
validateSingleTableProvidedForParallelSnapshot(tableIncludeList);
129+
validateSingleTableProvided(tableIncludeList, true);
77130

78131
// Publication auto create mode should not be for all tables.
79132
if (props.containsKey(PostgresConnectorConfig.PUBLICATION_AUTOCREATE_MODE.name())
@@ -94,15 +147,16 @@ public List<Map<String, String>> taskConfigs(int maxTasks) {
94147
return props == null ? Collections.emptyList() : Collections.singletonList(new HashMap<>(props));
95148
}
96149

97-
protected void validateSingleTableProvidedForParallelSnapshot(String tableIncludeList) throws DebeziumException {
150+
protected void validateSingleTableProvided(String tableIncludeList, boolean isSnapshot) throws DebeziumException {
98151
if (tableIncludeList == null) {
99152
throw new DebeziumException("No table provided, provide a table in the table.include.list");
100153
} else if (tableIncludeList.contains(",")) {
101154
// This might indicate the presence of multiple tables in the include list, we do not want that.
102-
throw new DebeziumException("parallel snapshot consumption is only supported with one table at a time");
155+
throw new DebeziumException("parallel " + (isSnapshot ? "snapshot" : "streaming") + " consumption is only supported with one table at a time");
103156
}
104157
}
105158

159+
// TODO Vaibhav (#26106): This method needs to be removed.
106160
protected List<Map<String, String>> getConfigForParallelSnapshotConsumption(int maxTasks) {
107161
List<Map<String, String>> taskConfigs = new ArrayList<>();
108162

@@ -137,6 +191,14 @@ protected String getQueryForParallelSnapshotSelect(long lowerBound, long upperBo
137191
props.get(PostgresConnectorConfig.PRIMARY_KEY_HASH_COLUMNS.name()), upperBound);
138192
}
139193

194+
// TODO Vaibhav (#26106): This is a copy of existing method, remove the older method in a separate PR.
195+
protected String getParallelSnapshotQuery(String lowerBound, String upperBound) {
196+
return String.format("SELECT * FROM %s WHERE yb_hash_code(%s) >= %s AND yb_hash_code(%s) < %s",
197+
props.get(PostgresConnectorConfig.TABLE_INCLUDE_LIST.name()),
198+
props.get(PostgresConnectorConfig.PRIMARY_KEY_HASH_COLUMNS.name()), lowerBound,
199+
props.get(PostgresConnectorConfig.PRIMARY_KEY_HASH_COLUMNS.name()), upperBound);
200+
}
201+
140202
@Override
141203
public void stop() {
142204
this.props = null;

0 commit comments

Comments
 (0)