druid
druid copied to clipboard
Enable querying entirely cold datasources
Problem
Currently, datasource schema doesn’t include columns from cold segments. This makes it impossible to query entirely cold datasource.
Approach
- Mechanism to backfill schema for cold segments in the metadata database. Note, that this is required only for segments created prior to enabling CentralizedDatasourceSchema feature.
- Update datasource schema building logic on the Coordinator to include schema from cold segments.
- Make Brokers aware of entirely cold datasource.
Backfill schema for cold segments
Leverage the existing schema backfill flow added as part of CentralizedDatasourceSchema
feature. Users are supposed to manually load the cold segments by making their replication factor as 1 and once the schema is backfilled (can be verified from the metadata database) they can unload the segment.
Handling entirely cold datasource
The problem with cold datasource is that Broker just doesn’t know about the datasource if none of the segment are available. So, the datasource wouldn’t even appear on the console for querying. We need a way for the Brokers to be aware of cold datasource, so that it can fetch its schema from the Coordinator.
Currently, brokers request schema for available datasources from Coordinator in each refresh cycle. Brokers now poll set of used datasources from the Coordinator first and then request their schema from the Coordinator.
Once Broker has schema for Cold datasources, it will show up in the console and become available for querying.
Key changes
-
CoordinatorSegmentMetadataCache
- It runs a scheduled thread to fetch used segments and build datasource schema from cold segments. It then merges this schema with datasource schema built using hot segments.
- The refresh logic is also updated to merge the hot datasource schema with cold schema.
-
BrokerSegmentMetadataCache
- The refresh condition is slightly updated, refresh is executed in each cycle if the feature is enabled.
- The refresh logic is also updated to poll used datasources from the Coordinator. This way Broker can fetch cold datasource schema.
-
DruidCoordinator
- Created a new class
SegmentReplicationStatusManager
which managessegmentReplicationStatus
&broadcastSegments
state. This was needed to avoid cyclic dependency betweenDruidCoordinator
andCoordinatorSegmentMetadataCache
.
- Created a new class
This PR has:
- [X] been self-reviewed.
- [ ] using the concurrency checklist (Remove this item if the PR doesn't have any relation to concurrency.)
- [ ] added documentation for new or modified features or behaviors.
- [ ] a release note entry in the PR description.
- [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
- [ ] added or updated version, license, or notice information in licenses.yaml
- [X] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
- [X] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
- [ ] added integration tests.
- [X] been tested in a test Druid cluster.