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

Added KSQL Config. Will have to add setting from CLI properties file. #39

Merged
Merged
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
6 changes: 5 additions & 1 deletion ksql-core/src/main/java/io/confluent/ksql/KSQLEngine.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.confluent.ksql.parser.tree.Table;
import io.confluent.ksql.planner.plan.PlanNode;
import io.confluent.ksql.util.DataSourceExtractor;
import io.confluent.ksql.util.KSQLConfig;
import io.confluent.ksql.util.Pair;
import io.confluent.ksql.util.PersistentQueryMetadata;
import io.confluent.ksql.util.QueryMetadata;
Expand Down Expand Up @@ -51,6 +52,8 @@ public class KSQLEngine implements Closeable {
private final Map<Long, PersistentQueryMetadata> persistentQueries;
private final Set<QueryMetadata> liveQueries;

private KSQLConfig ksqlConfig;

/**
* Runs the set of queries in the given query string.
*
Expand Down Expand Up @@ -263,8 +266,9 @@ public void close() {
public KSQLEngine(MetaStore metaStore, Map<String, Object> streamsProperties) {
validateStreamsProperties(streamsProperties);

this.ksqlConfig = new KSQLConfig(streamsProperties);
this.metaStore = metaStore;
this.queryEngine = new QueryEngine(streamsProperties);
this.queryEngine = new QueryEngine(ksqlConfig);
this.ddlEngine = new DDLEngine(this);
this.persistentQueries = new HashMap<>();
this.liveQueries = new HashSet<>();
Expand Down
21 changes: 13 additions & 8 deletions ksql-core/src/main/java/io/confluent/ksql/QueryEngine.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import io.confluent.ksql.structured.QueuedSchemaKStream;
import io.confluent.ksql.structured.SchemaKStream;
import io.confluent.ksql.structured.SchemaKTable;
import io.confluent.ksql.util.KSQLConfig;
import io.confluent.ksql.util.KSQLException;
import io.confluent.ksql.util.Pair;
import io.confluent.ksql.util.PersistentQueryMetadata;
Expand All @@ -42,9 +43,9 @@
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.kstream.KStreamBuilder;
import org.apache.kafka.streams.processor.internals.StreamsKafkaClient;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
Expand All @@ -53,22 +54,24 @@

public class QueryEngine {

private final Map<String, Object> streamsProperties;
private final KSQLConfig ksqlConfig;
private final AtomicLong queryIdCounter;

public QueryEngine(Map<String, Object> streamsProperties) {
Objects.requireNonNull(streamsProperties, "Streams properties map cannot be null as it may be mutated later on");
this.streamsProperties = streamsProperties;
private StreamsKafkaClient streamsKafkaClient;

public QueryEngine(KSQLConfig ksqlConfig) {
Objects.requireNonNull(ksqlConfig, "Streams properties map cannot be null as it may be mutated later on");
this.ksqlConfig = ksqlConfig;
this.streamsKafkaClient = new StreamsKafkaClient(new StreamsConfig(ksqlConfig.getKsqlConfigProps()));
this.queryIdCounter = new AtomicLong(1);
}

public Map<String, Object> getStreamsProperties() {
return new HashMap<>(streamsProperties);
return ksqlConfig.getKsqlConfigProps();
}

public void setStreamsProperty(String property, Object value) {
streamsProperties.put(property, value);
ksqlConfig.put(property, value);
}

public List<Pair<String, PlanNode>> buildLogicalPlans(MetaStore metaStore, List<Pair<String, Query>> queryList) {
Expand Down Expand Up @@ -146,7 +149,8 @@ public List<QueryMetadata> buildPhysicalPlans(
KStreamBuilder builder = new KStreamBuilder();

//Build a physical plan, in this case a Kafka Streams DSL
PhysicalPlanBuilder physicalPlanBuilder = new PhysicalPlanBuilder(builder);
PhysicalPlanBuilder physicalPlanBuilder = new PhysicalPlanBuilder(builder,
streamsKafkaClient, ksqlConfig);
SchemaKStream schemaKStream = physicalPlanBuilder.buildPhysicalPlan(logicalPlan);

OutputNode outputNode = physicalPlanBuilder.getPlanSink();
Expand Down Expand Up @@ -259,4 +263,5 @@ private String getBareQueryApplicationId() {
private String addTimeSuffix(String original) {
return String.format("%s_%d", original, System.currentTimeMillis());
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -11,11 +11,14 @@
import io.confluent.ksql.util.Pair;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

public class Analysis {

StructuredDataSource into;
Map<String, Object> intoProperties = new HashMap<>();
String intoFormat = null;
// TODO: Maybe have all as properties. At the moment this will only be set if format is avro.
String intoAvroSchemaFilePath = null;
Expand Down Expand Up @@ -133,5 +136,9 @@ public Expression getHavingExpression() {
public void setHavingExpression(Expression havingExpression) {
this.havingExpression = havingExpression;
}

public Map<String, Object> getIntoProperties() {
return intoProperties;
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import io.confluent.ksql.serde.avro.KSQLAvroTopicSerDe;
import io.confluent.ksql.serde.csv.KSQLCsvTopicSerDe;
import io.confluent.ksql.serde.json.KSQLJsonTopicSerDe;
import io.confluent.ksql.util.KSQLConfig;
import io.confluent.ksql.util.KSQLException;
import io.confluent.ksql.util.Pair;
import org.apache.kafka.connect.data.Field;
Expand Down Expand Up @@ -269,6 +270,7 @@ protected Node visitTable(final Table node, final AnalysisContext context) {
}
serde = serde.substring(1, serde.length() - 1);
analysis.setIntoFormat(serde);
analysis.getIntoProperties().put(DDLConfig.FORMAT_PROPERTY, serde);
if ("AVRO".equals(serde)) {
String avroSchemaFilePath = "/tmp/" + into.getName() + ".avro";
if (node.getProperties().get(DDLConfig.AVRO_SCHEMA_FILE) != null) {
Expand All @@ -281,6 +283,8 @@ protected Node visitTable(final Table node, final AnalysisContext context) {
avroSchemaFilePath = avroSchemaFilePath.substring(1, avroSchemaFilePath.length() - 1);
}
analysis.setIntoAvroSchemaFilePath(avroSchemaFilePath);
analysis.getIntoProperties().put(DDLConfig.AVRO_SCHEMA_FILE, avroSchemaFilePath);

}
}

Expand All @@ -294,10 +298,32 @@ protected Node visitTable(final Table node, final AnalysisContext context) {
}
intoKafkaTopicName = intoKafkaTopicName.substring(1, intoKafkaTopicName.length() - 1);
analysis.setIntoKafkaTopicName(intoKafkaTopicName);
analysis.getIntoProperties().put(DDLConfig.KAFKA_TOPIC_NAME_PROPERTY, intoKafkaTopicName);
}
if (node.getProperties().get(KSQLConfig.SINK_NUMBER_OF_PARTITIONS) != null) {
try {
int numberOfPartitions = Integer.parseInt(node.getProperties().get(KSQLConfig.SINK_NUMBER_OF_PARTITIONS).toString());
analysis.getIntoProperties().put(KSQLConfig.SINK_NUMBER_OF_PARTITIONS, numberOfPartitions);

} catch (NumberFormatException e) {
throw new KSQLException("Invalid number of partitions in WITH clause: " + node.getProperties().get(KSQLConfig.SINK_NUMBER_OF_PARTITIONS).toString());
}
}

if (node.getProperties().get(KSQLConfig.SINK_NUMBER_OF_REPLICATIONS) != null) {
try {
short numberOfReplications = Short.parseShort(node.getProperties().get(KSQLConfig.SINK_NUMBER_OF_REPLICATIONS).toString());
analysis.getIntoProperties().put(KSQLConfig.SINK_NUMBER_OF_REPLICATIONS, numberOfReplications);
} catch (NumberFormatException e) {
throw new KSQLException("Invalid number of replications in WITH clause: " + node
.getProperties().get(KSQLConfig.SINK_NUMBER_OF_REPLICATIONS).toString());
}
}

} else {
throw new KSQLException("INTO clause is not set correctly!");
}

analysis.setInto(into);
return null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import io.confluent.ksql.structured.SchemaKGroupedStream;
import io.confluent.ksql.structured.SchemaKStream;
import io.confluent.ksql.structured.SchemaKTable;
import io.confluent.ksql.util.KSQLConfig;
import io.confluent.ksql.util.KSQLException;
import io.confluent.ksql.util.SchemaUtil;
import io.confluent.ksql.util.SerDeUtil;
Expand All @@ -47,6 +48,7 @@
import org.apache.kafka.streams.kstream.KeyValueMapper;
import org.apache.kafka.streams.kstream.Reducer;
import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.processor.internals.StreamsKafkaClient;

import java.util.ArrayList;
import java.util.Arrays;
Expand All @@ -59,10 +61,15 @@ public class PhysicalPlanBuilder {

KStreamBuilder builder;
OutputNode planSink = null;
StreamsKafkaClient streamsKafkaClient = null;

public PhysicalPlanBuilder(final KStreamBuilder builder) {
KSQLConfig ksqlConfig = null;

public PhysicalPlanBuilder(final KStreamBuilder builder, final StreamsKafkaClient
streamsKafkaClient, final KSQLConfig ksqlConfig) {
this.builder = builder;
this.streamsKafkaClient = streamsKafkaClient;
this.ksqlConfig = ksqlConfig;
}

public SchemaKStream buildPhysicalPlan(final PlanNode logicalPlanRoot) throws Exception {
Expand Down Expand Up @@ -108,8 +115,7 @@ private SchemaKStream buildOutput(final OutputNode outputNode) throws Exception
ksqlStructuredDataOutputNode.getSource(),
SchemaUtil.removeImplicitRowKeyFromSchema(ksqlStructuredDataOutputNode.getSchema()),
ksqlStructuredDataOutputNode.getKsqlTopic(),
ksqlStructuredDataOutputNode.getKafkaTopicName()
);
ksqlStructuredDataOutputNode.getKafkaTopicName(), ksqlStructuredDataOutputNode.getOutputProperties());
if (ksqlStructuredDataOutputNodeNoRowKey.getKsqlTopic()
.getKsqlTopicSerDe() instanceof KSQLAvroTopicSerDe) {
KSQLAvroTopicSerDe ksqlAvroTopicSerDe = (KSQLAvroTopicSerDe) ksqlStructuredDataOutputNodeNoRowKey
Expand All @@ -119,11 +125,24 @@ private SchemaKStream buildOutput(final OutputNode outputNode) throws Exception
.getSchemaFilePath());
}

KSQLConfig ksqlConfigClone = ksqlConfig.clone();

Map<String, Object> outputProperties = ksqlStructuredDataOutputNodeNoRowKey
.getOutputProperties();
if (outputProperties.containsKey(KSQLConfig.SINK_NUMBER_OF_PARTITIONS)) {
ksqlConfigClone.put(KSQLConfig.SINK_NUMBER_OF_PARTITIONS, outputProperties.get(KSQLConfig
.SINK_NUMBER_OF_PARTITIONS));
}
if (outputProperties.containsKey(KSQLConfig.SINK_NUMBER_OF_REPLICATIONS)) {
ksqlConfigClone.put(KSQLConfig.SINK_NUMBER_OF_REPLICATIONS, outputProperties.get(KSQLConfig
.SINK_NUMBER_OF_REPLICATIONS));
}

SchemaKStream resultSchemaStream = schemaKStream.into(ksqlStructuredDataOutputNodeNoRowKey
.getKafkaTopicName(), SerDeUtil
.getRowSerDe(ksqlStructuredDataOutputNodeNoRowKey.getKsqlTopic().getKsqlTopicSerDe(),
ksqlStructuredDataOutputNodeNoRowKey.getSchema()), rowkeyIndexes);
ksqlStructuredDataOutputNodeNoRowKey.getSchema()), rowkeyIndexes,
streamsKafkaClient, ksqlConfigClone);


KSQLStructuredDataOutputNode ksqlStructuredDataOutputNodeWithRowkey = new
Expand All @@ -132,7 +151,8 @@ private SchemaKStream buildOutput(final OutputNode outputNode) throws Exception
ksqlStructuredDataOutputNodeNoRowKey.getSource(),
SchemaUtil.addImplicitKeyToSchema(ksqlStructuredDataOutputNodeNoRowKey.getSchema()),
ksqlStructuredDataOutputNodeNoRowKey.getKsqlTopic(),
ksqlStructuredDataOutputNodeNoRowKey.getKafkaTopicName()
ksqlStructuredDataOutputNodeNoRowKey.getKafkaTopicName(),
ksqlStructuredDataOutputNodeNoRowKey.getOutputProperties()
);
this.planSink = ksqlStructuredDataOutputNodeWithRowkey;
return resultSchemaStream;
Expand Down Expand Up @@ -432,7 +452,7 @@ private KSQLStructuredDataOutputNode addAvroSchemaToResultTopic(final KSQLStruct
KSQLStructuredDataOutputNode newKSQLStructuredDataOutputNode = new KSQLStructuredDataOutputNode(
ksqlStructuredDataOutputNode.getId(), ksqlStructuredDataOutputNode.getSource(),
ksqlStructuredDataOutputNode.getSchema(), newKSQLTopic,
ksqlStructuredDataOutputNode.getKafkaTopicName());
ksqlStructuredDataOutputNode.getKafkaTopicName(), ksqlStructuredDataOutputNode.getOutputProperties());
return newKSQLStructuredDataOutputNode;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ private OutputNode buildOutputNode(final Schema inputSchema, final PlanNode sour
sourcePlanNode,
inputSchema, intoStructuredDataSource.getKsqlTopic(),
intoStructuredDataSource.getKsqlTopic()
.getTopicName());
.getTopicName(), analysis.getIntoProperties());

}
throw new RuntimeException("INTO clause is not supported in SELECT.");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,23 +9,29 @@
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Schema;

import java.util.Map;

public class KSQLStructuredDataOutputNode extends OutputNode {

final String kafkaTopicName;
final KSQLTopic ksqlTopic;
private final Field keyField;
final Map<String, Object> outputProperties;


@JsonCreator
public KSQLStructuredDataOutputNode(@JsonProperty("id") final PlanNodeId id,
@JsonProperty("source") final PlanNode source,
@JsonProperty("schema") final Schema schema,
@JsonProperty("ksqlTopic") final KSQLTopic ksqlTopic,
@JsonProperty("topicName") final String topicName) {
@JsonProperty("topicName") final String topicName,
@JsonProperty("outputProperties") final Map<String, Object>
outputProperties) {
super(id, source, schema);
this.kafkaTopicName = topicName;
this.keyField = source.getKeyField();
this.ksqlTopic = ksqlTopic;
this.outputProperties = outputProperties;
}

public String getKafkaTopicName() {
Expand All @@ -41,4 +47,7 @@ public KSQLTopic getKsqlTopic() {
return ksqlTopic;
}

public Map<String, Object> getOutputProperties() {
return outputProperties;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -6,12 +6,14 @@
import io.confluent.ksql.parser.tree.Expression;
import io.confluent.ksql.physical.GenericRow;
import io.confluent.ksql.serde.KSQLTopicSerDe;
import io.confluent.ksql.util.KSQLConfig;

import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.processor.internals.StreamsKafkaClient;

import java.util.List;
import java.util.Set;
Expand Down Expand Up @@ -44,7 +46,8 @@ public SynchronousQueue<KeyValue<String, GenericRow>> getQueue() {

@Override
public SchemaKStream into(String kafkaTopicName, Serde<GenericRow> topicValueSerDe,
Set<Integer> rowkeyIndexes) {
Set<Integer> rowkeyIndexes, final StreamsKafkaClient
streamsKafkaClient, KSQLConfig ksqlConfig) {
throw new UnsupportedOperationException();
}

Expand Down
Loading