-
Notifications
You must be signed in to change notification settings - Fork 18
Adding configurable option to set source parallelism #103
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
Merged
jeffxiang
merged 9 commits into
pinterest:main
from
KevBrowne:add_configurable_source_parallelism
Nov 5, 2025
Merged
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
9dcc892
Adding configurable optional to set source parallelism
051e437
fixes
16708a9
make source infer parallelism
a009168
adding shuffle
2569d42
revert pom
40f5446
removing inference and keeping boolean flag
ace54aa
rework rescale apply logic
89e91d4
make common util
e20b397
create TableCommonUtils
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
183 changes: 183 additions & 0 deletions
183
...src/main/java/com/pinterest/flink/streaming/connectors/psc/table/PscTableCommonUtils.java
KevBrowne marked this conversation as resolved.
Show resolved
Hide resolved
|
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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"); | ||
| } | ||
| } | ||
|
|
||
|
|
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.