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

Don't replay all previous events found on the command topic #454 #480

Merged
merged 5 commits into from
Dec 1, 2017
Merged
Show file tree
Hide file tree
Changes from 3 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
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ public class StandaloneExecutor {

private static final Logger log = LoggerFactory.getLogger(StandaloneExecutor.class);

KsqlEngine ksqlEngine;
private final KsqlEngine ksqlEngine;

public StandaloneExecutor(Map streamProperties) throws ExecutionException, InterruptedException {
KsqlConfig ksqlConfig = new KsqlConfig(streamProperties);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -423,7 +423,7 @@ private void printAsTable(KsqlEntity ksqlEntity) {
columnHeaders = Arrays.asList("Query ID", "Kafka Topic", "Query String");
rowValues = runningQueries.stream()
.map(runningQuery -> Arrays.asList(
Long.toString(runningQuery.getId()),
runningQuery.getId().toString(),
runningQuery.getKafkaTopic(),
runningQuery.getQueryString()
)).collect(Collectors.toList());
Expand Down
12 changes: 2 additions & 10 deletions ksql-cli/src/test/java/io/confluent/ksql/CliTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@ public class CliTest extends TestRunner {

@ClassRule
public static final EmbeddedSingleNodeKafkaCluster CLUSTER = new EmbeddedSingleNodeKafkaCluster();

private static final String COMMANDS_KSQL_TOPIC_NAME = KsqlRestApplication.COMMANDS_KSQL_TOPIC_NAME;
private static final int PORT = 9098;
private static final String LOCAL_REST_SERVER_ADDR = "http://localhost:" + PORT;
Expand Down Expand Up @@ -183,23 +182,16 @@ private static void testCreateStreamAsSelect(String selectQuery, Schema resultSc
selectQuery += ";";
}
String resultKStreamName = "RESULT";
String resultTopicName = resultKStreamName;
final String queryString = "CREATE STREAM " + resultKStreamName + " AS " + selectQuery;

/* Start Stream Query */
test(queryString, build("Stream created and running"));

/* Assert Results */
Map<String, GenericRow> results = topicConsumer.readResults(resultTopicName, resultSchema, expectedResults.size(), new StringDeserializer());

assertThat(results, equalTo(expectedResults));
Map<String, GenericRow> results = topicConsumer.readResults(resultKStreamName, resultSchema, expectedResults.size(), new StringDeserializer());

/* Get first column of the first row in the result set to obtain the queryID */
String queryID = (String) ((List) run("list queries").data.toArray()[0]).get(0);

/* Clean Up */
run("terminate query " + queryID);
dropStream(resultKStreamName);
assertThat(results, equalTo(expectedResults));
}

private static void dropStream(String name) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import io.confluent.ksql.FakeException;
import io.confluent.ksql.TestTerminal;
import io.confluent.ksql.query.QueryId;
import io.confluent.ksql.serde.DataSource;
import io.confluent.ksql.GenericRow;
import io.confluent.ksql.rest.client.KsqlRestClient;
Expand Down Expand Up @@ -95,11 +96,11 @@ public void testPrintKSqlEntityList() throws IOException {
properties.put("k3", true);

List<Queries.RunningQuery> queries = new ArrayList<>();
queries.add(new Queries.RunningQuery("select * from t1", "TestTopic", 1));
queries.add(new Queries.RunningQuery("select * from t1", "TestTopic", new QueryId("0")));

for (int i = 0; i < 5; i++) {
KsqlEntityList entityList = new KsqlEntityList(Arrays.asList(
new CommandStatusEntity("e", "topic/1", "SUCCESS", "Success Message"),
new CommandStatusEntity("e", "topic/1/create", "SUCCESS", "Success Message"),
new ErrorMessageEntity("e", new FakeException()),
new PropertiesList("e", properties),
new Queries("e", queries),
Expand Down
58 changes: 58 additions & 0 deletions ksql-common/src/main/java/io/confluent/ksql/query/QueryId.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/**
* Copyright 2017 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.query;


import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonSubTypes;

import java.util.Objects;

@JsonSubTypes({})
public class QueryId {
private final String id;

@JsonCreator
public QueryId(@JsonProperty("id") final String id) {
Objects.requireNonNull(id, "id can't be null");
this.id = id;
}

public String getId() {
return id;
}

public String toString() {
return id;
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (getClass() != o.getClass()) {
return false;
}
QueryId queryId1 = (QueryId) o;
return Objects.equals(id, queryId1.id);
}

@Override
public int hashCode() {
return Objects.hash(id);
}
}
60 changes: 39 additions & 21 deletions ksql-engine/src/main/java/io/confluent/ksql/KsqlEngine.java
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import io.confluent.ksql.parser.tree.Statement;
import io.confluent.ksql.parser.tree.Table;
import io.confluent.ksql.planner.plan.PlanNode;
import io.confluent.ksql.query.QueryId;
import io.confluent.ksql.serde.DataSource;
import io.confluent.ksql.util.DataSourceExtractor;
import io.confluent.ksql.util.KafkaTopicClient;
Expand Down Expand Up @@ -75,7 +76,7 @@
import java.util.Optional;
import java.util.Set;

public class KsqlEngine implements Closeable {
public class KsqlEngine implements Closeable, QueryTeminator {

private static final Logger log = LoggerFactory.getLogger(KsqlEngine.class);

Expand All @@ -91,7 +92,7 @@ public class KsqlEngine implements Closeable {
private final DDLCommandExec ddlCommandExec;
private final QueryEngine queryEngine;

private final Map<Long, PersistentQueryMetadata> persistentQueries;
private final Map<QueryId, PersistentQueryMetadata> persistentQueries;
private final Set<QueryMetadata> liveQueries;

public final FunctionRegistry functionRegistry;
Expand All @@ -104,8 +105,7 @@ public KsqlEngine(final KsqlConfig ksqlConfig, final KafkaTopicClient topicClien
this.metaStore = new MetaStoreImpl();
this.topicClient = topicClient;
this.ddlCommandExec = new DDLCommandExec(metaStore);
this.queryEngine = new QueryEngine(this, new CommandFactories(topicClient));

this.queryEngine = new QueryEngine(this, new CommandFactories(topicClient, this));
this.persistentQueries = new HashMap<>();
this.liveQueries = new HashSet<>();
this.functionRegistry = new FunctionRegistry();
Expand All @@ -120,10 +120,9 @@ public KsqlEngine(final KsqlConfig ksqlConfig, final KafkaTopicClient topicClien
* @throws Exception Any exception thrown here!
*/
public List<QueryMetadata> buildMultipleQueries(
final boolean createNewAppId,
final String queriesString,
final Map<String, Object> overriddenProperties
) throws Exception {
final boolean createNewAppId,
final String queriesString,
final Map<String, Object> overriddenProperties) throws Exception {
for (String property : overriddenProperties.keySet()) {
if (IMMUTABLE_PROPERTIES.contains(property)) {
throw new IllegalArgumentException(
Expand Down Expand Up @@ -174,15 +173,16 @@ public List<QueryMetadata> planQueries(final boolean createNewAppId,
public QueryMetadata getQueryExecutionPlan(final Query query) throws Exception {

// Logical plan creation from the ASTs
List<Pair<String, PlanNode>> logicalPlans = queryEngine.buildLogicalPlans(metaStore, Arrays.asList(new Pair<>("", query)));
List<Pair<String, PlanNode>> logicalPlans = queryEngine.buildLogicalPlans(metaStore,
Collections.singletonList(new Pair<>("", query)));

// Physical plan creation from logical plans.
List<QueryMetadata> runningQueries = queryEngine.buildPhysicalPlans(
false,
logicalPlans,
Arrays.asList(new Pair<>("", query)),
Collections.emptyMap(),
false
false,
logicalPlans,
Collections.singletonList(new Pair<>("", query)),
Collections.emptyMap(),
false
);
return runningQueries.get(0);
}
Expand Down Expand Up @@ -231,7 +231,7 @@ private Pair<String, Statement> buildSingleQueryAst(final Statement statement,
log.info("Building AST for {}.", statementString);

if (statement instanceof Query) {
return new Pair<>(statementString, (Query) statement);
return new Pair<>(statementString, statement);
} else if (statement instanceof CreateStreamAsSelect) {
CreateStreamAsSelect createStreamAsSelect = (CreateStreamAsSelect) statement;
QuerySpecification querySpecification = (QuerySpecification) createStreamAsSelect.getQuery().getQueryBody();
Expand Down Expand Up @@ -298,13 +298,13 @@ private Pair<String, Statement> buildSingleQueryAst(final Statement statement,
tempMetaStore);
return new Pair<>(statementString, statement);
} else if (statement instanceof DropStream) {
ddlCommandExec.tryExecute(new DropSourceCommand((DropStream) statement, DataSource.DataSourceType.KSTREAM), tempMetaStore);
ddlCommandExec.tryExecute(new DropSourceCommand((DropStream) statement, DataSource.DataSourceType.KSTREAM),
ddlCommandExec.tryExecute(new DropSourceCommand((DropStream) statement, DataSource.DataSourceType.KSTREAM, this), tempMetaStore);
ddlCommandExec.tryExecute(new DropSourceCommand((DropStream) statement, DataSource.DataSourceType.KSTREAM, this),
tempMetaStoreForParser);
return new Pair<>(statementString, statement);
} else if (statement instanceof DropTable) {
ddlCommandExec.tryExecute(new DropSourceCommand((DropTable) statement, DataSource.DataSourceType.KTABLE), tempMetaStore);
ddlCommandExec.tryExecute(new DropSourceCommand((DropTable) statement, DataSource.DataSourceType.KTABLE),
ddlCommandExec.tryExecute(new DropSourceCommand((DropTable) statement, DataSource.DataSourceType.KTABLE, this), tempMetaStore);
ddlCommandExec.tryExecute(new DropSourceCommand((DropTable) statement, DataSource.DataSourceType.KTABLE, this),
tempMetaStoreForParser);
return new Pair<>(statementString, statement);
} else if (statement instanceof DropTopic) {
Expand Down Expand Up @@ -380,7 +380,8 @@ public DDLCommandExec getDDLCommandExec() {
return ddlCommandExec;
}

public boolean terminateQuery(final long queryId, final boolean closeStreams) {
@Override
public boolean terminateQuery(final QueryId queryId, final boolean closeStreams) {
QueryMetadata queryMetadata = persistentQueries.remove(queryId);
if (queryMetadata == null) {
return false;
Expand All @@ -392,7 +393,23 @@ public boolean terminateQuery(final long queryId, final boolean closeStreams) {
return true;
}

public Map<Long, PersistentQueryMetadata> getPersistentQueries() {
@Override
public void terminateQueryForEntity(final String entity) {
final Optional<PersistentQueryMetadata> query = persistentQueries.values()
.stream()
.filter(persistentQueryMetadata -> persistentQueryMetadata.getEntity().equalsIgnoreCase(entity))
.findFirst();

if (query.isPresent()) {
final PersistentQueryMetadata metadata = query.get();
log.info("Terminating persistent query {}", metadata.getId());
metadata.close();
persistentQueries.remove(metadata.getId());
liveQueries.remove(metadata);
}
}

public Map<QueryId, PersistentQueryMetadata> getPersistentQueries() {
return new HashMap<>(persistentQueries);
}

Expand Down Expand Up @@ -420,6 +437,7 @@ public void close() throws IOException {
}


@Override
public boolean terminateAllQueries() {
try {
for (QueryMetadata queryMetadata : liveQueries) {
Expand Down
10 changes: 3 additions & 7 deletions ksql-engine/src/main/java/io/confluent/ksql/QueryEngine.java
Original file line number Diff line number Diff line change
Expand Up @@ -50,19 +50,16 @@
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;

class QueryEngine {

private static final Logger log = LoggerFactory.getLogger(QueryEngine.class);
private final AtomicLong queryIdCounter;
private final KsqlEngine ksqlEngine;
private final DDLCommandFactory ddlCommandFactory;


QueryEngine(final KsqlEngine ksqlEngine, final DDLCommandFactory ddlCommandFactory) {
this.ddlCommandFactory = ddlCommandFactory;
this.queryIdCounter = new AtomicLong(1);
this.ksqlEngine = ksqlEngine;
}

Expand Down Expand Up @@ -120,8 +117,7 @@ List<QueryMetadata> buildPhysicalPlans(
final List<Pair<String, PlanNode>> logicalPlans,
final List<Pair<String, Statement>> statementList,
final Map<String, Object> overriddenStreamsProperties,
final boolean updateMetastore
) throws Exception {
final boolean updateMetastore) throws Exception {

List<QueryMetadata> physicalPlans = new ArrayList<>();

Expand Down Expand Up @@ -161,8 +157,8 @@ private void buildQueryPhysicalPlan(final List<QueryMetadata> physicalPlans,
addUniqueTimeSuffix,
overriddenStreamsProperties,
updateMetastore,
ksqlEngine.getMetaStore(),
queryIdCounter);
ksqlEngine.getMetaStore()
);

physicalPlans.add(physicalPlanBuilder.buildPhysicalPlan(statementPlanPair));
}
Expand Down
26 changes: 26 additions & 0 deletions ksql-engine/src/main/java/io/confluent/ksql/QueryTeminator.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/**
* Copyright 2017 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;

import io.confluent.ksql.query.QueryId;

public interface QueryTeminator {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

prob want to add in the missing 'r'

boolean terminateQuery(QueryId queryId, boolean closeStreams);

void terminateQueryForEntity(String entity);

boolean terminateAllQueries();
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import java.util.HashMap;
import java.util.Map;

import io.confluent.ksql.QueryTeminator;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same here for missing r

import io.confluent.ksql.parser.tree.CreateStream;
import io.confluent.ksql.parser.tree.CreateTable;
import io.confluent.ksql.parser.tree.DDLStatement;
Expand All @@ -33,14 +34,14 @@
public class CommandFactories implements DDLCommandFactory {
private final Map<Class<? extends DDLStatement>, DDLCommandFactory> factories = new HashMap<>();

public CommandFactories(final KafkaTopicClient topicClient) {
public CommandFactories(final KafkaTopicClient topicClient, final QueryTeminator queryTeminator) {
factories.put(RegisterTopic.class, (ddlStatement, properties) -> new RegisterTopicCommand((RegisterTopic)ddlStatement, properties));
factories.put(CreateStream.class, (ddlStatement, properties) -> new CreateStreamCommand((CreateStream) ddlStatement, properties, topicClient));
factories.put(CreateTable.class, (ddlStatement, properties) -> new CreateTableCommand((CreateTable)ddlStatement, properties, topicClient));
factories.put(DropStream.class, (ddlStatement, properties) -> new DropSourceCommand(
(DropStream) ddlStatement, DataSource.DataSourceType.KSTREAM));
(DropStream) ddlStatement, DataSource.DataSourceType.KSTREAM, queryTeminator));
factories.put(DropTable.class, (ddlStatement, properties) -> new DropSourceCommand(
(DropTable) ddlStatement, DataSource.DataSourceType.KTABLE));
(DropTable) ddlStatement, DataSource.DataSourceType.KTABLE, queryTeminator));
factories.put(DropTopic.class, (ddlStatement, properties) -> new DropTopicCommand(((DropTopic) ddlStatement)));
factories.put(SetProperty.class, (ddlStatement, properties) -> new SetPropertyCommand((SetProperty) ddlStatement, properties));
}
Expand Down
Loading