-
Notifications
You must be signed in to change notification settings - Fork 3.7k
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
Enable querying entirely cold datasources #16676
Changes from 1 commit
ca33d8b
1abba25
033c420
96e77c1
1739dce
6ed0b8b
966f5ac
d586a8e
b72b822
ad58f9b
2944959
b09df54
254cea2
25d23c6
7169b6d
89e2d64
ba380ec
d3c112c
debcb42
479382c
87113ca
2a66ae1
695ba92
a265cf0
8182f39
766082e
2bea1fb
8582f5d
7ed2d96
47bf4ab
8fb0a04
87a0926
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -65,7 +65,6 @@ | |
import java.util.Optional; | ||
import java.util.Set; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.ConcurrentMap; | ||
import java.util.concurrent.ConcurrentSkipListMap; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.Future; | ||
|
@@ -100,11 +99,11 @@ public class CoordinatorSegmentMetadataCache extends AbstractSegmentMetadataCach | |
private final SegmentReplicationStatusManager segmentReplicationStatusManager; | ||
|
||
// Schema for datasources from cold segments | ||
private final ConcurrentMap<String, DataSourceInformation> coldDatasourceSchema = new ConcurrentHashMap<>(); | ||
private final long coldDatasourceSchemaExecDurationMillis; | ||
private final ScheduledExecutorService coldDSScehmaExec; | ||
private final ConcurrentHashMap<String, DataSourceInformation> coldSchemaTable = new ConcurrentHashMap<>(); | ||
private final long coldSchemaExecPeriodMillis; | ||
private final ScheduledExecutorService coldScehmaExec; | ||
private @Nullable Future<?> cacheExecFuture = null; | ||
private @Nullable Future<?> coldDSSchemaExecFuture = null; | ||
private @Nullable Future<?> coldSchemaExecFuture = null; | ||
|
||
@Inject | ||
public CoordinatorSegmentMetadataCache( | ||
|
@@ -128,11 +127,11 @@ public CoordinatorSegmentMetadataCache( | |
this.segmentSchemaBackfillQueue = segmentSchemaBackfillQueue; | ||
this.sqlSegmentsMetadataManager = sqlSegmentsMetadataManager; | ||
this.segmentReplicationStatusManager = segmentReplicationStatusManager; | ||
this.coldDatasourceSchemaExecDurationMillis = | ||
this.coldSchemaExecPeriodMillis = | ||
segmentsMetadataManagerConfigSupplier.get().getPollDuration().getMillis(); | ||
coldDSScehmaExec = Executors.newSingleThreadScheduledExecutor( | ||
coldScehmaExec = Executors.newSingleThreadScheduledExecutor( | ||
new ThreadFactoryBuilder() | ||
.setNameFormat("DruidColdDSSchema-ScheduledExecutor-%d") | ||
.setNameFormat("DruidColdSchema-ScheduledExecutor-%d") | ||
.setDaemon(true) | ||
.build() | ||
); | ||
|
@@ -204,12 +203,15 @@ public void stop() | |
{ | ||
callbackExec.shutdownNow(); | ||
cacheExec.shutdownNow(); | ||
coldDSScehmaExec.shutdownNow(); | ||
coldScehmaExec.shutdownNow(); | ||
segmentSchemaCache.onLeaderStop(); | ||
segmentSchemaBackfillQueue.onLeaderStop(); | ||
if (cacheExecFuture != null) { | ||
cacheExecFuture.cancel(true); | ||
} | ||
if (coldSchemaExecFuture != null) { | ||
coldSchemaExecFuture.cancel(true); | ||
} | ||
} | ||
|
||
public void onLeaderStart() | ||
|
@@ -218,9 +220,9 @@ public void onLeaderStart() | |
try { | ||
segmentSchemaBackfillQueue.onLeaderStart(); | ||
cacheExecFuture = cacheExec.submit(this::cacheExecLoop); | ||
coldDSSchemaExecFuture = coldDSScehmaExec.schedule( | ||
coldSchemaExecFuture = coldScehmaExec.schedule( | ||
this::coldDatasourceSchemaExec, | ||
coldDatasourceSchemaExecDurationMillis, | ||
coldSchemaExecPeriodMillis, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is there a specific reason to undocumented these properties. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
We are not backfilling segment here. It is just looping over the segments, identifying cold segment and building their schema. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since this exec iterates over all the segments, what things do we have to figure out how much time it took for execution? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Makes sense. I am logging details if the execution duration is greater than 50 seconds. |
||
TimeUnit.MILLISECONDS | ||
); | ||
|
||
|
@@ -239,8 +241,8 @@ public void onLeaderStop() | |
if (cacheExecFuture != null) { | ||
cacheExecFuture.cancel(true); | ||
} | ||
if (coldDSSchemaExecFuture != null) { | ||
coldDSSchemaExecFuture.cancel(true); | ||
if (coldSchemaExecFuture != null) { | ||
coldSchemaExecFuture.cancel(true); | ||
} | ||
segmentSchemaCache.onLeaderStop(); | ||
segmentSchemaBackfillQueue.onLeaderStop(); | ||
|
@@ -387,17 +389,19 @@ public DataSourceInformation getDatasource(String name) | |
{ | ||
DataSourceInformation dataSourceInformation = tables.get(name); | ||
if (dataSourceInformation != null) { | ||
// implies that the datasource is entirely cold | ||
return dataSourceInformation; | ||
} | ||
return coldDatasourceSchema.get(name); | ||
return coldSchemaTable.get(name); | ||
} | ||
|
||
@Override | ||
public Map<String, DataSourceInformation> getDataSourceInformationMap() | ||
{ | ||
Map<String, DataSourceInformation> copy = new HashMap<>(tables); | ||
|
||
for (Map.Entry<String, DataSourceInformation> entry : coldDatasourceSchema.entrySet()) { | ||
for (Map.Entry<String, DataSourceInformation> entry : coldSchemaTable.entrySet()) { | ||
// add entirely cold datasource schema | ||
copy.computeIfAbsent(entry.getKey(), value -> entry.getValue()); | ||
} | ||
return ImmutableMap.copyOf(copy); | ||
|
@@ -452,7 +456,7 @@ public void refresh(final Set<SegmentId> segmentsToRefresh, final Set<String> da | |
|
||
RowSignature mergedSignature = rowSignature; | ||
|
||
DataSourceInformation coldDatasourceInformation = coldDatasourceSchema.get(dataSource); | ||
DataSourceInformation coldDatasourceInformation = coldSchemaTable.get(dataSource); | ||
if (coldDatasourceInformation != null) { | ||
if (rowSignature == null) { | ||
mergedSignature = coldDatasourceInformation.getRowSignature(); | ||
|
@@ -498,6 +502,7 @@ private Set<SegmentId> filterSegmentWithCachedSchema(Set<SegmentId> segmentIds) | |
return cachedSegments; | ||
} | ||
|
||
@VisibleForTesting | ||
protected void coldDatasourceSchemaExec() | ||
{ | ||
Collection<ImmutableDruidDataSource> immutableDataSources = | ||
|
@@ -513,8 +518,8 @@ protected void coldDatasourceSchemaExec() | |
Collection<DataSegment> dataSegments = dataSource.getSegments(); | ||
|
||
for (DataSegment segment : dataSegments) { | ||
if (segmentReplicationStatusManager.getReplicationFactor(segment.getId()) != null | ||
&& segmentReplicationStatusManager.getReplicationFactor(segment.getId()) != 0) { | ||
Integer replicationFactor = segmentReplicationStatusManager.getReplicationFactor(segment.getId()); | ||
if (replicationFactor != null && replicationFactor != 0) { | ||
continue; | ||
} | ||
Optional<SchemaPayloadPlus> optionalSchema = segmentSchemaCache.getSchemaForSegment(segment.getId()); | ||
|
@@ -537,9 +542,10 @@ protected void coldDatasourceSchemaExec() | |
|
||
log.debug("[%s] signature from cold segments is [%s]", dataSourceName, coldSignature); | ||
|
||
coldDatasourceSchema.put(dataSourceName, new DataSourceInformation(dataSourceName, coldSignature)); | ||
coldSchemaTable.put(dataSourceName, new DataSourceInformation(dataSourceName, coldSignature)); | ||
|
||
// update tables map with merged schema | ||
// update tables map with merged schema, if signature doesn't exist we do not add entry in this table | ||
// schema for entirely cold datasource is maintained separately | ||
tables.computeIfPresent( | ||
dataSourceName, | ||
(ds, info) -> { | ||
|
@@ -560,14 +566,15 @@ protected void coldDatasourceSchemaExec() | |
} | ||
|
||
// remove any stale datasource from the map | ||
coldDatasourceSchema.keySet().retainAll(dataSources); | ||
coldSchemaTable.keySet().retainAll(dataSources); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we have a test case for this ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, in |
||
} | ||
|
||
private RowSignature mergeHotAndColdSchema(RowSignature hot, RowSignature cold) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am very surprised you need a new method here. There should be existing logic which does this no ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I can refactor this a bit to have a single method for merging the RowSignature. |
||
{ | ||
final Map<String, ColumnType> columnTypes = new LinkedHashMap<>(); | ||
|
||
List<RowSignature> signatures = new ArrayList<>(); | ||
// hot datasource schema takes precedence | ||
signatures.add(hot); | ||
signatures.add(cold); | ||
|
||
|
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.
Why is this a demon thread ?
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 will update, we don't need a daemon thread here.