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

[oracle] oracle table source Add missing parameters #1720

Merged
merged 2 commits into from
Nov 9, 2022
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
34 changes: 33 additions & 1 deletion docs/content/connectors/oracle-cdc.md
Original file line number Diff line number Diff line change
Expand Up @@ -302,7 +302,39 @@ Connector Options
<td>Optional startup mode for Oracle CDC consumer, valid enumerations are "initial"
and "latest-offset".
Please see <a href="#startup-reading-position">Startup Reading Position</a> section for more detailed information.</td>
</tr>
</tr>
<tr>
<td>chunk-meta.group.size</td>
<td>optional</td>
<td style="word-wrap: break-word;">1000</td>
<td>Integer</td>
<td>The group size of chunk meta, if the meta size exceeds the group size, the meta will be divided into multiple groups.</td>
</tr>
<tr>
<td>connect.timeout</td>
<td>optional</td>
<td style="word-wrap: break-word;">30s</td>
<td>Duration</td>
<td>The maximum time that the connector should wait after trying to connect to the Oracle database server before timing out.</td>
</tr>
<tr>
<td>chunk-key.even-distribution.factor.lower-bound</td>
<td>optional</td>
<td style="word-wrap: break-word;">0.05d</td>
<td>Double</td>
<td>The lower bound of chunk key distribution factor. The distribution factor is used to determine whether the table is evenly distribution or not.
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.</td>
</tr>
<tr>
<td>chunk-key.even-distribution.factor.upper-bound</td>
<td>optional</td>
<td style="word-wrap: break-word;">1000.0d</td>
<td>Double</td>
<td>The upper bound of chunk key distribution factor. The distribution factor is used to determine whether the table is evenly distribution or not.
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.</td>
</tr>
<tr>
<td>debezium.*</td>
<td>optional</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@

import javax.annotation.Nullable;

import java.time.Duration;
import java.util.Collections;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -69,9 +70,13 @@ public class OracleTableSource implements ScanTableSource, SupportsReadingMetada
private final StartupOptions startupOptions;
private final boolean enableParallelRead;
private final int splitSize;
private final int splitMetaGroupSize;
private final int fetchSize;
private final Duration connectTimeout;
private final int connectionPoolSize;
private final int connectMaxRetries;
private final double distributionFactorUpper;
private final double distributionFactorLower;
private final String chunkKeyColumn;

// --------------------------------------------------------------------------------------------
Expand All @@ -98,9 +103,13 @@ public OracleTableSource(
StartupOptions startupOptions,
boolean enableParallelRead,
int splitSize,
int splitMetaGroupSize,
int fetchSize,
Duration connectTimeout,
int connectMaxRetries,
int connectionPoolSize,
double distributionFactorUpper,
double distributionFactorLower,
@Nullable String chunkKeyColumn) {
this.physicalSchema = physicalSchema;
this.url = url;
Expand All @@ -117,9 +126,13 @@ public OracleTableSource(
this.metadataKeys = Collections.emptyList();
this.enableParallelRead = enableParallelRead;
this.splitSize = splitSize;
this.splitMetaGroupSize = splitMetaGroupSize;
this.fetchSize = fetchSize;
this.connectTimeout = connectTimeout;
this.connectMaxRetries = connectMaxRetries;
this.connectionPoolSize = connectionPoolSize;
this.distributionFactorUpper = distributionFactorUpper;
this.distributionFactorLower = distributionFactorLower;
this.chunkKeyColumn = chunkKeyColumn;
}

Expand Down Expand Up @@ -164,9 +177,13 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
.deserializer(deserializer)
.debeziumProperties(dbzProperties)
.splitSize(splitSize)
.splitMetaGroupSize(splitMetaGroupSize)
.fetchSize(fetchSize)
.connectTimeout(connectTimeout)
.connectionPoolSize(connectionPoolSize)
.connectMaxRetries(connectMaxRetries)
.distributionFactorUpper(distributionFactorUpper)
.distributionFactorLower(distributionFactorLower)
.build();

return SourceProvider.of(oracleChangeEventSource);
Expand Down Expand Up @@ -223,9 +240,13 @@ public DynamicTableSource copy() {
startupOptions,
enableParallelRead,
splitSize,
splitMetaGroupSize,
fetchSize,
connectTimeout,
connectMaxRetries,
connectionPoolSize,
distributionFactorUpper,
distributionFactorLower,
chunkKeyColumn);
source.metadataKeys = metadataKeys;
source.producedDataType = producedDataType;
Expand Down Expand Up @@ -256,9 +277,13 @@ public boolean equals(Object o) {
&& Objects.equals(metadataKeys, that.metadataKeys)
&& Objects.equals(enableParallelRead, that.enableParallelRead)
&& Objects.equals(splitSize, that.splitSize)
&& Objects.equals(splitMetaGroupSize, that.splitMetaGroupSize)
&& Objects.equals(fetchSize, that.fetchSize)
&& Objects.equals(connectTimeout, that.connectTimeout)
&& Objects.equals(connectMaxRetries, that.connectMaxRetries)
&& Objects.equals(connectionPoolSize, that.connectionPoolSize)
&& Objects.equals(distributionFactorUpper, that.distributionFactorUpper)
&& Objects.equals(distributionFactorLower, that.distributionFactorLower)
&& Objects.equals(chunkKeyColumn, that.chunkKeyColumn);
}

Expand All @@ -280,9 +305,13 @@ public int hashCode() {
metadataKeys,
enableParallelRead,
splitSize,
splitMetaGroupSize,
fetchSize,
connectTimeout,
connectMaxRetries,
connectionPoolSize,
distributionFactorUpper,
distributionFactorLower,
chunkKeyColumn);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,22 +27,28 @@
import com.ververica.cdc.connectors.base.options.StartupOptions;
import com.ververica.cdc.debezium.table.DebeziumOptions;

import java.time.Duration;
import java.util.HashSet;
import java.util.Set;

import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.CONNECTION_POOL_SIZE;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_MAX_RETRIES;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.CONNECT_TIMEOUT;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.DATABASE_NAME;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.HOSTNAME;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.PASSWORD;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.SERVER_TIME_ZONE;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.TABLE_NAME;
import static com.ververica.cdc.connectors.base.options.JdbcSourceOptions.USERNAME;
import static com.ververica.cdc.connectors.base.options.SourceOptions.CHUNK_META_GROUP_SIZE;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SCAN_SNAPSHOT_FETCH_SIZE;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SCAN_STARTUP_MODE;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND;
import static com.ververica.cdc.connectors.base.options.SourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND;
import static com.ververica.cdc.connectors.base.utils.ObjectUtils.doubleCompare;
import static com.ververica.cdc.connectors.oracle.source.config.OracleSourceOptions.PORT;
import static com.ververica.cdc.connectors.oracle.source.config.OracleSourceOptions.SCHEMA_NAME;
import static com.ververica.cdc.connectors.oracle.source.config.OracleSourceOptions.URL;
Expand Down Expand Up @@ -74,18 +80,25 @@ public DynamicTableSource createDynamicTableSource(Context context) {

boolean enableParallelRead = config.get(SCAN_INCREMENTAL_SNAPSHOT_ENABLED);
int splitSize = config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE);
int splitMetaGroupSize = config.get(CHUNK_META_GROUP_SIZE);
int fetchSize = config.get(SCAN_SNAPSHOT_FETCH_SIZE);
Duration connectTimeout = config.get(CONNECT_TIMEOUT);
int connectMaxRetries = config.get(CONNECT_MAX_RETRIES);
int connectionPoolSize = config.get(CONNECTION_POOL_SIZE);
double distributionFactorUpper = config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND);
double distributionFactorLower = config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND);
String chunkKeyColumn =
config.getOptional(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN).orElse(null);
String serverTimezone = config.get(SERVER_TIME_ZONE);

if (enableParallelRead) {
validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1);
validateIntegerOption(SCAN_SNAPSHOT_FETCH_SIZE, fetchSize, 1);
validateIntegerOption(CHUNK_META_GROUP_SIZE, splitMetaGroupSize, 1);
validateIntegerOption(CONNECTION_POOL_SIZE, connectionPoolSize, 1);
validateIntegerOption(CONNECT_MAX_RETRIES, connectMaxRetries, 0);
validateDistributionFactorUpper(distributionFactorUpper);
validateDistributionFactorLower(distributionFactorLower);
}

return new OracleTableSource(
Expand All @@ -102,9 +115,13 @@ public DynamicTableSource createDynamicTableSource(Context context) {
startupOptions,
enableParallelRead,
splitSize,
splitMetaGroupSize,
fetchSize,
connectTimeout,
connectMaxRetries,
connectionPoolSize,
distributionFactorUpper,
distributionFactorLower,
chunkKeyColumn);
}

Expand Down Expand Up @@ -133,9 +150,13 @@ public Set<ConfigOption<?>> optionalOptions() {
options.add(SCAN_STARTUP_MODE);
options.add(SCAN_INCREMENTAL_SNAPSHOT_ENABLED);
options.add(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE);
options.add(CHUNK_META_GROUP_SIZE);
options.add(SCAN_SNAPSHOT_FETCH_SIZE);
options.add(CONNECT_TIMEOUT);
options.add(CONNECT_MAX_RETRIES);
options.add(CONNECTION_POOL_SIZE);
options.add(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND);
options.add(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND);
options.add(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN);
return options;
}
Expand Down Expand Up @@ -173,4 +194,28 @@ private void validateIntegerOption(
"The value of option '%s' must larger than %d, but is %d",
option.key(), exclusiveMin, optionValue));
}

/** Checks the value of given evenly distribution factor upper bound is valid. */
private void validateDistributionFactorUpper(double distributionFactorUpper) {
checkState(
doubleCompare(distributionFactorUpper, 1.0d) >= 0,
String.format(
"The value of option '%s' must larger than or equals %s, but is %s",
SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.key(),
1.0d,
distributionFactorUpper));
}

/** Checks the value of given evenly distribution factor lower bound is valid. */
private void validateDistributionFactorLower(double distributionFactorLower) {
checkState(
doubleCompare(distributionFactorLower, 0.0d) >= 0
&& doubleCompare(distributionFactorLower, 1.0d) <= 0,
String.format(
"The value of option '%s' must between %s and %s inclusively, but is %s",
SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.key(),
0.0d,
1.0d,
distributionFactorLower));
}
}
Loading