Skip to content
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
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,18 @@ public class PscConnectorOptions {

public static final ConfigOption<Integer> SINK_PARALLELISM = FactoryUtil.SINK_PARALLELISM;

public static final ConfigOption<Boolean> SCAN_ENABLE_RESCALE =
ConfigOptions.key("scan.enable-rescale")
.booleanType()
.defaultValue(false)
.withDescription(
"Enable smart rescale() to redistribute data when needed. When enabled, the connector " +
"automatically compares the configured pipeline parallelism (table.exec.resource.default-parallelism) " +
"with the topic's partition count. Rescale is applied ONLY when parallelism > partition count, " +
"avoiding unnecessary shuffle overhead otherwise. This allows downstream operators to fully utilize " +
"higher parallelism than the source partition count. " +
"Default: false (no automatic shuffling).");

// --------------------------------------------------------------------------------------------
// Psc specific options
// --------------------------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,9 @@ public class PscDynamicSource
/** Optional user-provided UID prefix for stabilizing operator UIDs across DAG changes. */
protected final @Nullable String sourceUidPrefix;

/** Enable rescale() shuffle to redistribute data across downstream operators. */
protected final boolean enableRescale;

public PscDynamicSource(
DataType physicalDataType,
@Nullable DecodingFormat<DeserializationSchema<RowData>> keyDecodingFormat,
Expand All @@ -187,7 +190,8 @@ public PscDynamicSource(
long boundedTimestampMillis,
boolean upsertMode,
String tableIdentifier,
@Nullable String sourceUidPrefix) {
@Nullable String sourceUidPrefix,
boolean enableRescale) {
// Format attributes
this.physicalDataType =
Preconditions.checkNotNull(
Expand Down Expand Up @@ -228,11 +232,12 @@ public PscDynamicSource(
this.upsertMode = upsertMode;
this.tableIdentifier = tableIdentifier;
this.sourceUidPrefix = sourceUidPrefix;
this.enableRescale = enableRescale;
}

/**
* Backward-compatible constructor without UID prefix. Delegates to the full constructor with a
* null {@code sourceUidPrefix}.
* Backward-compatible constructor without UID prefix and rescale. Delegates to the full
* constructor with null {@code sourceUidPrefix} and false {@code enableRescale}.
*/
public PscDynamicSource(
DataType physicalDataType,
Expand Down Expand Up @@ -270,7 +275,8 @@ public PscDynamicSource(
boundedTimestampMillis,
upsertMode,
tableIdentifier,
null);
null,
false);
}

@Override
Expand Down Expand Up @@ -302,16 +308,24 @@ public DataStream<RowData> produceDataStream(
DataStreamSource<RowData> sourceStream =
execEnv.fromSource(
pscSource, watermarkStrategy, "PscSource-" + tableIdentifier);

// Let Flink handle source parallelism automatically (defaults to partition count)
// Only add rescale if explicitly enabled to redistribute data across downstream operators
DataStream<RowData> resultStream = sourceStream;
if (enableRescale) {
resultStream = sourceStream.rescale();
}

// Prefer explicit user-provided UID prefix if present; otherwise rely on provider context.
if (sourceUidPrefix != null) {
final String trimmedPrefix = sourceUidPrefix.trim();
if (!trimmedPrefix.isEmpty()) {
sourceStream.uid(trimmedPrefix + ":" + PSC_TRANSFORMATION + ":" + tableIdentifier);
return sourceStream;
return resultStream;
}
}
providerContext.generateUid(PSC_TRANSFORMATION).ifPresent(sourceStream::uid);
return sourceStream;
return resultStream;
}

@Override
Expand Down Expand Up @@ -396,7 +410,8 @@ public DynamicTableSource copy() {
boundedTimestampMillis,
upsertMode,
tableIdentifier,
sourceUidPrefix);
sourceUidPrefix,
enableRescale);
copy.producedDataType = producedDataType;
copy.metadataKeys = metadataKeys;
copy.watermarkStrategy = watermarkStrategy;
Expand Down Expand Up @@ -436,6 +451,8 @@ public boolean equals(Object o) {
&& boundedTimestampMillis == that.boundedTimestampMillis
&& Objects.equals(upsertMode, that.upsertMode)
&& Objects.equals(tableIdentifier, that.tableIdentifier)
&& Objects.equals(sourceUidPrefix, that.sourceUidPrefix)
&& enableRescale == that.enableRescale
&& Objects.equals(watermarkStrategy, that.watermarkStrategy);
}

Expand All @@ -461,6 +478,8 @@ public int hashCode() {
boundedTimestampMillis,
upsertMode,
tableIdentifier,
sourceUidPrefix,
enableRescale,
watermarkStrategy);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS;
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS;
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptions.SCAN_TOPIC_PARTITION_DISCOVERY;
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptions.SCAN_ENABLE_RESCALE;
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptions.SINK_PARALLELISM;
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptions.SINK_PARTITIONER;
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptions.TOPIC_URI;
Expand All @@ -95,6 +96,7 @@
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptionsUtil.validateDeliveryGuarantee;
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptionsUtil.validateTableSinkOptions;
import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptionsUtil.validateTableSourceOptions;
import static com.pinterest.flink.streaming.connectors.psc.table.PscTableCommonUtils.shouldApplyRescale;

/**
* Factory for creating configured instances of {@link PscDynamicSource} and {@link
Expand Down Expand Up @@ -149,6 +151,7 @@ public Set<ConfigOption<?>> optionalOptions() {
options.add(SCAN_BOUNDED_MODE);
options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS);
options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS);
options.add(SCAN_ENABLE_RESCALE);
options.add(SOURCE_UID_PREFIX);
return options;
}
Expand Down Expand Up @@ -212,6 +215,13 @@ public DynamicTableSource createDynamicTableSource(Context context) {

final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null);

// Determine if rescale should be applied based on parallelism vs partition count
final boolean shouldRescale = shouldApplyRescale(
tableOptions,
context.getConfiguration(),
getSourceTopicUris(tableOptions),
properties);

return createPscTableSource(
physicalDataType,
keyDecodingFormat.orElse(null),
Expand All @@ -229,7 +239,8 @@ public DynamicTableSource createDynamicTableSource(Context context) {
boundedOptions.specificOffsets,
boundedOptions.boundedTimestampMillis,
context.getObjectIdentifier().asSummaryString(),
tableOptions.getOptional(SOURCE_UID_PREFIX).orElse(null));
tableOptions.getOptional(SOURCE_UID_PREFIX).orElse(null),
shouldRescale);
}

@Override
Expand Down Expand Up @@ -395,7 +406,8 @@ protected PscDynamicSource createPscTableSource(
Map<PscTopicUriPartition, Long> specificEndOffsets,
long endTimestampMillis,
String tableIdentifier,
@Nullable String sourceUidPrefix) {
@Nullable String sourceUidPrefix,
boolean enableRescale) {
return new PscDynamicSource(
physicalDataType,
keyDecodingFormat,
Expand All @@ -414,7 +426,8 @@ protected PscDynamicSource createPscTableSource(
endTimestampMillis,
false,
tableIdentifier,
sourceUidPrefix);
sourceUidPrefix,
enableRescale);
}

protected PscDynamicSink creatPscTableSink(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,183 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.pinterest.flink.streaming.connectors.psc.table;

import com.pinterest.psc.common.TopicUri;
import com.pinterest.psc.config.PscConfiguration;
import com.pinterest.psc.metadata.TopicUriMetadata;
import com.pinterest.psc.metadata.client.PscMetadataClient;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.table.api.config.ExecutionConfigOptions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Properties;

import static com.pinterest.flink.streaming.connectors.psc.table.PscConnectorOptions.SCAN_ENABLE_RESCALE;

/**
* Common utility functions for PSC dynamic table source and sink.
*
* <p>This class provides shared utility methods used across PSC table factories,
* including smart rescale decision logic, metadata queries, and other common operations.
*/
public class PscTableCommonUtils {

private static final Logger LOG = LoggerFactory.getLogger(PscTableCommonUtils.class);

/**
* Determines whether rescale() should be applied based on:
* 1. scan.enable-rescale flag must be true
* 2. Global parallelism (table.exec.resource.default-parallelism) > partition count
*
* <p>This automatically avoids unnecessary shuffle overhead when partitions >= parallelism.
*
* @param tableOptions User's table configuration options
* @param globalConfig Global Flink configuration
* @param topicUris List of topic URIs to query for partition counts
* @param pscProperties PSC properties for metadata client connection
* @return true if rescale should be applied, false otherwise
*/
public static boolean shouldApplyRescale(
ReadableConfig tableOptions,
ReadableConfig globalConfig,
List<String> topicUris,
Properties pscProperties) {

// First check if rescale is enabled by user
if (!tableOptions.get(SCAN_ENABLE_RESCALE)) {
return false;
}

// Get the global default parallelism
int defaultParallelism = globalConfig.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM);

// If parallelism is -1 (not set) or <= 0, cannot determine, so don't apply rescale
if (defaultParallelism <= 0) {
LOG.info("scan.enable-rescale is true, but table.exec.resource.default-parallelism is {} (not set). " +
"Rescale will not be applied. Set a positive parallelism value to enable automatic rescale.",
defaultParallelism);
return false;
}

// Query partition count from PSC metadata
int partitionCount = getTopicPartitionCount(topicUris, pscProperties);

// If partition count couldn't be determined, don't apply rescale (fail-safe)
if (partitionCount <= 0) {
LOG.warn("scan.enable-rescale is true, but partition count could not be determined. " +
"Rescale will not be applied.");
return false;
}

// Apply rescale only if user-configured parallelism exceeds partition count
boolean shouldRescale = defaultParallelism > partitionCount;

if (shouldRescale) {
LOG.info("Applying rescale(): configured parallelism ({}) > partition count ({}). " +
"Data will be redistributed to fully utilize downstream operators.",
defaultParallelism, partitionCount);
} else {
LOG.info("Skipping rescale(): configured parallelism ({}) <= partition count ({}). " +
"No shuffle needed as source will naturally match or exceed desired parallelism.",
defaultParallelism, partitionCount);
}

return shouldRescale;
}

/**
* Queries the minimum partition count across all specified topic URIs.
*
* <p>For multi-topic sources, returns the minimum partition count as a conservative approach.
* If any topic has fewer partitions, that becomes the bottleneck.
*
* @param topicUris List of topic URIs to query
* @param pscProperties PSC properties for metadata client connection
* @return Minimum partition count across all topics, or -1 if count cannot be determined
*/
private static int getTopicPartitionCount(List<String> topicUris, Properties pscProperties) {
if (topicUris == null || topicUris.isEmpty()) {
LOG.warn("No topic URIs provided for partition count query.");
return -1;
}

PscMetadataClient metadataClient = null;
try {
// Create PSC configuration from properties
PscConfiguration pscConfig = new PscConfiguration();
for (String key : pscProperties.stringPropertyNames()) {
pscConfig.setProperty(key, pscProperties.getProperty(key));
}

metadataClient = new PscMetadataClient(pscConfig);

int minPartitionCount = Integer.MAX_VALUE;

for (String topicUriStr : topicUris) {
try {
TopicUri topicUri = TopicUri.validate(topicUriStr);

// Query metadata for this topic
Map<TopicUri, TopicUriMetadata> metadataMap = metadataClient.describeTopicUris(
topicUri, // cluster URI (can use full topic URI)
java.util.Collections.singleton(topicUri),
Duration.ofSeconds(10));

TopicUriMetadata metadata = metadataMap.get(topicUri);
if (metadata != null) {
int partitionCount = metadata.getTopicUriPartitions().size();
LOG.debug("Topic {} has {} partitions", topicUriStr, partitionCount);
minPartitionCount = Math.min(minPartitionCount, partitionCount);
} else {
LOG.warn("No metadata returned for topic {}", topicUriStr);
}
} catch (Exception e) {
LOG.warn("Failed to query partition count for topic {}: {}",
topicUriStr, e.getMessage());
}
}

return (minPartitionCount == Integer.MAX_VALUE) ? -1 : minPartitionCount;

} catch (Exception e) {
LOG.warn("Failed to create PSC metadata client or query partition count: {}",
e.getMessage());
return -1;
} finally {
if (metadataClient != null) {
try {
metadataClient.close();
} catch (Exception e) {
LOG.warn("Failed to close PSC metadata client: {}", e.getMessage());
}
}
}
}

/** Private constructor to prevent instantiation. */
private PscTableCommonUtils() {
throw new UnsupportedOperationException("Utility class should not be instantiated");
}
}


Loading