-
Notifications
You must be signed in to change notification settings - Fork 1.3k
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
Add multi stream ingestion support #13790
Add multi stream ingestion support #13790
Conversation
Codecov ReportAttention: Patch coverage is
Additional details and impacted files@@ Coverage Diff @@
## master #13790 +/- ##
============================================
+ Coverage 61.75% 64.03% +2.27%
- Complexity 207 1605 +1398
============================================
Files 2436 2703 +267
Lines 133233 149053 +15820
Branches 20636 22849 +2213
============================================
+ Hits 82274 95439 +13165
- Misses 44911 46620 +1709
- Partials 6048 6994 +946
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. |
@@ -686,9 +686,8 @@ public void ingestionStreamConfigsTest() { | |||
// only 1 stream config allowed |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit, update comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
done
@@ -173,15 +173,18 @@ public static void validate(TableConfig tableConfig, @Nullable Schema schema, @N | |||
|
|||
// Only allow realtime tables with non-null stream.type and LLC consumer.type | |||
if (tableConfig.getTableType() == TableType.REALTIME) { | |||
Map<String, String> streamConfigMap = IngestionConfigUtils.getStreamConfigMap(tableConfig); | |||
List<Map<String, String>> streamConfigMaps = IngestionConfigUtils.getStreamConfigMaps(tableConfig); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
let's add a validation to avoid upsert table creating with multiple topics for now. One of the most important reason is that upsert table requires same primary keys to be distributed to the same host. it will be a bit complex to validate whether all source topics are partitioned equally (partition key, partition counts, parttion algorithms).
there are other potential concerns including race condition consumption during reload, rebalance, pause ingestion etc.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a good idea, updated.
for (int i = 0; i < streamConfigs.size(); i++) { | ||
final int index = i; | ||
try { | ||
partitionIds.addAll(getPartitionIds(streamConfigs.get(index)).stream().map( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can you please elaborate on why we don't need to maintain orders for partitionIds? we use list for streamConfigs and use unordered set to store partitionids?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This function is overloading an existing same name function. The other one is also returning a Set<>. Usage of the output is only checking if partitionId exists instead of checking its order.
0bfb91f
to
1af31b2
Compare
1af31b2
to
87dacbd
Compare
Add the production running report: Notes: the feature is running with Kafka's multiple topics ingestion. We do not have resources to run it with other or multiple type of streams. |
_streamPartitionMsgOffsetFactory = | ||
StreamConsumerFactoryProvider.create(streamConfigs.get(0)).createStreamMsgOffsetFactory(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this breaks when mixing streams that do not use the same offset factory type, e.g. kinesis and kafka. (there's a lot of this specific case for offset factory, won't mark all)
We could UT, or shall we add a TODO for them since we can't easily test e2e internally?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will add a enforcement check when we fetch the streamConfigs to enforce them to be same for now.
In long term, we need to redefine the structure of streamConfig for the usage.
@@ -1294,7 +1315,7 @@ IdealState ensureAllPartitionsConsuming(TableConfig tableConfig, StreamConfig st | |||
selectStartOffset(offsetCriteria, partitionId, partitionIdToStartOffset, | |||
partitionIdToSmallestOffset, tableConfig.getTableName(), offsetFactory, | |||
latestSegmentZKMetadata.getEndOffset()); | |||
createNewConsumingSegment(tableConfig, streamConfig, latestSegmentZKMetadata, currentTimeMs, | |||
createNewConsumingSegment(tableConfig, streamConfigs.get(0), latestSegmentZKMetadata, currentTimeMs, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we use the partitionId to choose the correct streamConfig?
Or we'd need to document that segment flush settings are only used from the first streamConfig in the table config's list (though I feel different flush settings per stream will eventually be a future requirement)
@@ -87,6 +89,33 @@ public MissingConsumingSegmentFinder(String realtimeTableName, ZkHelixPropertySt | |||
} | |||
} | |||
|
|||
public MissingConsumingSegmentFinder(String realtimeTableName, ZkHelixPropertyStore<ZNRecord> propertyStore, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The old constructor is no longer used, can we remove it and update the tests?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed
_streamConfigs = streamConfigs; | ||
_partitionGroupConsumptionStatusList = partitionGroupConsumptionStatusList; | ||
_newPartitionGroupMetadataList = new ArrayList<>(); | ||
} | ||
|
||
public PartitionGroupMetadataFetcher(StreamConfig streamConfig, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Similar here, let's remove the unused constructor
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
removed
* @param tableConfig realtime table config | ||
* @return streamConfigs List of maps | ||
*/ | ||
public static List<Map<String, String>> getStreamConfigMaps(TableConfig tableConfig) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we remove the old method if it is no longer used?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed
for (Map.Entry<Integer, LLCSegmentName> entry : partitionGroupIdToLatestSegment.entrySet()) { | ||
int partitionGroupId = entry.getKey(); | ||
LLCSegmentName llcSegmentName = entry.getValue(); | ||
SegmentZKMetadata segmentZKMetadata = | ||
getSegmentZKMetadata(streamConfig.getTableNameWithType(), llcSegmentName.getSegmentName()); | ||
getSegmentZKMetadata(streamConfigs.get(0).getTableNameWithType(), llcSegmentName.getSegmentName()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: idealState.getId()
instead of .get(0)
?
: getPartitionGroupConsumptionStatusList(idealState, streamConfig); | ||
OffsetCriteria originalOffsetCriteria = streamConfig.getOffsetCriteria(); | ||
: getPartitionGroupConsumptionStatusList(idealState, streamConfigs); | ||
// FIXME: Right now, we assume topics are sharing same offset criteria |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does it make sense to add a precondition to check this?
* @param tableConfig realtime table config | ||
* @return streamConfigs List of maps | ||
*/ | ||
public static List<Map<String, String>> getStreamConfigMaps(TableConfig tableConfig) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add deprecated annotation to the old getStreamConfigMap()
if we are not removing it
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed the old one.
&& tableConfig.getIngestionConfig().getStreamIngestionConfig() != null) { | ||
List<Map<String, String>> streamConfigMaps = | ||
tableConfig.getIngestionConfig().getStreamIngestionConfig().getStreamConfigMaps(); | ||
Preconditions.checkState(streamConfigMaps.size() > 0, "Table must have at least 1 stream"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
(nit)
Preconditions.checkState(streamConfigMaps.size() > 0, "Table must have at least 1 stream"); | |
Preconditions.checkState(!streamConfigMaps.isEmpty(), "Table must have at least 1 stream"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed
List<Map<String, String>> streamConfigMaps = | ||
tableConfig.getIngestionConfig().getStreamIngestionConfig().getStreamConfigMaps(); | ||
Preconditions.checkState(streamConfigMaps.size() > 0, "Table must have at least 1 stream"); | ||
// For now, with multiple topics, we only support same type of stream (e.g. Kafka) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What is the reason for this limitation? Some comments explaining this would be good.
Only apply this check when there are multiple streams to match the current behavior
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added detailed explanations. Basically it is due to our resources not able to cover the testing of other stream types.
streamConfigMap); | ||
for (Map<String, String> streamConfigMap : streamConfigMaps) { | ||
StreamConfig.validateConsumerType( | ||
streamConfigMap.getOrDefault(StreamConfigProperties.STREAM_TYPE, "kafka"), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
(format) Not comply to Pinot Style
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not sure how this works, but the mvn check style could pass.
* @param partitionGroupConsumptionStatusList | ||
* @return | ||
*/ | ||
public static List<PartitionGroupMetadata> getPartitionGroupMetadataList(List<StreamConfig> streamConfigs, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Deprecate the old method or remove it. Please also clean up all usages of the old one
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed
87dacbd
to
cae4dc5
Compare
Thanks @itschrispeck for identifying a edge case issue and proposing the fix in the commit 6bd1307 to address the missing segments issue if cannot fetch partition metadata from the stream. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Overall LGTM, suggest we thoroughly document the current limitations (e.g. no mixed stream types)
...re/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeSegmentDataManager.java
Show resolved
Hide resolved
...t-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java
Outdated
Show resolved
Hide resolved
pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java
Outdated
Show resolved
Hide resolved
...ain/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
Outdated
Show resolved
Hide resolved
…ments Summary: Ensure transient exceptions do not prevent creating new consuming segments. If some exception is hit, attempt to reconcile any successful fetches with partition group metadata. This ensures consuming partitions are not dropped, and attempts to add and new partitions discovered successfully. Test Plan: After deployment, despite still some `TransientConsumerException`, no new missing consuming segments appear {F1002071843} {F1002071523} Reviewers: gaoxin, tingchen Reviewed By: gaoxin JIRA Issues: EVA-8951 Differential Revision: https://code.uberinternal.com/D15748639
ba53fa3
to
ad34f17
Compare
@Jackie-Jiang could you pls review again and see if we still have blockers before merging? Thanks |
Thanks again for contributing this feature. Is there a user doc associated with this feature |
@kishoreg thanks for bringing this up. After the PR merged, I would update the pinot-doc with the feature and its usage. In general, users could use the exact same way to define the table config with existing interfaces. I would provide an example in PR description. |
feature
Reference: #13780 Design Doc
Please refer to design doc for details. TLDR:
Feature tested on multiple Kafka topics with different decoder format. Due to resource limitations, not able to test other upstream source e2e.
Some TODOs:
From user point of view, the feature does not change any existing interfaces. Users could define the table config in the same way and combine with any other transform functions or instance assignment strategies. A sample table ingestion config would look like: