-
Notifications
You must be signed in to change notification settings - Fork 1k
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 logging for every request to ksqlDB #6615
Changes from all commits
8caaa34
295a0a8
ff2e715
051a4dd
7d33fab
5d0425e
3e16849
291558e
32f6b8e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,154 @@ | ||
/* | ||
* Copyright 2020 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.api.server; | ||
|
||
import static io.confluent.ksql.rest.server.KsqlRestConfig.KSQL_LOGGING_SERVER_RATE_LIMITED_REQUEST_PATHS_CONFIG; | ||
import static io.confluent.ksql.rest.server.KsqlRestConfig.KSQL_LOGGING_SERVER_SKIPPED_RESPONSE_CODES_CONFIG; | ||
|
||
import com.google.common.annotations.VisibleForTesting; | ||
import com.google.common.collect.ImmutableMap; | ||
import com.google.common.collect.ImmutableSet; | ||
import com.google.common.util.concurrent.RateLimiter; | ||
import io.confluent.ksql.api.auth.ApiUser; | ||
import io.confluent.ksql.rest.server.KsqlRestConfig; | ||
import io.vertx.core.Handler; | ||
import io.vertx.core.http.HttpMethod; | ||
import io.vertx.core.http.HttpVersion; | ||
import io.vertx.ext.web.RoutingContext; | ||
import io.vertx.ext.web.impl.Utils; | ||
import java.time.Clock; | ||
import java.util.Map; | ||
import java.util.Map.Entry; | ||
import java.util.Optional; | ||
import java.util.Set; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.function.Function; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
public class LoggingHandler implements Handler<RoutingContext> { | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(LoggingHandler.class); | ||
static final String HTTP_HEADER_USER_AGENT = "User-Agent"; | ||
|
||
private final Set<Integer> skipResponseCodes; | ||
private final Map<String, Double> rateLimitedPaths; | ||
private final Logger logger; | ||
private final Clock clock; | ||
private final Function<Double, RateLimiter> rateLimiterFactory; | ||
|
||
private final Map<String, RateLimiter> rateLimiters = new ConcurrentHashMap<>(); | ||
|
||
public LoggingHandler(final Server server) { | ||
this(server, LOG, Clock.systemUTC(), RateLimiter::create); | ||
} | ||
|
||
@VisibleForTesting | ||
LoggingHandler( | ||
final Server server, | ||
final Logger logger, | ||
final Clock clock, | ||
final Function<Double, RateLimiter> rateLimiterFactory) { | ||
this.skipResponseCodes = getSkipResponseCodes(server.getConfig()); | ||
this.rateLimitedPaths = getSkipRequestPaths(server.getConfig()); | ||
this.logger = logger; | ||
this.clock = clock; | ||
this.rateLimiterFactory = rateLimiterFactory; | ||
} | ||
|
||
@Override | ||
public void handle(final RoutingContext routingContext) { | ||
routingContext.addEndHandler(ar -> { | ||
// After the response is complete, log results here. | ||
if (skipResponseCodes.contains(routingContext.response().getStatusCode())) { | ||
return; | ||
} | ||
if (rateLimitedPaths.containsKey(routingContext.request().path())) { | ||
final String path = routingContext.request().path(); | ||
final double rateLimit = rateLimitedPaths.get(path); | ||
rateLimiters.computeIfAbsent(path, (k) -> rateLimiterFactory.apply(rateLimit)); | ||
if (!rateLimiters.get(path).tryAcquire()) { | ||
return; | ||
} | ||
} | ||
final long contentLength = routingContext.request().response().bytesWritten(); | ||
final HttpVersion version = routingContext.request().version(); | ||
final HttpMethod method = routingContext.request().method(); | ||
final String uri = routingContext.request().uri(); | ||
final int status = routingContext.request().response().getStatusCode(); | ||
final long requestBodyLength = routingContext.request().bytesRead(); | ||
final String versionFormatted; | ||
switch (version) { | ||
case HTTP_1_0: | ||
versionFormatted = "HTTP/1.0"; | ||
break; | ||
case HTTP_1_1: | ||
versionFormatted = "HTTP/1.1"; | ||
break; | ||
case HTTP_2: | ||
versionFormatted = "HTTP/2.0"; | ||
break; | ||
default: | ||
versionFormatted = "-"; | ||
} | ||
final String name = Optional.ofNullable((ApiUser) routingContext.user()) | ||
.map(u -> u.getPrincipal().getName()) | ||
.orElse("-"); | ||
final String userAgent = Optional.ofNullable( | ||
routingContext.request().getHeader(HTTP_HEADER_USER_AGENT)).orElse("-"); | ||
final String timestamp = Utils.formatRFC1123DateTime(clock.millis()); | ||
final String message = String.format( | ||
"%s - %s [%s] \"%s %s %s\" %d %d \"-\" \"%s\" %d", | ||
routingContext.request().remoteAddress().host(), | ||
name, | ||
timestamp, | ||
method, | ||
uri, | ||
versionFormatted, | ||
status, | ||
contentLength, | ||
userAgent, | ||
requestBodyLength); | ||
doLog(status, message); | ||
}); | ||
routingContext.next(); | ||
} | ||
|
||
private static Set<Integer> getSkipResponseCodes(final KsqlRestConfig config) { | ||
// Already validated as all ints | ||
return config.getList(KSQL_LOGGING_SERVER_SKIPPED_RESPONSE_CODES_CONFIG) | ||
.stream() | ||
.map(Integer::parseInt).collect(ImmutableSet.toImmutableSet()); | ||
} | ||
|
||
private static Map<String, Double> getSkipRequestPaths(final KsqlRestConfig config) { | ||
// Already validated as having double values | ||
return config.getStringAsMap(KSQL_LOGGING_SERVER_RATE_LIMITED_REQUEST_PATHS_CONFIG) | ||
.entrySet().stream() | ||
.collect(ImmutableMap.toImmutableMap(Entry::getKey, | ||
entry -> Double.parseDouble(entry.getValue()))); | ||
} | ||
|
||
private void doLog(final int status, final String message) { | ||
if (status >= 500) { | ||
logger.error(message); | ||
} else if (status >= 400) { | ||
logger.warn(message); | ||
} else { | ||
logger.info(message); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -15,13 +15,16 @@ | |
|
||
package io.confluent.ksql.rest.server; | ||
|
||
import static io.confluent.ksql.configdef.ConfigValidators.intList; | ||
import static io.confluent.ksql.configdef.ConfigValidators.mapWithDoubleValue; | ||
import static io.confluent.ksql.configdef.ConfigValidators.oneOrMore; | ||
import static io.confluent.ksql.configdef.ConfigValidators.zeroOrPositive; | ||
|
||
import com.google.common.annotations.VisibleForTesting; | ||
import io.confluent.ksql.configdef.ConfigValidators; | ||
import io.confluent.ksql.rest.DefaultErrorMessages; | ||
import io.confluent.ksql.rest.ErrorMessages; | ||
import io.confluent.ksql.util.KsqlConfig; | ||
import io.confluent.ksql.util.KsqlException; | ||
import io.confluent.ksql.util.KsqlServerException; | ||
import io.vertx.core.http.ClientAuth; | ||
|
@@ -319,6 +322,16 @@ public class KsqlRestConfig extends AbstractConfig { | |
"The key store certificate alias to be used for internal client requests. If not set, " | ||
+ "the system will fall back on the Vert.x default choice"; | ||
|
||
public static final String KSQL_LOGGING_SERVER_SKIPPED_RESPONSE_CODES_CONFIG = | ||
KSQL_CONFIG_PREFIX + "logging.server.skipped.response.codes"; | ||
private static final String KSQL_LOGGING_SERVER_SKIPPED_RESPONSE_CODES_DOC = | ||
"A list of HTTP response codes to skip during server request logging"; | ||
|
||
public static final String KSQL_LOGGING_SERVER_RATE_LIMITED_REQUEST_PATHS_CONFIG = | ||
KSQL_CONFIG_PREFIX + "logging.server.rate.limited.request.paths"; | ||
private static final String KSQL_LOGGING_SERVER_RATE_LIMITED_REQUEST_PATHS_DOC = | ||
"A list of path:rate_limit pairs, to rate limit the server request logging"; | ||
|
||
private static final ConfigDef CONFIG_DEF; | ||
|
||
static { | ||
|
@@ -605,6 +618,20 @@ public class KsqlRestConfig extends AbstractConfig { | |
KSQL_AUTHENTICATION_PLUGIN_DEFAULT, | ||
ConfigDef.Importance.LOW, | ||
KSQL_AUTHENTICATION_PLUGIN_DOC | ||
).define( | ||
KSQL_LOGGING_SERVER_SKIPPED_RESPONSE_CODES_CONFIG, | ||
Type.LIST, | ||
"", | ||
intList(), | ||
ConfigDef.Importance.LOW, | ||
KSQL_LOGGING_SERVER_SKIPPED_RESPONSE_CODES_DOC | ||
).define( | ||
KSQL_LOGGING_SERVER_RATE_LIMITED_REQUEST_PATHS_CONFIG, | ||
Type.STRING, | ||
"", | ||
mapWithDoubleValue(), | ||
ConfigDef.Importance.LOW, | ||
KSQL_LOGGING_SERVER_RATE_LIMITED_REQUEST_PATHS_DOC | ||
); | ||
} | ||
|
||
|
@@ -903,4 +930,8 @@ private static String getLocalHostName() { | |
} | ||
} | ||
|
||
public Map<String, String> getStringAsMap(final String key) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This looks unused? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is called in |
||
final String value = getString(key).trim(); | ||
return KsqlConfig.parseStringAsMap(key, value); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does it make sense to add a short docs topic on the meaning of this format, since there are adaptations from the standard Apache log format?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since it's mixed with the other debug statements and the format may change if we get additional requested fields, it doesn't seem like should commit to a particular format for the moment. I'm happy to write up a short doc explaining the format if this ends up being what we stick with.