-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat(executor): Allow SourceDescription returning executors to return…
… cluster wide stream stats (#7252) * feat: add random-beans libraries These give a simple implementation of object mother pattern. Again this should help us simplify testing a bit * refactor(executor): turn RemoteDataAugmenter into an Executor There is really no reason for this class to handle the transform. Instea, let each call site ow modify the results returned from RemoteHostExecutor in situ as they please. It's fine to return the first KsqlEntity always because we run the executor on singular statements, so KsqlEntityList always has length 1 * feat(executor): add an executor for fetching remote source descriptions * feat(executor): encapsulate cluster wide query metrics ClusterQueryStats encapsulates the cluster wide query stats for the entire cluster. It comes with one constructor for assembling the stats divided by host * fix(test): get to the bottom of the test flakes A few problems fixed in this commit: 1. some tests were missing the correct mockito test runner - this could cause trouble if some of the code called out to by these uses mockito mocks 2. rules need to be public and annotated, otherwise they cause races 3. remove static declaration of side effects - also caused races * feat(executor): collate cluster wide source stats on all ListSources This means SourceDescription queries will be able to give you a per-node breakdown of stats available on the new fields: - clusterStatistics - clusterErrorStats
- Loading branch information
Showing
21 changed files
with
708 additions
and
178 deletions.
There are no files selected for viewing
This file contains 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 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 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 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 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 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
100 changes: 100 additions & 0 deletions
100
ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ClusterQueryStats.java
This file contains 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,100 @@ | ||
/* | ||
* Copyright 2021 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
|
||
package io.confluent.ksql.rest.server.execution; | ||
|
||
import static com.google.common.collect.ImmutableMap.toImmutableMap; | ||
|
||
import com.google.common.collect.ImmutableMap; | ||
import io.confluent.ksql.metrics.TopicSensors; | ||
import io.confluent.ksql.rest.entity.SourceDescription; | ||
import io.confluent.ksql.util.KsqlHostInfo; | ||
import java.util.Collection; | ||
import java.util.Map; | ||
|
||
public final class ClusterQueryStats { | ||
private final ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> stats; | ||
private final ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> errors; | ||
private final String sourceName; | ||
|
||
private ClusterQueryStats( | ||
final ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> stats, | ||
final ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> errors, | ||
final String sourceName | ||
) { | ||
this.stats = stats; | ||
this.errors = errors; | ||
this.sourceName = sourceName; | ||
} | ||
|
||
public static ClusterQueryStats create( | ||
final KsqlHostInfo localHostInfo, | ||
final SourceDescription localSourceDescription, | ||
final Collection<RemoteSourceDescription> remoteSourceDescriptions | ||
) { | ||
final Map<KsqlHostInfo, SourceDescription> rds = remoteSourceDescriptions.stream() | ||
.collect(toImmutableMap( | ||
RemoteSourceDescription::getKsqlHostInfo, | ||
RemoteSourceDescription::getSourceDescription | ||
)); | ||
|
||
|
||
final ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> remoteStats = rds | ||
.entrySet() | ||
.stream() | ||
.collect( | ||
toImmutableMap( | ||
Map.Entry::getKey, | ||
e -> ImmutableMap.copyOf(e.getValue().getStatisticsMap()) | ||
) | ||
); | ||
final ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> remoteErrors = rds | ||
.entrySet() | ||
.stream() | ||
.collect( | ||
toImmutableMap( | ||
Map.Entry::getKey, | ||
e -> ImmutableMap.copyOf(e.getValue().getErrorStatsMap()) | ||
) | ||
); | ||
|
||
|
||
final ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> stats = ImmutableMap | ||
.<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>>builder() | ||
.put(localHostInfo, localSourceDescription.getStatisticsMap()) | ||
.putAll(remoteStats) | ||
.build(); | ||
|
||
final ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> errors = ImmutableMap | ||
.<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>>builder() | ||
.put(localHostInfo, localSourceDescription.getErrorStatsMap()) | ||
.putAll(remoteErrors) | ||
.build(); | ||
|
||
return new ClusterQueryStats(stats, errors, localSourceDescription.getName()); | ||
} | ||
|
||
public ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> getStats() { | ||
return stats; | ||
} | ||
|
||
public ImmutableMap<KsqlHostInfo, ImmutableMap<String, TopicSensors.Stat>> getErrors() { | ||
return errors; | ||
} | ||
|
||
public String getSourceName() { | ||
return sourceName; | ||
} | ||
} |
Oops, something went wrong.