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: Add metrics for stream pull queries to WS #8174

Merged
merged 7 commits into from
Sep 29, 2021
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 @@ -32,7 +32,7 @@ public void setCallback(final MetricsCallback callback) {
this.callbackRef.set(callback);
}

void reportMetrics(final int statusCode, final long requestBytes, final long responseBytes,
public void reportMetrics(final int statusCode, final long requestBytes, final long responseBytes,
final long startTimeNanos) {
final MetricsCallback callback = callbackRef.get();
if (callback != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,11 @@
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
import com.google.common.util.concurrent.RateLimiter;
import io.confluent.ksql.analyzer.ImmutableAnalysis;
import io.confluent.ksql.api.server.MetricsCallbackHolder;
import io.confluent.ksql.api.server.SlidingWindowRateLimiter;
import io.confluent.ksql.config.SessionConfig;
import io.confluent.ksql.engine.KsqlEngine;
import io.confluent.ksql.engine.PullQueryExecutionUtil;
import io.confluent.ksql.execution.streams.RoutingFilter.RoutingFilterFactory;
import io.confluent.ksql.internal.PullQueryExecutorMetrics;
import io.confluent.ksql.metastore.model.DataSource;
Expand All @@ -33,8 +35,12 @@
import io.confluent.ksql.parser.tree.Query;
import io.confluent.ksql.parser.tree.Statement;
import io.confluent.ksql.physical.pull.HARouting;
import io.confluent.ksql.physical.pull.PullPhysicalPlan.PullPhysicalPlanType;
import io.confluent.ksql.physical.pull.PullPhysicalPlan.PullSourceType;
import io.confluent.ksql.physical.pull.PullPhysicalPlan.RoutingNodeType;
import io.confluent.ksql.physical.scalablepush.PushRouting;
import io.confluent.ksql.properties.DenyListPropertyValidator;
import io.confluent.ksql.query.TransientQueryQueue;
import io.confluent.ksql.rest.ApiJsonMapper;
import io.confluent.ksql.rest.Errors;
import io.confluent.ksql.rest.entity.KsqlMediaType;
Expand All @@ -46,11 +52,13 @@
import io.confluent.ksql.rest.server.resources.streaming.PushQueryPublisher.PushQuerySubscription;
import io.confluent.ksql.rest.util.CommandStoreUtil;
import io.confluent.ksql.rest.util.ConcurrencyLimiter;
import io.confluent.ksql.rest.util.ConcurrencyLimiter.Decrementer;
import io.confluent.ksql.rest.util.ScalablePushUtil;
import io.confluent.ksql.security.KsqlAuthorizationValidator;
import io.confluent.ksql.security.KsqlSecurityContext;
import io.confluent.ksql.statement.ConfiguredStatement;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.KsqlConstants.KsqlQueryType;
import io.confluent.ksql.util.KsqlStatementException;
import io.confluent.ksql.util.StreamPullQueryMetadata;
import io.confluent.ksql.version.metrics.ActivenessRegistrar;
Expand All @@ -62,8 +70,11 @@
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KafkaStreams.State;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -298,6 +309,17 @@ private void handleQuery(final RequestContext info, final Query query,
return;
}
case KSTREAM: {
final MetricsCallbackHolder metricsCallbackHolder = new MetricsCallbackHolder();
final AtomicReference<StreamPullQueryMetadata> resultForMetrics =
new AtomicReference<>(null);
final AtomicReference<Decrementer> refDecrementer = new AtomicReference<>(null);
initializeStreamMetrics(metricsCallbackHolder, resultForMetrics, refDecrementer,
analysis, startTimeNanos);

PullQueryExecutionUtil.checkRateLimit(rateLimiter);
pullBandRateLimiter.allow(KsqlQueryType.PULL);
refDecrementer.set(pullConcurrencyLimiter.increment());

final StreamPullQueryMetadata queryMetadata =
ksqlEngine.createStreamPullQuery(
info.securityContext.getServiceContext(),
Expand All @@ -306,6 +328,7 @@ private void handleQuery(final RequestContext info, final Query query,
true
);

resultForMetrics.set(queryMetadata);
localCommands.ifPresent(lc -> lc.write(queryMetadata.getTransientQueryMetadata()));

final PushQuerySubscription subscription =
Expand All @@ -320,7 +343,7 @@ private void handleQuery(final RequestContext info, final Query query,
);
queryMetadata.getTransientQueryMetadata().start();

streamSubscriber.onSubscribe(subscription);
streamSubscriber.onSubscribe(subscription, metricsCallbackHolder, startTimeNanos);
return;
}
default:
Expand Down Expand Up @@ -356,6 +379,62 @@ private void handleQuery(final RequestContext info, final Query query,
}
}

private void initializeStreamMetrics(
final MetricsCallbackHolder metricsCallbackHolder,
final AtomicReference<StreamPullQueryMetadata> resultForMetrics,
final AtomicReference<Decrementer> refDecrementer,
final ImmutableAnalysis analysis,
final long startTimeNanos) {

metricsCallbackHolder.setCallback(
(statusCode, requestBytes, responseBytes, startTimeNs) ->
pullQueryMetrics.ifPresent(metrics -> {

final StreamPullQueryMetadata m = resultForMetrics.get();
final KafkaStreams.State state = m == null ? null : m.getTransientQueryMetadata()
.getKafkaStreams().state();

if (m == null || state == null
|| state.equals(State.ERROR)
|| state.equals(State.PENDING_ERROR)) {
metrics.recordLatencyForError(startTimeNs);
metrics.recordZeroRowsReturnedForError();
metrics.recordZeroRowsProcessedForError();
} else {
final boolean isWindowed = analysis
.getFrom()
.getDataSource()
.getKsqlTopic()
.getKeyFormat().isWindowed();
final PullSourceType sourceType = isWindowed
? PullSourceType.WINDOWED_STREAM : PullSourceType.NON_WINDOWED_STREAM;
// There is no WHERE clause constraint information in the persistent logical plan
final PullPhysicalPlanType planType = PullPhysicalPlanType.UNKNOWN;
final RoutingNodeType routingNodeType = RoutingNodeType.SOURCE_NODE;
metrics.recordLatency(
startTimeNanos,
sourceType,
planType,
routingNodeType
);
final TransientQueryQueue rowQueue = (TransientQueryQueue)
m.getTransientQueryMetadata().getRowQueue();
// The rows read from the underlying data source equal the rows read by the user
// since the WHERE condition is pushed to the data source
metrics.recordRowsReturned(rowQueue.getTotalRowsQueued(), sourceType, planType,
routingNodeType);
metrics.recordRowsProcessed(rowQueue.getTotalRowsQueued(), sourceType, planType,
routingNodeType);
}
// Decrement on happy or exception path
final Decrementer decrementer = refDecrementer.get();
if (decrementer != null) {
decrementer.decrementAtMostOnce();
}
})
);
}

private void handlePrintTopic(final RequestContext info, final PrintTopic printTopic) {
final String topicName = printTopic.getTopic();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,14 @@
import static io.netty.handler.codec.http.websocketx.WebSocketCloseStatus.PROTOCOL_ERROR;

import com.fasterxml.jackson.core.JsonProcessingException;
import io.confluent.ksql.api.server.MetricsCallbackHolder;
import io.confluent.ksql.rest.ApiJsonMapper;
import io.confluent.ksql.rest.util.EntityUtil;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.vertx.core.http.ServerWebSocket;
import java.io.IOException;
import java.util.Collection;
import java.util.Optional;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -37,6 +39,8 @@ class WebSocketSubscriber<T> implements Flow.Subscriber<Collection<T>>, AutoClos
private Flow.Subscription subscription;
private volatile boolean closed;
private volatile boolean drainHandlerSet;
private Optional<MetricsCallbackHolder> metricsCallbackHolderOptional = Optional.empty();
private long startTimeNanos;

WebSocketSubscriber(final ServerWebSocket websocket) {
this.websocket = websocket;
Expand All @@ -47,6 +51,16 @@ public void onSubscribe(final Flow.Subscription subscription) {
subscription.request(1);
}

public void onSubscribe(final Flow.Subscription subscription,
final MetricsCallbackHolder metricsCallbackHolder,
final long startTimeNanos) {
this.subscription = subscription;
subscription.request(1);
metricsCallbackHolderOptional = Optional.of(metricsCallbackHolder);
this.startTimeNanos = startTimeNanos;
}


@Override
public void onNext(final Collection<T> rows) {
for (final T row : rows) {
Expand Down Expand Up @@ -76,11 +90,14 @@ public void onError(final Throwable e) {
? "KSQL exception: " + e.getClass().getSimpleName()
: e.getMessage();

metricsCallbackHolderOptional.ifPresent(mc -> mc.reportMetrics(0, 0, 0, startTimeNanos));
SessionUtil.closeSilently(websocket, INTERNAL_SERVER_ERROR.code(), msg);
}

@Override
public void onComplete() {
// We don't have the status code , request and response size in bytes
metricsCallbackHolderOptional.ifPresent(mc -> mc.reportMetrics(0, 0, 0, startTimeNanos));
SessionUtil.closeSilently(websocket, NORMAL_CLOSURE.code(), "done");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,12 @@

import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.is;

import com.google.common.collect.ImmutableMap;
import io.confluent.ksql.api.utils.QueryResponse;
import io.confluent.ksql.engine.KsqlEngine;
import io.confluent.ksql.integration.IntegrationTestHarness;
import io.confluent.ksql.integration.Retry;
import io.confluent.ksql.name.ColumnName;
import io.confluent.ksql.rest.entity.StreamedRow;
import io.confluent.ksql.rest.server.TestKsqlRestApp;
Expand All @@ -35,21 +34,12 @@
import io.confluent.ksql.serde.SerdeFeatures;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.PageViewDataProvider;
import io.confluent.ksql.util.VertxCompletableFuture;
import io.vertx.core.Vertx;
import io.vertx.core.buffer.Buffer;
import io.vertx.core.http.HttpVersion;
import io.vertx.core.json.JsonObject;
import io.vertx.ext.web.client.HttpResponse;
import io.vertx.ext.web.client.WebClient;
import io.vertx.ext.web.client.WebClientOptions;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Metrics;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
Expand All @@ -62,9 +52,6 @@
@Ignore
public class PullQueryMetricsFunctionalTest {

private Vertx vertx;
private WebClient client;

private static final PageViewDataProvider PAGE_VIEWS_PROVIDER = new PageViewDataProvider();
private static final String PAGE_VIEW_TOPIC = PAGE_VIEWS_PROVIDER.topicName();
private static final String PAGE_VIEW_STREAM = PAGE_VIEWS_PROVIDER.sourceName();
Expand Down Expand Up @@ -175,14 +162,17 @@ public class PullQueryMetricsFunctionalTest {
TABLE_TAGS
);

private Metrics metrics;

@ClassRule
public static final RuleChain CHAIN = RuleChain.outerRule(TEST_HARNESS).around(REST_APP);

@Rule
public final Retry retry = Retry.of(5, AssertionError.class, 3, TimeUnit.SECONDS);

@Rule
public final Timeout timeout = Timeout.seconds(60);

private Metrics metrics;

@BeforeClass
public static void setUpClass() {
TEST_HARNESS.ensureTopics(PAGE_VIEW_TOPIC);
Expand Down Expand Up @@ -215,22 +205,6 @@ public static void setUpClass() {
@Before
public void setUp() {
metrics = ((KsqlEngine)REST_APP.getEngine()).getEngineMetrics().getMetrics();
vertx = Vertx.vertx();
client = createClient();
}

@After
public void tearDown() {
if (client != null) {
client.close();
}
if (vertx != null) {
vertx.close();
}
}

@AfterClass
public static void classTearDown() {
}

@Test
Expand Down Expand Up @@ -277,79 +251,4 @@ public void shouldVerifyMetrics() {
assertThat(totalRequestsStreamMetric.metricValue(), is(1.0));
assertThat((Double)requestDistributionStreamMetric.metricValue(), greaterThan(1.0));
}

@Test
public void shouldVerifyMetricsHttp2() {
// Given:
final KafkaMetric recordsReturnedTableMetric = metrics.metric(recordsReturnedTable);
final KafkaMetric latencyTableMetric = metrics.metric(latencyTable);
final KafkaMetric responseSizeTableMetric = metrics.metric(responseSizeTable);
final KafkaMetric totalRequestsTableMetric = metrics.metric(totalRequestsTable);
final KafkaMetric requestDistributionTableMetric = metrics.metric(requestDistributionTable);

final KafkaMetric recordsReturnedStreamMetric = metrics.metric(recordsReturnedStream);
final KafkaMetric latencyStreamMetric = metrics.metric(latencyStream);
final KafkaMetric responseSizeStreamMetric = metrics.metric(responseSizeStream);
final KafkaMetric totalRequestsStreamMetric = metrics.metric(totalRequestsStream);
final KafkaMetric requestDistributionStreamMetric = metrics.metric(requestDistributionStream);

// When:
final String sqlTable = "SELECT COUNT, USERID from " + AGG_TABLE + " WHERE USERID='" + AN_AGG_KEY + "';";
QueryResponse queryResponse1 = executeQuery(sqlTable);
assertThat(queryResponse1.rows, hasSize(1));

final String sqlStream = "SELECT * from " + PAGE_VIEW_STREAM + " WHERE PAGEID='" + A_STREAM_KEY + "';";
QueryResponse queryResponse2 = executeQuery(sqlStream);
assertThat(queryResponse2.rows, hasSize(1));

// Then:
assertThat(recordsReturnedTableMetric.metricValue(), is(2.0));
assertThat((Double)latencyTableMetric.metricValue(), greaterThan(1.0));
assertThat((Double)responseSizeTableMetric.metricValue(), greaterThan(1.0));
assertThat(totalRequestsTableMetric.metricValue(), is(2.0));
assertThat((Double)requestDistributionTableMetric.metricValue(), greaterThan(1.0));

assertThat(recordsReturnedStreamMetric.metricValue(), is(2.0));
assertThat((Double)latencyStreamMetric.metricValue(), greaterThan(1.0));
assertThat((Double)responseSizeStreamMetric.metricValue(), greaterThan(1.0));
assertThat(totalRequestsStreamMetric.metricValue(), is(2.0));
assertThat((Double)requestDistributionStreamMetric.metricValue(), greaterThan(1.0));
}

private QueryResponse executeQuery(final String sql) {
return executeQueryWithVariables(sql, new JsonObject());
}

private QueryResponse executeQueryWithVariables(final String sql, final JsonObject variables) {
JsonObject properties = new JsonObject();
JsonObject requestBody = new JsonObject()
.put("sql", sql).put("properties", properties).put("sessionVariables", variables);
HttpResponse<Buffer> response = sendRequest("/query-stream", requestBody.toBuffer());
return new QueryResponse(response.bodyAsString());
}

private WebClient createClient() {
WebClientOptions options = new WebClientOptions().
setProtocolVersion(HttpVersion.HTTP_2).setHttp2ClearTextUpgrade(false)
.setDefaultHost("localhost").setDefaultPort(REST_APP.getListeners().get(0).getPort());
return WebClient.create(vertx, options);
}

private HttpResponse<Buffer> sendRequest(final String uri, final Buffer requestBody) {
return sendRequest(client, uri, requestBody);
}

private HttpResponse<Buffer> sendRequest(final WebClient client, final String uri,
final Buffer requestBody) {
VertxCompletableFuture<HttpResponse<Buffer>> requestFuture = new VertxCompletableFuture<>();
client
.post(uri)
.sendBuffer(requestBody, requestFuture);
try {
return requestFuture.get();
} catch (Exception e) {
throw new RuntimeException(e);
}
}

}
Loading