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

feat: Adds rate limiting to pull queries #4951

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
Original file line number Diff line number Diff line change
Expand Up @@ -203,6 +203,11 @@ public class KsqlConfig extends AbstractConfig {
public static final String KSQL_QUERY_PULL_METRICS_ENABLED_DOC =
"Config to enable/disable collecting JMX metrics for pull queries.";

public static final String KSQL_QUERY_PULL_MAX_QPS_CONFIG = "ksql.query.pull.max.qps";
public static final Integer KSQL_QUERY_PULL_MAX_QPS_DEFAULT = Integer.MAX_VALUE;
public static final String KSQL_QUERY_PULL_MAX_QPS_DOC = "The maximum qps allowed for pull "
+ "queries. Once the limit is hit, queries will fail immediately";

public static final Collection<CompatibilityBreakingConfigDef> COMPATIBLY_BREAKING_CONFIG_DEFS
= ImmutableList.of();

Expand Down Expand Up @@ -632,6 +637,13 @@ private static ConfigDef buildConfigDef(final ConfigGeneration generation) {
Importance.LOW,
KSQL_QUERY_PULL_METRICS_ENABLED_DOC
)
.define(
KSQL_QUERY_PULL_MAX_QPS_CONFIG,
Type.INT,
KSQL_QUERY_PULL_MAX_QPS_DEFAULT,
Importance.LOW,
KSQL_QUERY_PULL_MAX_QPS_DOC
)
.withClientSslSupport();

for (final CompatibilityBreakingConfigDef compatibilityBreakingConfigDef
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

package io.confluent.ksql.rest.server.execution;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.BoundType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
Expand All @@ -23,6 +24,7 @@
import com.google.common.collect.Range;
import com.google.common.collect.Sets;
import com.google.common.collect.Sets.SetView;
import com.google.common.util.concurrent.RateLimiter;
import io.confluent.ksql.GenericRow;
import io.confluent.ksql.KsqlExecutionContext;
import io.confluent.ksql.analyzer.ImmutableAnalysis;
Expand Down Expand Up @@ -129,6 +131,7 @@ public final class PullQueryExecutor {

private final KsqlExecutionContext executionContext;
private final RoutingFilterFactory routingFilterFactory;
private final RateLimiter rateLimiter;

public PullQueryExecutor(
final KsqlExecutionContext executionContext,
Expand All @@ -138,6 +141,8 @@ public PullQueryExecutor(
this.executionContext = Objects.requireNonNull(executionContext, "executionContext");
this.routingFilterFactory =
Objects.requireNonNull(routingFilterFactory, "routingFilterFactory");
this.rateLimiter = RateLimiter.create(ksqlConfig.getInt(
KsqlConfig.KSQL_QUERY_PULL_MAX_QPS_CONFIG));
}

@SuppressWarnings("unused") // Needs to match validator API.
Expand Down Expand Up @@ -169,6 +174,15 @@ public TableRowsEntity execute(
}

try {
final RoutingOptions routingOptions = new ConfigRoutingOptions(
statement.getConfig(), statement.getConfigOverrides(), statement.getRequestProperties());
final boolean isAlreadyForwarded = routingOptions.skipForwardRequest();

// Only check the rate limit at the forwarding host
if (!isAlreadyForwarded) {
checkRateLimit();
}

final ImmutableAnalysis analysis = new RewrittenAnalysis(
analyze(statement, executionContext),
new ColumnReferenceRewriter()::process
Expand Down Expand Up @@ -201,7 +215,8 @@ public TableRowsEntity execute(
statement,
executionContext,
serviceContext,
pullQueryContext
pullQueryContext,
routingOptions
);
} catch (final Exception e) {
pullQueryMetrics.ifPresent(metrics -> metrics.recordErrorRate(1));
Expand All @@ -213,15 +228,21 @@ public TableRowsEntity execute(
}
}

@VisibleForTesting
void checkRateLimit() {
if (!rateLimiter.tryAcquire()) {
throw new KsqlException("Host is at rate limit for pull queries. Currently set to "
+ rateLimiter.getRate() + " qps.");
}
}

private TableRowsEntity handlePullQuery(
final ConfiguredStatement<Query> statement,
final KsqlExecutionContext executionContext,
final ServiceContext serviceContext,
final PullQueryContext pullQueryContext
final PullQueryContext pullQueryContext,
final RoutingOptions routingOptions
) {
final RoutingOptions routingOptions = new ConfigRoutingOptions(
statement.getConfig(), statement.getConfigOverrides(), statement.getRequestProperties());

// Get active and standby nodes for this key
final Locator locator = pullQueryContext.mat.locator();
final List<KsqlNode> filteredAndOrderedNodes = locator.locate(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import static io.confluent.ksql.rest.entity.KsqlStatementErrorMessageMatchers.statement;
import static io.confluent.ksql.rest.server.resources.KsqlRestExceptionMatchers.exceptionStatementErrorMessage;
import static io.confluent.ksql.rest.server.resources.KsqlRestExceptionMatchers.exceptionStatusCode;
import static junit.framework.TestCase.fail;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.is;
import static org.mockito.Mockito.mock;
Expand All @@ -39,6 +40,7 @@
import io.confluent.ksql.util.KsqlException;
import java.util.Optional;
import org.eclipse.jetty.http.HttpStatus.Code;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.runners.Enclosed;
Expand Down Expand Up @@ -118,4 +120,22 @@ public void shouldRedirectQueriesToQueryEndPoint() {
);
}
}

@RunWith(MockitoJUnitRunner.class)
public static class RateLimit {

@Rule
public final TemporaryEngine engine = new TemporaryEngine()
.withConfigs(ImmutableMap.of(KsqlConfig.KSQL_QUERY_PULL_MAX_QPS_CONFIG, 2));

@Test
public void shouldRateLimit() {
Copy link
Member

Choose a reason for hiding this comment

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

How does this fail here? There are no queries issued?

Copy link
Member Author

@AlanConfluent AlanConfluent Apr 1, 2020

Choose a reason for hiding this comment

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

It checks the limit before the request is issued. It effectively asks for permission, and if it's at the limit, it's not given permission.

PullQueryExecutor pullQueryExecutor = new PullQueryExecutor(
engine.getEngine(), ROUTING_FILTER_FACTORY, engine.getKsqlConfig());

// When:
pullQueryExecutor.checkRateLimit();
Assert.assertThrows(KsqlException.class, pullQueryExecutor::checkRateLimit);
}
}
}