Skip to content

Commit

Permalink
feat: Support pausing/resuming persistent queries
Browse files Browse the repository at this point in the history
Addresses: #6403
Implements KLIP-63: PAUSE and RESUME for persistent queries.
  • Loading branch information
jnh5y committed Jun 15, 2022
1 parent e9ecd72 commit f79cb2e
Show file tree
Hide file tree
Showing 43 changed files with 1,263 additions and 25 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,10 @@
import io.confluent.ksql.cli.console.table.builder.FunctionNameListTableBuilder;
import io.confluent.ksql.cli.console.table.builder.KafkaTopicsListTableBuilder;
import io.confluent.ksql.cli.console.table.builder.ListVariablesTableBuilder;
import io.confluent.ksql.cli.console.table.builder.PauseQueryTableBuilder;
import io.confluent.ksql.cli.console.table.builder.PropertiesListTableBuilder;
import io.confluent.ksql.cli.console.table.builder.QueriesTableBuilder;
import io.confluent.ksql.cli.console.table.builder.ResumeQueryTableBuilder;
import io.confluent.ksql.cli.console.table.builder.StreamsListTableBuilder;
import io.confluent.ksql.cli.console.table.builder.TableBuilder;
import io.confluent.ksql.cli.console.table.builder.TablesListTableBuilder;
Expand Down Expand Up @@ -71,6 +73,7 @@
import io.confluent.ksql.rest.entity.KsqlErrorMessage;
import io.confluent.ksql.rest.entity.KsqlStatementErrorMessage;
import io.confluent.ksql.rest.entity.KsqlWarning;
import io.confluent.ksql.rest.entity.PauseQueryEntity;
import io.confluent.ksql.rest.entity.PropertiesList;
import io.confluent.ksql.rest.entity.Queries;
import io.confluent.ksql.rest.entity.QueryDescription;
Expand All @@ -79,6 +82,7 @@
import io.confluent.ksql.rest.entity.QueryHostStat;
import io.confluent.ksql.rest.entity.QueryOffsetSummary;
import io.confluent.ksql.rest.entity.QueryTopicOffsetSummary;
import io.confluent.ksql.rest.entity.ResumeQueryEntity;
import io.confluent.ksql.rest.entity.RunningQuery;
import io.confluent.ksql.rest.entity.SourceDescription;
import io.confluent.ksql.rest.entity.SourceDescriptionEntity;
Expand Down Expand Up @@ -191,6 +195,10 @@ public class Console implements Closeable {
tablePrinter(WarningEntity.class, WarningEntityTableBuilder::new))
.put(VariablesList.class,
tablePrinter(VariablesList.class, ListVariablesTableBuilder::new))
.put(PauseQueryEntity.class,
tablePrinter(PauseQueryEntity.class, PauseQueryTableBuilder::new))
.put(ResumeQueryEntity.class,
tablePrinter(ResumeQueryEntity.class, ResumeQueryTableBuilder::new))
.put(TerminateQueryEntity.class,
tablePrinter(TerminateQueryEntity.class, TerminateQueryTableBuilder::new))
.put(AssertTopicEntity.class, Console::printAssertTopic)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Copyright 2022 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.cli.console.table.builder;

import io.confluent.ksql.cli.console.table.Table;
import io.confluent.ksql.rest.entity.PauseQueryEntity;

/**
* This builder is only used to pause persistent queries.
*/
public class PauseQueryTableBuilder implements TableBuilder<PauseQueryEntity> {

@Override
public Table buildTable(final PauseQueryEntity entity) {
return new Table.Builder()
.withColumnHeaders("Message")
.withRow("Paused query \"" + entity.getQueryId() + '"')
.build();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Copyright 2022 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.cli.console.table.builder;

import io.confluent.ksql.cli.console.table.Table;
import io.confluent.ksql.rest.entity.ResumeQueryEntity;

/**
* This builder is only used to resume persistent queries.
*/
public class ResumeQueryTableBuilder implements TableBuilder<ResumeQueryEntity> {

@Override
public Table buildTable(final ResumeQueryEntity entity) {
return new Table.Builder()
.withColumnHeaders("Message")
.withRow("Resumed query \"" + entity.getQueryId() + '"')
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ public enum KsqlQueryStatus {
RUNNING,
ERROR,
UNRESPONSIVE,
PAUSED,
}

public static KsqlQueryStatus fromStreamsState(final KafkaStreams.State state) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.function.BiFunction;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -183,6 +184,10 @@ public List<PersistentQueryMetadata> getPersistentQueries() {
return ksqlEngine.getPersistentQueries();
}

public Optional<PersistentQueryMetadata> getPersistentQuery(final QueryId queryId) {
return ksqlEngine.getPersistentQuery(queryId);
}

public void close() {
ksqlEngine.close();
serviceContext.close();
Expand All @@ -196,6 +201,16 @@ public void terminateQuery(final QueryId queryId) {
});
}

@VisibleForTesting
public void pauseQuery(final QueryId queryId) {
ksqlEngine.getPersistentQuery(queryId).ifPresent(QueryMetadata::pause);
}

@VisibleForTesting
public void resumeQuery(final QueryId queryId) {
ksqlEngine.getPersistentQuery(queryId).ifPresent(QueryMetadata::resume);
}

private static ExecuteResult execute(
final KsqlExecutionContext executionContext,
final ParsedStatement stmt,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,13 +40,15 @@
import io.confluent.ksql.schema.ksql.PhysicalSchema;
import io.confluent.ksql.schema.query.QuerySchemas;
import io.confluent.ksql.serde.KeyFormat;
import io.confluent.ksql.util.KsqlConstants.KsqlQueryStatus;
import io.confluent.ksql.util.QueryMetadataImpl.TimeBoundedQueue;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Function;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KafkaStreams.State;
Expand All @@ -62,6 +64,7 @@ public class BinPackedPersistentQueryMetadataImpl implements PersistentQueryMeta
private static final Logger LOG = LoggerFactory
.getLogger(BinPackedPersistentQueryMetadataImpl.class);

private final AtomicBoolean isPaused = new AtomicBoolean(false);
private final KsqlConstants.PersistentQueryType persistentQueryType;
private final String statementString;
private final String executionPlan;
Expand Down Expand Up @@ -416,6 +419,27 @@ public void start() {
everStarted = true;
}

@Override
public KsqlQueryStatus getQueryStatus() {
if (isPaused.get()) {
return KsqlQueryStatus.PAUSED;
} else {
return KsqlConstants.fromStreamsState(getState());
}
}

@Override
public void pause() {
sharedKafkaStreamsRuntime.getKafkaStreams().pauseNamedTopology(topology.name());
isPaused.set(true);
}

@Override
public void resume() {
sharedKafkaStreamsRuntime.getKafkaStreams().resumeNamedTopology(topology.name());
isPaused.set(false);
}

@Override
public void register() {
sharedKafkaStreamsRuntime.register(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import io.confluent.ksql.query.QueryId;
import io.confluent.ksql.rest.entity.StreamsTaskMetadata;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.util.KsqlConstants.KsqlQueryStatus;
import java.util.Collection;
import java.util.List;
import java.util.Map;
Expand All @@ -43,6 +44,8 @@ public interface QueryMetadata {

KafkaStreams.State getState();

KsqlQueryStatus getQueryStatus();

String getExecutionPlan();

String getQueryApplicationId();
Expand Down Expand Up @@ -73,6 +76,10 @@ public interface QueryMetadata {

KafkaStreams getKafkaStreams();

void pause();

void resume();

void close();

void start();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import io.confluent.ksql.query.QueryId;
import io.confluent.ksql.rest.entity.StreamsTaskMetadata;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.util.KsqlConstants.KsqlQueryStatus;
import io.confluent.ksql.util.KsqlConstants.KsqlQueryType;
import java.time.Duration;
import java.util.Collection;
Expand All @@ -44,6 +45,7 @@
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KafkaStreams.State;
Expand All @@ -57,7 +59,7 @@

public class QueryMetadataImpl implements QueryMetadata {
private static final Logger LOG = LoggerFactory.getLogger(QueryMetadataImpl.class);

private final AtomicBoolean isPaused = new AtomicBoolean(false);
private final String statementString;
private final String executionPlan;
private final String queryApplicationId;
Expand Down Expand Up @@ -421,6 +423,27 @@ public void start() {
kafkaStreams.start();
}

@Override
public KsqlQueryStatus getQueryStatus() {
if (isPaused.get()) {
return KsqlQueryStatus.PAUSED;
} else {
return KsqlConstants.fromStreamsState(getState());
}
}

@Override
public void pause() {
kafkaStreams.pause();
isPaused.set(true);
}

@Override
public void resume() {
kafkaStreams.resume();
isPaused.set(false);
}

public static class RetryEvent implements QueryMetadata.RetryEvent {
private final Ticker ticker;
private final QueryId queryId;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,16 @@ private SandboxedBinPackedPersistentQueryMetadataImpl(
super(queryMetadata, listener);
}

@Override
public void pause() {
// no-op
}

@Override
public void resume() {
// no-op
}

@Override
public void stop() {
//no-op
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,16 @@ private SandboxedPersistentQueryMetadataImpl(
super(queryMetadata, listener);
}

@Override
public void pause() {
// no-op
}

@Override
public void resume() {
// no-op
}

@Override
public void close() {
getListener().onClose(this);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,16 @@ public synchronized void close() {
kafkaStreams.cleanUp();
}

@Override
public void pause(final QueryId queryId) {
//no-op
}

@Override
public void resume(final QueryId queryId) {
//no-op
}

@Override
public void start(final QueryId queryId) {
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,10 @@ public boolean isError(final QueryId queryId) {

public abstract void close();

public abstract void pause(QueryId queryId);

public abstract void resume(QueryId queryId);

public abstract void stop(QueryId queryId, boolean isCreateOrReplace);

public abstract void start(QueryId queryId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -227,6 +227,14 @@ public synchronized void close() {
kafkaStreams.cleanUp();
}

@Override
public void pause(final QueryId queryId) {
}

@Override
public void resume(final QueryId queryId) {
}

@Override
public void start(final QueryId queryId) {
log.info("Attempting to start query {} in runtime {}", queryId, getApplicationId());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,7 @@ public void setCompletionHandler(final CompletionHandler completionHandler) {
rowQueue.setCompletionHandler(completionHandler);
}


@Override
public void close() {
// Push queries can be closed by both terminate commands and the client ending the request, so
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2266,6 +2266,21 @@ public void shouldBeAbleToPrepareTerminateAndDrop() {
// Then: did not throw.
}

@Test
public void shouldBeAbleToPreparePauseAndResume() {
// Given:
givenSqlAlreadyExecuted("CREATE STREAM FOO AS SELECT * FROM TEST1;");

final List<ParsedStatement> parsed = ksqlEngine.parse(
"PAUSE CSAS_FOO_0;"
+ "RESUME CSAS_FOO_0;");

// When:
parsed.forEach(ksqlEngine::prepare);

// Then: did not throw.
}

@Test
public void shouldIgnoreLegacyDeleteTopicPartOfDropCommand() {
// Given:
Expand Down
Loading

0 comments on commit f79cb2e

Please sign in to comment.