-
Notifications
You must be signed in to change notification settings - Fork 1k
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
KSQL-508: add metrics interceptors instead of replacing #585
Merged
Changes from 1 commit
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
4fbe023
KSQL-508: add metrics interceptors instead of replacing
rodesai 7e96cd8
Fix indentation
rodesai 5281547
Unneccessary imports
rodesai 99306fa
Address review comments
rodesai f997de1
Merge branch 'master' into KSQL-508
rodesai 1b42daa
Merge branch 'master' into KSQL-508
rodesai c3aabf3
dont use fully qualified LinkedList name
rodesai File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
23 changes: 23 additions & 0 deletions
23
ksql-engine/src/main/java/io/confluent/ksql/physical/KafkaStreamsBuilder.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,23 @@ | ||
/** | ||
* Copyright 2018 Confluent Inc. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software distributed under the License | ||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.physical; | ||
|
||
import org.apache.kafka.streams.KafkaStreams; | ||
import org.apache.kafka.streams.StreamsBuilder; | ||
import org.apache.kafka.streams.StreamsConfig; | ||
|
||
public interface KafkaStreamsBuilder { | ||
KafkaStreams buildKafkaStreams(StreamsBuilder builder, StreamsConfig conf); | ||
} |
26 changes: 26 additions & 0 deletions
26
ksql-engine/src/main/java/io/confluent/ksql/physical/KafkaStreamsBuilderImpl.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,26 @@ | ||
/** | ||
* Copyright 2018 Confluent Inc. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software distributed under the License | ||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.physical; | ||
|
||
import org.apache.kafka.streams.KafkaStreams; | ||
import org.apache.kafka.streams.StreamsBuilder; | ||
import org.apache.kafka.streams.StreamsConfig; | ||
|
||
public class KafkaStreamsBuilderImpl implements KafkaStreamsBuilder { | ||
@Override | ||
public KafkaStreams buildKafkaStreams(StreamsBuilder builder, StreamsConfig conf) { | ||
return new KafkaStreams(builder.build(), conf); | ||
} | ||
} |
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 |
---|---|---|
|
@@ -67,6 +67,7 @@ public class PhysicalPlanBuilder { | |
private final MetaStore metaStore; | ||
private final boolean updateMetastore; | ||
private final SchemaRegistryClient schemaRegistryClient; | ||
private final KafkaStreamsBuilder kafkaStreamsBuilder; | ||
|
||
public PhysicalPlanBuilder(final StreamsBuilder builder, | ||
final KsqlConfig ksqlConfig, | ||
|
@@ -76,7 +77,8 @@ public PhysicalPlanBuilder(final StreamsBuilder builder, | |
final Map<String, Object> overriddenStreamsProperties, | ||
final boolean updateMetastore, | ||
final MetaStore metaStore, | ||
final SchemaRegistryClient schemaRegistryClient) { | ||
final SchemaRegistryClient schemaRegistryClient, | ||
final KafkaStreamsBuilder kafkaStreamsBuilder) { | ||
this.builder = builder; | ||
this.ksqlConfig = ksqlConfig; | ||
this.kafkaTopicClient = kafkaTopicClient; | ||
|
@@ -86,8 +88,23 @@ public PhysicalPlanBuilder(final StreamsBuilder builder, | |
this.metaStore = metaStore; | ||
this.updateMetastore = updateMetastore; | ||
this.schemaRegistryClient = schemaRegistryClient; | ||
this.kafkaStreamsBuilder = kafkaStreamsBuilder; | ||
} | ||
|
||
public PhysicalPlanBuilder(final StreamsBuilder builder, | ||
final KsqlConfig ksqlConfig, | ||
final KafkaTopicClient kafkaTopicClient, | ||
final MetastoreUtil metastoreUtil, | ||
final FunctionRegistry functionRegistry, | ||
final Map<String, Object> overriddenStreamsProperties, | ||
final boolean updateMetastore, | ||
final MetaStore metaStore, | ||
final SchemaRegistryClient schemaRegistryClient) { | ||
this(builder, ksqlConfig, kafkaTopicClient, metastoreUtil, functionRegistry, overriddenStreamsProperties, | ||
updateMetastore, metaStore, schemaRegistryClient, new KafkaStreamsBuilderImpl()); | ||
} | ||
|
||
|
||
public QueryMetadata buildPhysicalPlan(final Pair<String, PlanNode> statementPlanPair) throws Exception { | ||
final SchemaKStream resultStream = statementPlanPair.getRight().buildStream(builder, | ||
ksqlConfig, | ||
|
@@ -135,14 +152,12 @@ private QueryMetadata buildPlanForBareQuery(final QueuedSchemaKStream schemaKStr | |
|
||
final String applicationId = addTimeSuffix(getBareQueryApplicationId(serviceId, transientQueryPrefix)); | ||
|
||
StreamsConfig streamsConfig = buildStreamsConfig(applicationId, ksqlConfig, overriddenStreamsProperties); | ||
KafkaStreams streams = buildStreams(builder, streamsConfig); | ||
KafkaStreams streams = buildStreams(builder, applicationId, ksqlConfig, overriddenStreamsProperties); | ||
|
||
SchemaKStream sourceSchemaKstream = schemaKStream.getSourceSchemaKStreams().get(0); | ||
|
||
return new QueuedQueryMetadata( | ||
statement, | ||
streamsConfig, | ||
streams, | ||
bareOutputNode, | ||
schemaKStream.getExecutionPlan(""), | ||
|
@@ -194,13 +209,12 @@ private QueryMetadata buildPlanForStructuredOutputNode(String sqlExpression, fin | |
final QueryId queryId = sinkDataSource.getPersistentQueryId(); | ||
final String applicationId = serviceId + persistanceQueryPrefix + queryId; | ||
|
||
StreamsConfig streamsConfig = buildStreamsConfig(applicationId, ksqlConfig, overriddenStreamsProperties); | ||
KafkaStreams streams = buildStreams(builder, streamsConfig); | ||
KafkaStreams streams = buildStreams(builder, applicationId, ksqlConfig, overriddenStreamsProperties); | ||
|
||
TopologyDescription topologyDescription = builder.build().describe(); | ||
|
||
return new PersistentQueryMetadata(statement, | ||
streamsConfig, streams, outputNode, schemaKStream | ||
streams, outputNode, schemaKStream | ||
.getExecutionPlan(""), queryId, | ||
(schemaKStream instanceof SchemaKTable) ? DataSource | ||
.DataSourceType.KTABLE : DataSource.DataSourceType | ||
|
@@ -223,9 +237,11 @@ private String addTimeSuffix(String original) { | |
private void updateListProperty(Map<String, Object> properties, String key, Object value) { | ||
Object obj = properties.getOrDefault(key, new java.util.LinkedList<String>()); | ||
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 guess there no need to import LL with java.util full path. |
||
List valueList; | ||
// The property value is either a comma-separated string of class names, or a list of class names | ||
if (obj instanceof String) { | ||
// If its a string just split it on the separator so we dont have to worry about adding a separator | ||
String asString = (String)obj; | ||
valueList = new LinkedList<>(Arrays.asList(asString)); | ||
valueList = new LinkedList<>(Arrays.asList(asString.split("\\s*,\\s*"))); | ||
} else if (obj instanceof List) { | ||
valueList = (List) obj; | ||
} else { | ||
|
@@ -235,8 +251,12 @@ private void updateListProperty(Map<String, Object> properties, String key, Obje | |
properties.put(key, valueList); | ||
} | ||
|
||
private StreamsConfig buildStreamsConfig(final String applicationId, final KsqlConfig ksqlConfig, | ||
final Map<String, Object> overriddenProperties) { | ||
private KafkaStreams buildStreams( | ||
final StreamsBuilder builder, | ||
final String applicationId, | ||
final KsqlConfig ksqlConfig, | ||
final Map<String, Object> overriddenProperties | ||
) { | ||
Map<String, Object> newStreamsProperties = ksqlConfig.getKsqlStreamConfigProps(); | ||
newStreamsProperties.putAll(overriddenProperties); | ||
newStreamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); | ||
|
@@ -260,12 +280,7 @@ private StreamsConfig buildStreamsConfig(final String applicationId, final KsqlC | |
ConsumerCollector.class.getCanonicalName()); | ||
updateListProperty(newStreamsProperties, StreamsConfig.producerPrefix(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG), | ||
ProducerCollector.class.getCanonicalName()); | ||
return new StreamsConfig(newStreamsProperties); | ||
} | ||
|
||
private KafkaStreams buildStreams(final StreamsBuilder builder, final StreamsConfig config) { | ||
return new KafkaStreams(builder.build(), config); | ||
return kafkaStreamsBuilder.buildKafkaStreams(builder, new StreamsConfig(newStreamsProperties)); | ||
} | ||
|
||
} | ||
|
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
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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 much better - it would be nice if you could create a fixture or mock that has internal state which is used validate its use; this would be a pita and result in a test thats a bit too functional
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 went down that path (using EasyMock) originally but there is so much boilerplate. And its not really adding any more validation than what's being done here. Both approaches validate that KafkaStreamsBuilder is used once and check that the config is consistent with what is expected. I think the only additional thing the mock would do is validate how the mock is used with respect to other mocks, which the test doesn't use anyway.