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

Add multi stream ingestion support #13790

Open
wants to merge 3 commits into
base: master
Choose a base branch
from
Open
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 @@ -562,10 +562,13 @@ protected void configure() {
_helixResourceManager.getAllRealtimeTables().forEach(rt -> {
TableConfig tableConfig = _helixResourceManager.getTableConfig(rt);
if (tableConfig != null) {
Map<String, String> streamConfigMap = IngestionConfigUtils.getStreamConfigMap(tableConfig);
List<Map<String, String>> streamConfigMaps = IngestionConfigUtils.getStreamConfigMaps(tableConfig);
try {
StreamConfig.validateConsumerType(streamConfigMap.getOrDefault(StreamConfigProperties.STREAM_TYPE, "kafka"),
streamConfigMap);
for (Map<String, String> streamConfigMap : streamConfigMaps) {
StreamConfig.validateConsumerType(
streamConfigMap.getOrDefault(StreamConfigProperties.STREAM_TYPE, "kafka"),
streamConfigMap);
}
} catch (Exception e) {
existingHlcTables.add(rt);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.helix.model.ExternalView;
import org.apache.helix.model.IdealState;
Expand Down Expand Up @@ -419,10 +420,11 @@ private void updateSegmentMetrics(String tableNameWithType, TableConfig tableCon
numInvalidEndTime);

if (tableType == TableType.REALTIME && tableConfig != null) {
StreamConfig streamConfig =
new StreamConfig(tableConfig.getTableName(), IngestionConfigUtils.getStreamConfigMap(tableConfig));
List<StreamConfig> streamConfigs = IngestionConfigUtils.getStreamConfigMaps(tableConfig).stream().map(
streamConfig -> new StreamConfig(tableConfig.getTableName(), streamConfig)
).collect(Collectors.toList());
new MissingConsumingSegmentFinder(tableNameWithType, propertyStore, _controllerMetrics,
streamConfig).findAndEmitMetrics(idealState);
streamConfigs).findAndEmitMetrics(idealState);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,4 +99,28 @@ public static List<PartitionGroupMetadata> getPartitionGroupMetadataList(StreamC
throw new RuntimeException(fetcherException);
}
}

/**
* Fetches the list of {@link PartitionGroupMetadata} for the new partition groups for the stream,
* with the help of the {@link PartitionGroupConsumptionStatus} of the current partitionGroups.
* In particular, this method is used to fetch from multiple stream topics.
* @param streamConfigs
* @param partitionGroupConsumptionStatusList
* @return
*/
public static List<PartitionGroupMetadata> getPartitionGroupMetadataList(List<StreamConfig> streamConfigs,
List<PartitionGroupConsumptionStatus> partitionGroupConsumptionStatusList) {
PartitionGroupMetadataFetcher partitionGroupMetadataFetcher =
new PartitionGroupMetadataFetcher(streamConfigs, partitionGroupConsumptionStatusList);
try {
DEFAULT_IDEALSTATE_UPDATE_RETRY_POLICY.attempt(partitionGroupMetadataFetcher);
return partitionGroupMetadataFetcher.getPartitionGroupMetadataList();
} catch (Exception e) {
Exception fetcherException = partitionGroupMetadataFetcher.getException();
LOGGER.error("Could not get PartitionGroupMetadata for topic: {} of table: {}",
streamConfigs.stream().map(streamConfig -> streamConfig.getTopicName()).reduce((a, b) -> a + "," + b),
streamConfigs.get(0).getTableNameWithType(), fetcherException);
throw new RuntimeException(fetcherException);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,9 @@
import java.time.Instant;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.helix.AccessOption;
import org.apache.helix.model.IdealState;
import org.apache.helix.store.zk.ZkHelixPropertyStore;
Expand Down Expand Up @@ -87,6 +89,33 @@ public MissingConsumingSegmentFinder(String realtimeTableName, ZkHelixPropertySt
}
}

public MissingConsumingSegmentFinder(String realtimeTableName, ZkHelixPropertyStore<ZNRecord> propertyStore,
ControllerMetrics controllerMetrics, List<StreamConfig> streamConfigs) {
_realtimeTableName = realtimeTableName;
_controllerMetrics = controllerMetrics;
_segmentMetadataFetcher = new SegmentMetadataFetcher(propertyStore, controllerMetrics);
_streamPartitionMsgOffsetFactory =
StreamConsumerFactoryProvider.create(streamConfigs.get(0)).createStreamMsgOffsetFactory();

// create partition group id to largest stream offset map
_partitionGroupIdToLargestStreamOffsetMap = new HashMap<>();
streamConfigs.stream().map(streamConfig -> {
streamConfig.setOffsetCriteria(OffsetCriteria.LARGEST_OFFSET_CRITERIA);
return streamConfig;
});
try {
PinotTableIdealStateBuilder.getPartitionGroupMetadataList(streamConfigs, Collections.emptyList())
.forEach(metadata -> {
_partitionGroupIdToLargestStreamOffsetMap.put(metadata.getPartitionGroupId(), metadata.getStartOffset());
});
} catch (Exception e) {
LOGGER.warn("Problem encountered in fetching stream metadata for topics: {} of table: {}. "
+ "Continue finding missing consuming segment only with ideal state information.",
streamConfigs.stream().map(streamConfig -> streamConfig.getTopicName()).collect(Collectors.toList()),
streamConfigs.get(0).getTableNameWithType());
}
}

@VisibleForTesting
MissingConsumingSegmentFinder(String realtimeTableName, SegmentMetadataFetcher segmentMetadataFetcher,
Map<Integer, StreamPartitionMsgOffset> partitionGroupIdToLargestStreamOffsetMap,
Expand Down
Loading
Loading