diff --git a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java index 4d9b781d2891..1e5ad8b22e4a 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java +++ b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java @@ -167,11 +167,21 @@ public class KsqlConfig extends AbstractConfig { "Config to enable or disable transient pull queries on a specific KSQL server."; public static final boolean KSQL_QUERY_PULL_ENABLE_DEFAULT = true; - public static final String KSQL_QUERY_PULL_ROUTING_TIMEOUT_MS_CONFIG = - "ksql.query.pull.routing.timeout.ms"; - public static final Long KSQL_QUERY_PULL_ROUTING_TIMEOUT_MS_DEFAULT = 30000L; - public static final String KSQL_QUERY_PULL_ROUTING_TIMEOUT_MS_DOC = "Timeout in milliseconds " - + "when waiting for the lookup of the owner of a row key"; + public static final String KSQL_QUERY_PULL_ENABLE_STANDBY_READS = + "ksql.query.pull.enable.stale.reads"; + private static final String KSQL_QUERY_PULL_ENABLE_STANDBY_READS_DOC = + "Config to enable/disable forwarding pull queries to standby hosts when the active is dead. " + + "This means that stale values may be returned for these queries since standby hosts" + + "receive updates from the changelog topic (to which the active writes to) " + + "asynchronously. Turning on this configuration, effectively sacrifices " + + "consistency for higher availability. " + + "Possible values are \"true\", \"false\". Setting to \"true\" guarantees high " + + "availability for pull queries. If set to \"false\", pull queries will fail when" + + "the active is dead and until a new active is elected. Default value is \"false\". " + + "For using this functionality, the server must be configured with " + + "to ksql.streams.num.standby.replicas >= 1"; + public static final boolean KSQL_QUERY_PULL_ENABLE_STANDBY_READS_DEFAULT = false; + public static final String KSQL_QUERY_PULL_STREAMSTORE_REBALANCING_TIMEOUT_MS_CONFIG = "ksql.query.pull.streamsstore.rebalancing.timeout.ms"; @@ -517,11 +527,11 @@ private static ConfigDef buildConfigDef(final ConfigGeneration generation) { Importance.LOW, KSQL_QUERY_PULL_ENABLE_DOC ).define( - KSQL_QUERY_PULL_ROUTING_TIMEOUT_MS_CONFIG, - ConfigDef.Type.LONG, - KSQL_QUERY_PULL_ROUTING_TIMEOUT_MS_DEFAULT, - Importance.LOW, - KSQL_QUERY_PULL_ROUTING_TIMEOUT_MS_DOC + KSQL_QUERY_PULL_ENABLE_STANDBY_READS, + Type.BOOLEAN, + KSQL_QUERY_PULL_ENABLE_STANDBY_READS_DEFAULT, + Importance.MEDIUM, + KSQL_QUERY_PULL_ENABLE_STANDBY_READS_DOC ).define( KSQL_QUERY_PULL_STREAMSTORE_REBALANCING_TIMEOUT_MS_CONFIG, ConfigDef.Type.LONG, diff --git a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlHost.java b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlHostInfo.java similarity index 82% rename from ksql-common/src/main/java/io/confluent/ksql/util/KsqlHost.java rename to ksql-common/src/main/java/io/confluent/ksql/util/KsqlHostInfo.java index 50da8cdaa47e..5e65da19bef1 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlHost.java +++ b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlHostInfo.java @@ -18,17 +18,18 @@ import com.google.errorprone.annotations.Immutable; import java.util.Objects; + /** * Immutable representation of {@link org.apache.kafka.streams.state.HostInfo HostInfo} * from KStreams. */ @Immutable -public class KsqlHost { +public class KsqlHostInfo { private final String host; private final int port; - public KsqlHost(final String host, final int port) { + public KsqlHostInfo(final String host, final int port) { this.host = host; this.port = port; } @@ -42,8 +43,8 @@ public boolean equals(final Object o) { return false; } - final KsqlHost hostInfo = (KsqlHost) o; - return port == hostInfo.port && host.equals(hostInfo.host); + final KsqlHostInfo other = (KsqlHostInfo) o; + return this.host.equals(other.host) && port == other.port; } @Override @@ -61,6 +62,6 @@ public int port() { @Override public String toString() { - return "KsqlHost{host='" + this.host + '\'' + ", port=" + this.port + '}'; + return "KsqlHostInfo{host='" + this.host + '\'' + ", port=" + this.port + '}'; } } diff --git a/ksql-execution/src/main/java/io/confluent/ksql/services/DisabledKsqlClient.java b/ksql-execution/src/main/java/io/confluent/ksql/services/DisabledKsqlClient.java index d220375fd01f..c879159f0c43 100644 --- a/ksql-execution/src/main/java/io/confluent/ksql/services/DisabledKsqlClient.java +++ b/ksql-execution/src/main/java/io/confluent/ksql/services/DisabledKsqlClient.java @@ -20,7 +20,7 @@ import io.confluent.ksql.rest.entity.KsqlEntityList; import io.confluent.ksql.rest.entity.LagReportingMessage; import io.confluent.ksql.rest.entity.StreamedRow; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import java.net.URI; import java.util.List; @@ -52,7 +52,7 @@ public RestResponse> makeQueryRequest( @Override public void makeAsyncHeartbeatRequest( final URI serverEndPoint, - final KsqlHost host, + final KsqlHostInfo host, final long timestamp ) { throw new UnsupportedOperationException("KSQL client is disabled"); diff --git a/ksql-execution/src/main/java/io/confluent/ksql/services/SimpleKsqlClient.java b/ksql-execution/src/main/java/io/confluent/ksql/services/SimpleKsqlClient.java index 3accbf7ebe6f..a47349c75d13 100644 --- a/ksql-execution/src/main/java/io/confluent/ksql/services/SimpleKsqlClient.java +++ b/ksql-execution/src/main/java/io/confluent/ksql/services/SimpleKsqlClient.java @@ -20,7 +20,7 @@ import io.confluent.ksql.rest.entity.KsqlEntityList; import io.confluent.ksql.rest.entity.LagReportingMessage; import io.confluent.ksql.rest.entity.StreamedRow; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import java.net.URI; import java.util.List; import javax.annotation.concurrent.ThreadSafe; @@ -46,7 +46,7 @@ RestResponse> makeQueryRequest( */ void makeAsyncHeartbeatRequest( URI serverEndPoint, - KsqlHost host, + KsqlHostInfo host, long timestamp ); @@ -58,7 +58,8 @@ void makeAsyncHeartbeatRequest( RestResponse makeClusterStatusRequest(URI serverEndPoint); /** - * Send lag information to remote Ksql server. + * Send a request to remote Ksql server to inquire to inquire about which state stores the + * remote server maintains as an active and standby. * @param serverEndPoint the remote destination. * @param lagReportingMessage the host lag data */ diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/ActiveHostFilter.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/ActiveHostFilter.java new file mode 100644 index 000000000000..5d7daeed8de0 --- /dev/null +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/ActiveHostFilter.java @@ -0,0 +1,47 @@ +/* + * 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.rest.server; + +import io.confluent.ksql.execution.streams.RoutingFilter; +import io.confluent.ksql.util.KsqlHostInfo; +import org.apache.kafka.streams.state.HostInfo; + +/** + * Filters for the active host. + */ +public class ActiveHostFilter implements RoutingFilter { + + public ActiveHostFilter() { + } + + /** + * Returns true if the host is the active host for a particular state store. + * @param activeHost the active host for a particular state store + * @param host The host for which the status is checked + * @param storeName Ignored + * @param partition Ignored + * @return true if the host is the active, false otherwise + */ + @Override + public boolean filter( + final HostInfo activeHost, + final KsqlHostInfo host, + final String storeName, + final int partition) { + + return host.host().equals(activeHost.host()) && host.port() == activeHost.port(); + } +} diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/HeartbeatAgent.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/HeartbeatAgent.java index 96c2743fe2da..b6af30d0970e 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/HeartbeatAgent.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/HeartbeatAgent.java @@ -25,7 +25,7 @@ import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.util.HostStatus; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import io.confluent.ksql.util.PersistentQueryMetadata; import java.net.URI; import java.net.URL; @@ -80,12 +80,12 @@ public final class HeartbeatAgent { private final ServiceContext serviceContext; private final HeartbeatConfig config; private final List hostStatusListeners; - private final ConcurrentHashMap> receivedHeartbeats; - private final ConcurrentHashMap hostsStatus; + private final ConcurrentHashMap> receivedHeartbeats; + private final ConcurrentHashMap hostsStatus; private final ScheduledExecutorService scheduledExecutorService; private final ServiceManager serviceManager; private final Clock clock; - private KsqlHost localHost; + private KsqlHostInfo localHost; private URL localUrl; public static HeartbeatAgent.Builder builder() { @@ -114,7 +114,7 @@ private HeartbeatAgent(final KsqlEngine engine, * @param hostInfo The host information of the remote Ksql server. * @param timestamp The timestamp the heartbeat was sent. */ - public void receiveHeartbeat(final KsqlHost hostInfo, final long timestamp) { + public void receiveHeartbeat(final KsqlHostInfo hostInfo, final long timestamp) { final TreeMap heartbeats = receivedHeartbeats.computeIfAbsent( hostInfo, key -> new TreeMap<>()); synchronized (heartbeats) { @@ -127,12 +127,12 @@ public void receiveHeartbeat(final KsqlHost hostInfo, final long timestamp) { * Returns the current view of the cluster containing all hosts discovered (whether alive or dead) * @return status of discovered hosts */ - public Map getHostsStatus() { + public Map getHostsStatus() { return Collections.unmodifiableMap(hostsStatus); } @VisibleForTesting - void setHostsStatus(final Map status) { + void setHostsStatus(final Map status) { hostsStatus.putAll(status); } @@ -156,7 +156,7 @@ void stopAgent() { void setLocalAddress(final String applicationServer) { final HostInfo hostInfo = ServerUtil.parseHostInfo(applicationServer); - this.localHost = new KsqlHost(hostInfo.host(), hostInfo.port()); + this.localHost = new KsqlHostInfo(hostInfo.host(), hostInfo.port()); try { this.localUrl = new URL(applicationServer); } catch (final Exception e) { @@ -226,30 +226,30 @@ private void processHeartbeats(final long windowStart, final long windowEnd) { return; } - for (Entry hostEntry: hostsStatus.entrySet()) { - final KsqlHost ksqlHost = hostEntry.getKey(); + for (Entry hostEntry: hostsStatus.entrySet()) { + final KsqlHostInfo ksqlHostInfo = hostEntry.getKey(); final HostStatus hostStatus = hostEntry.getValue(); - if (ksqlHost.equals(localHost)) { + if (ksqlHostInfo.equals(localHost)) { continue; } - final TreeMap heartbeats = receivedHeartbeats.get(ksqlHost); + final TreeMap heartbeats = receivedHeartbeats.get(ksqlHostInfo); //For previously discovered hosts, if they have not received any heartbeats, mark them dead if (heartbeats == null || heartbeats.isEmpty()) { - hostsStatus.computeIfPresent(ksqlHost, (host, status) -> status.withHostAlive(false)); + hostsStatus.computeIfPresent(ksqlHostInfo, (host, status) -> status.withHostAlive(false)); } else { final TreeMap copy; synchronized (heartbeats) { - LOG.debug("Process heartbeats: {} of host: {}", heartbeats, ksqlHost); + LOG.debug("Process heartbeats: {} of host: {}", heartbeats, ksqlHostInfo); // 1. remove heartbeats older than window heartbeats.headMap(windowStart).clear(); copy = new TreeMap<>(heartbeats.subMap(windowStart, true, windowEnd, true)); } // 2. count consecutive missed heartbeats and mark as alive or dead - final boolean isAlive = decideStatus(ksqlHost, windowStart, windowEnd, copy); + final boolean isAlive = decideStatus(ksqlHostInfo, windowStart, windowEnd, copy); if (!isAlive) { - LOG.info("Host: {} marked as dead.", ksqlHost); + LOG.info("Host: {} marked as dead.", ksqlHostInfo); } - hostsStatus.computeIfPresent(ksqlHost, (host, status) -> status + hostsStatus.computeIfPresent(ksqlHostInfo, (host, status) -> status .withHostAlive(isAlive).withLastStatusUpdateMs(windowEnd)); } } @@ -259,7 +259,7 @@ private void processHeartbeats(final long windowStart, final long windowEnd) { } private boolean decideStatus( - final KsqlHost ksqlHost, final long windowStart, final long windowEnd, + final KsqlHostInfo ksqlHostInfo, final long windowStart, final long windowEnd, final TreeMap heartbeats ) { long missedCount = 0; @@ -289,7 +289,7 @@ private boolean decideStatus( if (windowEnd - prev - 1 > 0) { missedCount = (windowEnd - prev - 1) / config.heartbeatSendIntervalMs; } - LOG.debug("Host: {} has {} missing heartbeats", ksqlHost, missedCount); + LOG.debug("Host: {} has {} missing heartbeats", ksqlHostInfo, missedCount); return (missedCount < config.heartbeatMissedThreshold); } } @@ -306,8 +306,8 @@ class SendHeartbeatService extends AbstractScheduledService { @Override protected void runOneIteration() { - for (Entry hostStatusEntry: hostsStatus.entrySet()) { - final KsqlHost remoteHost = hostStatusEntry.getKey(); + for (Entry hostStatusEntry: hostsStatus.entrySet()) { + final KsqlHostInfo remoteHost = hostStatusEntry.getKey(); try { if (!remoteHost.equals(localHost)) { final URI remoteUri = ServerUtil.buildRemoteUri( @@ -364,7 +364,7 @@ protected void runOneIteration() { // Only add to map if it is the first time it is discovered. Design decision to // optimistically consider every newly discovered server as alive to avoid situations of // unavailability until the heartbeating kicks in. - final KsqlHost host = new KsqlHost(hostInfo.host(), hostInfo.port()); + final KsqlHostInfo host = new KsqlHostInfo(hostInfo.host(), hostInfo.port()); hostsStatus.computeIfAbsent(host, key -> new HostStatus(true, clock.millis())); } } catch (Throwable t) { @@ -492,6 +492,6 @@ public interface HostStatusListener { * Call when the map of host statuses are updated * @param hostsStatusMap The new host status map */ - void onHostStatusUpdated(Map hostsStatusMap); + void onHostStatusUpdated(Map hostsStatusMap); } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index eeea549f4573..e11569f45761 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -30,6 +30,8 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.ServiceInfo; import io.confluent.ksql.engine.KsqlEngine; +import io.confluent.ksql.execution.streams.RoutingFilter; +import io.confluent.ksql.execution.streams.RoutingFilters; import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.function.MutableFunctionRegistry; import io.confluent.ksql.function.UserFunctionLoader; @@ -51,6 +53,7 @@ import io.confluent.ksql.rest.server.computation.CommandStore; import io.confluent.ksql.rest.server.computation.InteractiveStatementExecutor; import io.confluent.ksql.rest.server.context.KsqlSecurityContextBinder; +import io.confluent.ksql.rest.server.execution.PullQueryExecutor; import io.confluent.ksql.rest.server.filters.KsqlAuthorizationFilter; import io.confluent.ksql.rest.server.resources.ClusterStatusResource; import io.confluent.ksql.rest.server.resources.HealthCheckResource; @@ -229,7 +232,8 @@ public void setupResources(final Configurable config, final KsqlRestConfig ap config.register(HealthCheckResource.create(ksqlResource, serviceContext, this.config)); if (heartbeatAgent.isPresent()) { config.register(new HeartbeatResource(heartbeatAgent.get())); - config.register(new ClusterStatusResource(heartbeatAgent.get(), lagReportingAgent)); + config.register(new ClusterStatusResource( + ksqlEngine, heartbeatAgent.get(), lagReportingAgent)); } if (lagReportingAgent.isPresent()) { config.register(new LagReportingResource(lagReportingAgent.get())); @@ -472,6 +476,11 @@ protected void registerWebSocketEndpoints(final ServerContainer container) { ErrorMessages.class )); + final RoutingFilters routingFilters = initializeRoutingFilters( + ksqlConfigNoPort, heartbeatAgent); + final PullQueryExecutor pullQueryExecutor = new PullQueryExecutor( + ksqlEngine, heartbeatAgent, routingFilters); + container.addEndpoint( ServerEndpointConfig.Builder .create( @@ -496,7 +505,8 @@ public T getEndpointInstance(final Class endpointClass) { errorHandler, securityExtension, serverState, - serviceContext.getSchemaRegistryClientFactory() + serviceContext.getSchemaRegistryClientFactory(), + pullQueryExecutor ); } }) @@ -597,6 +607,14 @@ static KsqlRestApplication buildApplication( ErrorMessages.class )); + final Optional lagReportingAgent = + initializeLagReportingAgent(restConfig, ksqlEngine, serviceContext); + final Optional heartbeatAgent = + initializeHeartbeatAgent(restConfig, ksqlEngine, serviceContext, lagReportingAgent); + final RoutingFilters routingFilters = initializeRoutingFilters(ksqlConfig, heartbeatAgent); + + final PullQueryExecutor pullQueryExecutor = new PullQueryExecutor( + ksqlEngine, heartbeatAgent, routingFilters); final StreamedQueryResource streamedQueryResource = new StreamedQueryResource( ksqlEngine, commandStore, @@ -605,7 +623,8 @@ static KsqlRestApplication buildApplication( Duration.ofMillis(restConfig.getLong(DISTRIBUTED_COMMAND_RESPONSE_TIMEOUT_MS_CONFIG)), versionChecker::updateLastRequestTime, authorizationValidator, - errorHandler + errorHandler, + pullQueryExecutor ); final KsqlResource ksqlResource = new KsqlResource( @@ -649,11 +668,6 @@ static KsqlRestApplication buildApplication( final Consumer rocksDBConfigSetterHandler = RocksDBConfigSetterHandler::maybeConfigureRocksDBConfigSetter; - final Optional lagReportingAgent = - initializeLagReportingAgent(restConfig, ksqlEngine, serviceContext); - final Optional heartbeatAgent = - initializeHeartbeatAgent(restConfig, ksqlEngine, serviceContext, lagReportingAgent); - return new KsqlRestApplication( serviceContext, ksqlEngine, @@ -726,6 +740,18 @@ private static Optional initializeLagReportingAgent( return Optional.empty(); } + private static RoutingFilters initializeRoutingFilters( + final KsqlConfig ksqlConfig, + final Optional heartbeatAgent) { + final ImmutableList.Builder filterBuilder = ImmutableList.builder(); + if (!ksqlConfig.getBoolean(KsqlConfig.KSQL_QUERY_PULL_ENABLE_STANDBY_READS)) { + filterBuilder.add(new ActiveHostFilter()); + } + filterBuilder.add(new LivenessFilter(heartbeatAgent)); + final RoutingFilters routingFilters = new RoutingFilters(filterBuilder.build()); + return routingFilters; + } + private void registerCommandTopic() { final String commandTopic = commandStore.getCommandTopicName(); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestConfig.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestConfig.java index d523732fb3bc..dcb0dec09abf 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestConfig.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestConfig.java @@ -153,7 +153,6 @@ public class KsqlRestConfig extends RestConfig { KSQL_CONFIG_PREFIX + "lag.reporting.enable"; private static final String KSQL_LAG_REPORTING_ENABLE_DOC = "Whether lag reporting is enabled or not. It is disabled by default."; - public static final String KSQL_LAG_REPORTING_SEND_INTERVAL_MS_CONFIG = KSQL_CONFIG_PREFIX + "lag.reporting.send.interval.ms"; private static final String KSQL_LAG_REPORTING_SEND_INTERVAL_MS_DOC = diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/LagReportingAgent.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/LagReportingAgent.java index 788fbe94ef56..62e7d8ddaea8 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/LagReportingAgent.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/LagReportingAgent.java @@ -23,7 +23,7 @@ import com.google.common.util.concurrent.ServiceManager; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.rest.entity.HostStoreLags; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.entity.LagInfoEntity; import io.confluent.ksql.rest.entity.LagReportingMessage; import io.confluent.ksql.rest.entity.QueryStateStoreId; @@ -31,7 +31,7 @@ import io.confluent.ksql.rest.server.HeartbeatAgent.HostStatusListener; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.util.HostStatus; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import io.confluent.ksql.util.Pair; import io.confluent.ksql.util.PersistentQueryMetadata; import java.net.URI; @@ -75,8 +75,8 @@ public final class LagReportingAgent implements HostStatusListener { private final ServiceManager serviceManager; private final Clock clock; - private final Map receivedLagInfo; - private final AtomicReference> aliveHostsRef; + private final Map receivedLagInfo; + private final AtomicReference> aliveHostsRef; private URL localUrl; @@ -147,13 +147,13 @@ void stopAgent() { public void receiveHostLag(final LagReportingMessage lagReportingMessage) { final HostStoreLags hostStoreLags = lagReportingMessage.getHostStoreLags(); final long updateTimeMs = hostStoreLags.getUpdateTimeMs(); - final KsqlHostEntity KsqlHostEntity = lagReportingMessage.getKsqlHost(); - final KsqlHost KsqlHost = KsqlHostEntity.toKsqlHost(); + final KsqlHostInfoEntity KsqlHostInfoEntity = lagReportingMessage.getKsqlHost(); + final KsqlHostInfo KsqlHostInfo = KsqlHostInfoEntity.toKsqlHost(); - LOG.debug("Receive lag at: {} from host: {} lag: {} ", updateTimeMs, KsqlHostEntity, - hostStoreLags.getStateStoreLags()); + LOG.debug("Receive lag at: {} from host: {} lag: {} ", updateTimeMs, KsqlHostInfoEntity, + hostStoreLags.getStateStoreLags()); - receivedLagInfo.compute(KsqlHost, (hi, previousHostLagInfo) -> + receivedLagInfo.compute(KsqlHostInfo, (hi, previousHostLagInfo) -> previousHostLagInfo != null && previousHostLagInfo.getUpdateTimeMs() > updateTimeMs ? previousHostLagInfo : hostStoreLags); } @@ -165,8 +165,8 @@ public void receiveHostLag(final LagReportingMessage lagReportingMessage) { * @return A map which is keyed by host and contains lag information */ public Optional getHostsPartitionLagInfo( - final KsqlHost host, final QueryStateStoreId queryStateStoreId, final int partition) { - final Set aliveHosts = aliveHostsRef.get(); + final KsqlHostInfo host, final QueryStateStoreId queryStateStoreId, final int partition) { + final Set aliveHosts = aliveHostsRef.get(); if (!aliveHosts.contains(host)) { return Optional.empty(); } @@ -179,19 +179,19 @@ public Optional getHostsPartitionLagInfo( * Returns a map of host -> store -> partition -> LagInfo. Meant for being exposed in testing * and debug resources. */ - public ImmutableMap getAllLags() { + public ImmutableMap getAllLags() { return receivedLagInfo.entrySet().stream() .collect(ImmutableMap.toImmutableMap( - e -> new KsqlHostEntity(e.getKey().host(), e.getKey().port()), + e -> new KsqlHostInfoEntity(e.getKey().host(), e.getKey().port()), Entry::getValue)); } - public Optional getLagPerHost(final KsqlHost host) { + public Optional getLagPerHost(final KsqlHostInfo host) { return Optional.ofNullable(receivedLagInfo.get(host)); } @Override - public void onHostStatusUpdated(final Map hostsStatusMap) { + public void onHostStatusUpdated(final Map hostsStatusMap) { aliveHostsRef.set(hostsStatusMap.entrySet().stream() .filter(entry -> entry.getValue().isHostAlive()) .map(Entry::getKey) @@ -226,8 +226,8 @@ protected void runOneIteration() { final LagReportingMessage message = createLagReportingMessage(localLagMap); - final Set aliveHosts = aliveHostsRef.get(); - for (KsqlHost host: aliveHosts) { + final Set aliveHosts = aliveHostsRef.get(); + for (KsqlHostInfo host: aliveHosts) { try { final URI remoteUri = ServerUtil.buildRemoteUri(localUrl, host.host(), host.port()); LOG.debug("Sending lag to host {} at {}", host.host(), clock.millis()); @@ -258,7 +258,7 @@ private LagReportingMessage createLagReportingMessage( }).collect(ImmutableMap.toImmutableMap(Pair::getLeft, Pair::getRight)); map.put(storeEntry.getKey(), new StateStoreLags(partitionMap)); } - return new LagReportingMessage(new KsqlHostEntity(localUrl.getHost(), localUrl.getPort()), + return new LagReportingMessage(new KsqlHostInfoEntity(localUrl.getHost(), localUrl.getPort()), new HostStoreLags(map.build(), clock.millis())); } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/LivenessFilter.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/LivenessFilter.java new file mode 100644 index 000000000000..666cff8c81c4 --- /dev/null +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/LivenessFilter.java @@ -0,0 +1,61 @@ +/* + * 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.rest.server; + +import io.confluent.ksql.execution.streams.RoutingFilter; +import io.confluent.ksql.util.HostStatus; +import io.confluent.ksql.util.KsqlHostInfo; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import org.apache.kafka.streams.state.HostInfo; + +/** + * Filters ksql hosts based on whether they are alive or dead. + */ +public class LivenessFilter implements RoutingFilter { + + private final Optional heartbeatAgent; + + public LivenessFilter(final Optional heartbeatAgent) { + this.heartbeatAgent = Objects.requireNonNull(heartbeatAgent, "heartbeatAgent"); + } + + /** + * Returns true if the host is alive. If the heartbeat agent is not enabled, all hosts are + * assumed to be alive. + * @param activeHost the active host for a particular state store + * @param host The host for which the status is checked + * @param storeName Ignored + * @param partition Ignored + * @return true if the host is alive, false otherwise. + */ + @Override + public boolean filter( + final HostInfo activeHost, + final KsqlHostInfo host, + final String storeName, + final int partition) { + + if (!heartbeatAgent.isPresent()) { + return true; + } + + final Map allHostsStatus = heartbeatAgent.get().getHostsStatus(); + final HostStatus status = allHostsStatus.get(host); + return status == null ? true : allHostsStatus.get(host).isHostAlive(); + } +} diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java index 1152a5f9e0c2..ee2e167a7c26 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java @@ -32,7 +32,6 @@ import io.confluent.ksql.parser.tree.ListTables; import io.confluent.ksql.parser.tree.ListTopics; import io.confluent.ksql.parser.tree.ListTypes; -import io.confluent.ksql.parser.tree.Query; import io.confluent.ksql.parser.tree.SetProperty; import io.confluent.ksql.parser.tree.ShowColumns; import io.confluent.ksql.parser.tree.Statement; @@ -55,8 +54,6 @@ @SuppressWarnings({"unchecked", "rawtypes"}) public enum CustomExecutors { - PULL_QUERY(Query.class, PullQueryExecutor::execute), - LIST_TOPICS(ListTopics.class, ListTopicsExecutor::execute), LIST_STREAMS(ListStreams.class, ListSourceExecutor::streams), LIST_TABLES(ListTables.class, ListSourceExecutor::tables), diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java index 7525110f91aa..d24587b50b0d 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java @@ -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; @@ -45,10 +46,11 @@ import io.confluent.ksql.execution.expression.tree.UnqualifiedColumnReferenceExp; import io.confluent.ksql.execution.expression.tree.VisitParentExpressionVisitor; import io.confluent.ksql.execution.plan.SelectExpression; +import io.confluent.ksql.execution.streams.RoutingFilter; import io.confluent.ksql.execution.streams.materialization.Locator; import io.confluent.ksql.execution.streams.materialization.Locator.KsqlNode; import io.confluent.ksql.execution.streams.materialization.Materialization; -import io.confluent.ksql.execution.streams.materialization.MaterializationTimeOutException; +import io.confluent.ksql.execution.streams.materialization.MaterializationException; import io.confluent.ksql.execution.streams.materialization.PullProcessingContext; import io.confluent.ksql.execution.streams.materialization.TableRow; import io.confluent.ksql.execution.transform.KsqlTransformer; @@ -67,11 +69,11 @@ import io.confluent.ksql.query.QueryId; import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.client.RestResponse; -import io.confluent.ksql.rest.entity.KsqlEntity; import io.confluent.ksql.rest.entity.StreamedRow; import io.confluent.ksql.rest.entity.StreamedRow.Header; import io.confluent.ksql.rest.entity.TableRowsEntity; import io.confluent.ksql.rest.entity.TableRowsEntityFactory; +import io.confluent.ksql.rest.server.HeartbeatAgent; import io.confluent.ksql.rest.server.resources.KsqlRestException; import io.confluent.ksql.schema.ksql.DefaultSqlValueCoercer; import io.confluent.ksql.schema.ksql.FormatOptions; @@ -101,11 +103,15 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.kafka.connect.data.Struct; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; // CHECKSTYLE_RULES.OFF: ClassDataAbstractionCoupling public final class PullQueryExecutor { // CHECKSTYLE_RULES.ON: ClassDataAbstractionCoupling + private static final Logger LOG = LoggerFactory.getLogger(PullQueryExecutor.class); + private static final Set VALID_WINDOW_BOUNDS_TYPES = ImmutableSet.of( Type.EQUAL, Type.GREATER_THAN, @@ -117,30 +123,31 @@ public final class PullQueryExecutor { private static final String VALID_WINDOW_BOUNDS_TYPES_STRING = VALID_WINDOW_BOUNDS_TYPES.toString(); - private PullQueryExecutor() { - } + private final KsqlExecutionContext executionContext; + private final Optional heartbeatAgent; + private final RoutingFilter routingFilters; - public static void validate( - final ConfiguredStatement statement, - final Map sessionProperties, + public PullQueryExecutor( final KsqlExecutionContext executionContext, - final ServiceContext serviceContext + final Optional heartbeatAgent, + final RoutingFilter routingFilters ) { - throw new KsqlRestException(Errors.queryEndpoint(statement.getStatementText())); + this.executionContext = Objects.requireNonNull(executionContext, "executionContext"); + this.heartbeatAgent = Objects.requireNonNull(heartbeatAgent, "heartbeatAgent"); + this.routingFilters = Objects.requireNonNull(routingFilters, "routingFilters"); } - public static Optional execute( + public static void validate( final ConfiguredStatement statement, final Map sessionProperties, final KsqlExecutionContext executionContext, final ServiceContext serviceContext ) { - return Optional.of(execute(statement, executionContext, serviceContext)); + throw new KsqlRestException(Errors.queryEndpoint(statement.getStatementText())); } - public static TableRowsEntity execute( + public TableRowsEntity execute( final ConfiguredStatement statement, - final KsqlExecutionContext executionContext, final ServiceContext serviceContext ) { if (!statement.getStatement().isPullQuery()) { @@ -167,6 +174,7 @@ public static TableRowsEntity execute( final WhereInfo whereInfo = extractWhereInfo(analysis, query); final QueryId queryId = uniqueQueryId(); + final QueryContext.Stacker contextStacker = new Stacker(); final Materialization mat = query @@ -175,70 +183,139 @@ public static TableRowsEntity execute( final Struct rowKey = asKeyStruct(whereInfo.rowkey, query.getPhysicalSchema()); - final KsqlConfig ksqlConfig = statement.getConfig(); - final KsqlNode owner = getOwner(ksqlConfig, rowKey, mat); - if (!owner.isLocal()) { - return proxyTo(owner, statement, serviceContext); - } + final PullQueryContext pullQueryContext = new PullQueryContext( + rowKey, + mat, + analysis, + whereInfo, + queryId, + contextStacker); - final Result result; - if (whereInfo.windowStartBounds.isPresent()) { - final Range windowStart = whereInfo.windowStartBounds.get(); + return handlePullQuery( + statement, + executionContext, + serviceContext, + pullQueryContext + ); - final List rows = mat.windowed() - .get(rowKey, windowStart); + } catch (final Exception e) { + throw new KsqlStatementException( + e.getMessage() == null ? "Server Error" : e.getMessage(), + statement.getStatementText(), + e + ); + } + } - result = new Result(mat.schema(), rows); - } else { - final List rows = mat.nonWindowed() - .get(rowKey) - .map(ImmutableList::of) - .orElse(ImmutableList.of()); + private TableRowsEntity handlePullQuery( + final ConfiguredStatement statement, + final KsqlExecutionContext executionContext, + final ServiceContext serviceContext, + final PullQueryContext pullQueryContext + ) { + // Get active and standby nodes for this key + final Locator locator = pullQueryContext.mat.locator(); + final List filteredAndOrderedNodes = locator.locate( + pullQueryContext.rowKey, + routingFilters + ); - result = new Result(mat.schema(), rows); - } + if (filteredAndOrderedNodes.isEmpty()) { + throw new MaterializationException("All nodes are dead or exceed max allowed lag."); + } - final LogicalSchema outputSchema; - final List> rows; - if (isSelectStar(statement.getStatement().getSelect())) { - outputSchema = TableRowsEntityFactory - .buildSchema(result.schema, mat.windowType().isPresent()); - rows = TableRowsEntityFactory.createRows(result.rows); - } else { - outputSchema = selectOutputSchema(result, executionContext, analysis, mat.windowType()); - - rows = handleSelects( - result, - statement, - executionContext, - analysis, - outputSchema, - mat.windowType(), - queryId, - contextStacker - ); + // Nodes are ordered by preference: active is first if alive then standby nodes in + // increasing order of lag. + for (KsqlNode node : filteredAndOrderedNodes) { + try { + return routeQuery(node, statement, executionContext, serviceContext, pullQueryContext); + } catch (Exception t) { + LOG.debug("Error routing query {} to host {} at timestamp {}", + statement.getStatementText(), node, System.currentTimeMillis()); } + } + throw new MaterializationException(String.format( + "Unable to execute pull query: %s", statement.getStatementText())); + } - return new TableRowsEntity( - statement.getStatementText(), - queryId, + private TableRowsEntity routeQuery( + final KsqlNode node, + final ConfiguredStatement statement, + final KsqlExecutionContext executionContext, + final ServiceContext serviceContext, + final PullQueryContext pullQueryContext + ) { + + if (node.isLocal()) { + LOG.debug("Query {} executed locally at host {} at timestamp {}.", + statement.getStatementText(), node.location(), System.currentTimeMillis()); + return queryRowsLocally( + statement, + executionContext, + pullQueryContext); + } else { + LOG.debug("Query {} routed to host {} at timestamp {}.", + statement.getStatementText(), node.location(), System.currentTimeMillis()); + return forwardTo(node, statement, serviceContext); + } + } + + @VisibleForTesting + TableRowsEntity queryRowsLocally( + final ConfiguredStatement statement, + final KsqlExecutionContext executionContext, + final PullQueryContext pullQueryContext + ) { + final Result result; + if (pullQueryContext.whereInfo.windowStartBounds.isPresent()) { + final Range windowStart = pullQueryContext.whereInfo.windowStartBounds.get(); + + final List rows = pullQueryContext.mat.windowed() + .get(pullQueryContext.rowKey, windowStart); + + result = new Result(pullQueryContext.mat.schema(), rows); + } else { + final List rows = pullQueryContext.mat.nonWindowed() + .get(pullQueryContext.rowKey) + .map(ImmutableList::of) + .orElse(ImmutableList.of()); + + result = new Result(pullQueryContext.mat.schema(), rows); + } + + final LogicalSchema outputSchema; + final List> rows; + if (isSelectStar(statement.getStatement().getSelect())) { + outputSchema = TableRowsEntityFactory.buildSchema( + result.schema, pullQueryContext.mat.windowType().isPresent()); + rows = TableRowsEntityFactory.createRows(result.rows); + } else { + outputSchema = selectOutputSchema( + result, executionContext, pullQueryContext.analysis, pullQueryContext.mat.windowType()); + rows = handleSelects( + result, + statement, + executionContext, + pullQueryContext.analysis, outputSchema, - rows - ); - } catch (final Exception e) { - throw new KsqlStatementException( - e.getMessage() == null ? "Server Error" : e.getMessage(), - statement.getStatementText(), - e + pullQueryContext.mat.windowType(), + pullQueryContext.queryId, + pullQueryContext.contextStacker ); } + return new TableRowsEntity( + statement.getStatementText(), + pullQueryContext.queryId, + outputSchema, + rows + ); } - private static QueryId uniqueQueryId() { + private QueryId uniqueQueryId() { return new QueryId("query_" + System.currentTimeMillis()); } - private static ImmutableAnalysis analyze( + private ImmutableAnalysis analyze( final ConfiguredStatement statement, final KsqlExecutionContext executionContext ) { @@ -251,6 +328,55 @@ private static ImmutableAnalysis analyze( return queryAnalyzer.analyze(statement.getStatement(), Optional.empty()); } + private static final class PullQueryContext { + private final Struct rowKey; + private final Materialization mat; + private final ImmutableAnalysis analysis; + private final WhereInfo whereInfo; + private final QueryId queryId; + private final QueryContext.Stacker contextStacker; + + private PullQueryContext( + final Struct rowKey, + final Materialization mat, + final ImmutableAnalysis analysis, + final WhereInfo whereInfo, + final QueryId queryId, + final QueryContext.Stacker contextStacker + ) { + this.rowKey = Objects.requireNonNull(rowKey, "rowkey"); + this.mat = Objects.requireNonNull(mat, "materialization"); + this.analysis = Objects.requireNonNull(analysis, "analysis"); + this.whereInfo = Objects.requireNonNull(whereInfo, "whereInfo"); + this.queryId = Objects.requireNonNull(queryId, "queryId"); + this.contextStacker = Objects.requireNonNull(contextStacker, "contextStacker"); + } + + public Struct getRowKey() { + return rowKey; + } + + public Materialization getMat() { + return mat; + } + + public ImmutableAnalysis getAnalysis() { + return analysis; + } + + public WhereInfo getWhereInfo() { + return whereInfo; + } + + public QueryId getQueryId() { + return queryId; + } + + public QueryContext.Stacker getContextStacker() { + return contextStacker; + } + } + private static final class WhereInfo { private final Object rowkey; @@ -279,7 +405,7 @@ private Result( } } - private static WhereInfo extractWhereInfo( + private WhereInfo extractWhereInfo( final ImmutableAnalysis analysis, final PersistentQueryMetadata query ) { @@ -317,7 +443,7 @@ private static WhereInfo extractWhereInfo( return new WhereInfo(rowKey, Optional.of(windowStart)); } - private static Object extractRowKeyWhereClause( + private Object extractRowKeyWhereClause( final List comparisons, final boolean windowed, final LogicalSchema schema @@ -337,7 +463,7 @@ private static Object extractRowKeyWhereClause( return coerceRowKey(schema, right, windowed); } - private static Object coerceRowKey( + private Object coerceRowKey( final LogicalSchema schema, final Object right, final boolean windowed @@ -353,7 +479,7 @@ private static Object coerceRowKey( + "to the type of column ROWKEY: " + sqlType)); } - private static Range extractWhereClauseWindowBounds( + private Range extractWhereClauseWindowBounds( final Optional> maybeComparisons ) { if (!maybeComparisons.isPresent()) { @@ -363,7 +489,7 @@ private static Range extractWhereClauseWindowBounds( final List comparisons = maybeComparisons.get(); final Map> byType = comparisons.stream() - .collect(Collectors.groupingBy(PullQueryExecutor::getSimplifiedBoundType)); + .collect(Collectors.groupingBy(this::getSimplifiedBoundType)); final SetView unsupported = Sets.difference(byType.keySet(), VALID_WINDOW_BOUNDS_TYPES); if (!unsupported.isEmpty()) { @@ -410,7 +536,7 @@ private static Range extractWhereClauseWindowBounds( return extractWindowBound(lower, upper); } - private static Type getSimplifiedBoundType(final ComparisonExpression comparison) { + private Type getSimplifiedBoundType(final ComparisonExpression comparison) { final Type type = comparison.getType(); final boolean inverted = comparison.getRight() instanceof UnqualifiedColumnReferenceExp; @@ -426,7 +552,7 @@ private static Type getSimplifiedBoundType(final ComparisonExpression comparison } } - private static Range extractWindowBound( + private Range extractWindowBound( final Optional lowerComparison, final Optional upperComparison ) { @@ -455,7 +581,7 @@ private static Range extractWindowBound( return Range.range(lower, lowerType, upper, upperType); } - private static BoundType getRangeBoundType(final ComparisonExpression lowerComparison) { + private BoundType getRangeBoundType(final ComparisonExpression lowerComparison) { final boolean openBound = lowerComparison.getType() == Type.LESS_THAN || lowerComparison.getType() == Type.GREATER_THAN; @@ -464,13 +590,13 @@ private static BoundType getRangeBoundType(final ComparisonExpression lowerCompa : BoundType.CLOSED; } - private static Expression getNonColumnRefSide(final ComparisonExpression comparison) { + private Expression getNonColumnRefSide(final ComparisonExpression comparison) { return comparison.getRight() instanceof UnqualifiedColumnReferenceExp ? comparison.getLeft() : comparison.getRight(); } - private static Instant asInstant(final Expression other) { + private Instant asInstant(final Expression other) { if (other instanceof IntegerLiteral) { return Instant.ofEpochMilli(((IntegerLiteral) other).getValue()); } @@ -502,7 +628,7 @@ private enum ComparisonTarget { WINDOWSTART } - private static Map> extractComparisons( + private Map> extractComparisons( final Expression exp ) { if (exp instanceof ComparisonExpression) { @@ -533,7 +659,7 @@ private static Map> extractComparis throw invalidWhereClauseException("Unsupported expression: " + exp, false); } - private static ComparisonTarget extractWhereClauseTarget(final ComparisonExpression comparison) { + private ComparisonTarget extractWhereClauseTarget(final ComparisonExpression comparison) { final UnqualifiedColumnReferenceExp column; if (comparison.getRight() instanceof UnqualifiedColumnReferenceExp) { column = (UnqualifiedColumnReferenceExp) comparison.getRight(); @@ -552,12 +678,12 @@ private static ComparisonTarget extractWhereClauseTarget(final ComparisonExpress } } - private static boolean isSelectStar(final Select select) { + private boolean isSelectStar(final Select select) { final List selects = select.getSelectItems(); return selects.size() == 1 && selects.get(0) instanceof AllColumns; } - private static List> handleSelects( + private List> handleSelects( final Result input, final ConfiguredStatement statement, final KsqlExecutionContext executionContext, @@ -645,7 +771,7 @@ private static List> handleSelects( return output.build(); } - private static void validateProjection( + private void validateProjection( final GenericRow fullRow, final LogicalSchema schema ) { @@ -659,7 +785,7 @@ private static void validateProjection( } } - private static LogicalSchema selectOutputSchema( + private LogicalSchema selectOutputSchema( final Result input, final KsqlExecutionContext executionContext, final ImmutableAnalysis analysis, @@ -691,7 +817,7 @@ private static LogicalSchema selectOutputSchema( return schemaBuilder.build(); } - private static PersistentQueryMetadata findMaterializingQuery( + private PersistentQueryMetadata findMaterializingQuery( final KsqlExecutionContext executionContext, final ImmutableAnalysis analysis ) { @@ -715,35 +841,13 @@ private static PersistentQueryMetadata findMaterializingQuery( .orElseThrow(() -> new KsqlException("Materializing query has been stopped")); } - private static SourceName getSourceName(final ImmutableAnalysis analysis) { + private SourceName getSourceName(final ImmutableAnalysis analysis) { final DataSource source = analysis.getFromDataSources().get(0).getDataSource(); return source.getName(); } - private static KsqlNode getOwner( - final KsqlConfig ksqlConfig, - final Struct rowKey, - final Materialization mat - ) { - final Locator locator = mat.locator(); - - final long timeoutMs = - ksqlConfig.getLong(KsqlConfig.KSQL_QUERY_PULL_ROUTING_TIMEOUT_MS_CONFIG); - final long threshold = System.currentTimeMillis() + timeoutMs; - while (System.currentTimeMillis() < threshold) { - final Optional owner = locator.locate(rowKey); - if (owner.isPresent()) { - return owner.get(); - } - } - - throw new MaterializationTimeOutException( - "The owner of the key could not be determined within the configured timeout: " - + timeoutMs + "ms, config: " + KsqlConfig.KSQL_QUERY_PULL_ROUTING_TIMEOUT_MS_CONFIG - ); - } - - private static TableRowsEntity proxyTo( + @VisibleForTesting + TableRowsEntity forwardTo( final KsqlNode owner, final ConfiguredStatement statement, final ServiceContext serviceContext @@ -791,7 +895,7 @@ private static TableRowsEntity proxyTo( ); } - private static KsqlException notMaterializedException(final SourceName sourceTable) { + private KsqlException notMaterializedException(final SourceName sourceTable) { return new KsqlException("'" + sourceTable.toString(FormatOptions.noEscape()) + "' is not materialized. " + PullQueryValidator.NEW_QUERY_SYNTAX_SHORT_HELP @@ -804,7 +908,7 @@ private static KsqlException notMaterializedException(final SourceName sourceTab ); } - private static KsqlException invalidWhereClauseException( + private KsqlException invalidWhereClauseException( final String msg, final boolean windowed ) { @@ -833,7 +937,7 @@ private static KsqlException invalidWhereClauseException( ); } - private static Struct asKeyStruct(final Object rowKey, final PhysicalSchema physicalSchema) { + private Struct asKeyStruct(final Object rowKey, final PhysicalSchema physicalSchema) { final Struct key = new Struct(physicalSchema.keySchema().ksqlSchema()); key.put(SchemaUtil.ROWKEY_NAME.name(), rowKey); return key; @@ -853,4 +957,4 @@ public Optional visitQualifiedColumnReference( return Optional.of(new UnqualifiedColumnReferenceExp(node.getReference())); } } -} +} \ No newline at end of file diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ClusterStatusResource.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ClusterStatusResource.java index edbcd7709517..088e482455b3 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ClusterStatusResource.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ClusterStatusResource.java @@ -15,24 +15,32 @@ package io.confluent.ksql.rest.server.resources; +import static java.util.Objects.requireNonNull; + import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.engine.KsqlEngine; +import io.confluent.ksql.rest.entity.ActiveStandbyEntity; import io.confluent.ksql.rest.entity.ClusterStatusResponse; import io.confluent.ksql.rest.entity.HostStatusEntity; import io.confluent.ksql.rest.entity.HostStoreLags; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; +import io.confluent.ksql.rest.entity.TopicPartitionEntity; import io.confluent.ksql.rest.entity.Versions; import io.confluent.ksql.rest.server.HeartbeatAgent; import io.confluent.ksql.rest.server.LagReportingAgent; import io.confluent.ksql.util.HostStatus; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import javax.ws.rs.GET; import javax.ws.rs.Path; import javax.ws.rs.Produces; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import org.apache.kafka.streams.state.HostInfo; +import org.apache.kafka.streams.state.StreamsMetadata; /** * Endpoint that reports the view of the cluster that this server has. @@ -44,15 +52,19 @@ @Produces({Versions.KSQL_V1_JSON, MediaType.APPLICATION_JSON}) public class ClusterStatusResource { + private final KsqlEngine engine; private final HeartbeatAgent heartbeatAgent; private final Optional lagReportingAgent; private static final HostStoreLags EMPTY_HOST_STORE_LAGS = new HostStoreLags(ImmutableMap.of(), 0); - public ClusterStatusResource(final HeartbeatAgent heartbeatAgent, - final Optional lagReportingAgent) { - this.heartbeatAgent = heartbeatAgent; - this.lagReportingAgent = lagReportingAgent; + public ClusterStatusResource( + final KsqlEngine engine, + final HeartbeatAgent heartbeatAgent, + final Optional lagReportingAgent) { + this.engine = requireNonNull(engine, "engine"); + this.heartbeatAgent = requireNonNull(heartbeatAgent, "heartbeatAgent"); + this.lagReportingAgent = requireNonNull(lagReportingAgent, "lagReportingAgent"); } @GET @@ -62,24 +74,82 @@ public Response checkClusterStatus() { } private ClusterStatusResponse getResponse() { - final Map allHostStatus = heartbeatAgent.getHostsStatus(); + final Map allHostStatus = heartbeatAgent.getHostsStatus(); - final Map response = allHostStatus + final Map response = allHostStatus .entrySet() .stream() .collect(Collectors.toMap( - entry -> new KsqlHostEntity(entry.getKey().host(), entry.getKey().port()) , + entry -> new KsqlHostInfoEntity(entry.getKey().host(), entry.getKey().port()) , entry -> new HostStatusEntity(entry.getValue().isHostAlive(), entry.getValue().getLastStatusUpdateMs(), + getActiveStandbyInformation(entry.getKey()), getHostStoreLags(entry.getKey())) )); return new ClusterStatusResponse(response); } - private HostStoreLags getHostStoreLags(final KsqlHost ksqlHost) { + + private HostStoreLags getHostStoreLags(final KsqlHostInfo ksqlHostInfo) { return lagReportingAgent - .flatMap(agent -> agent.getLagPerHost(ksqlHost)) + .flatMap(agent -> agent.getLagPerHost(ksqlHostInfo)) .orElse(EMPTY_HOST_STORE_LAGS); } + + private Map getActiveStandbyInformation( + final KsqlHostInfo ksqlHostInfo + ) { + return engine.getPersistentQueries().stream() + .flatMap(persistentQueryMetadata -> persistentQueryMetadata.getAllMetadata() + .stream() + .map(streamsMetadata -> new QueryIdAndStreamsMetadata( + persistentQueryMetadata.getQueryId().toString(), streamsMetadata))) + .filter(queryIdAndStreamsMetadata -> + queryIdAndStreamsMetadata.streamsMetadata != StreamsMetadata.NOT_AVAILABLE) + .filter(queryIdAndStreamsMetadata -> + queryIdAndStreamsMetadata.streamsMetadata.hostInfo().equals(asHostInfo( + ksqlHostInfo))) + .collect(Collectors.toMap(queryIdAndStreamsMetadata -> + queryIdAndStreamsMetadata.queryId , + QueryIdAndStreamsMetadata::toActiveStandbyEntity)); + } + + private static final class QueryIdAndStreamsMetadata { + + final String queryId; + final StreamsMetadata streamsMetadata; + + QueryIdAndStreamsMetadata( + final String queryId, + final StreamsMetadata streamsMetadata + ) { + this.queryId = requireNonNull(queryId, "queryId"); + this.streamsMetadata = requireNonNull(streamsMetadata, "md"); + } + + public ActiveStandbyEntity toActiveStandbyEntity() { + final Set activePartitions = streamsMetadata.topicPartitions() + .stream() + .map(topicPartition -> new TopicPartitionEntity( + topicPartition.topic(), topicPartition.partition())) + .collect(Collectors.toSet()); + + final Set standByPartitions = streamsMetadata.standbyTopicPartitions() + .stream() + .map(topicPartition -> new TopicPartitionEntity( + topicPartition.topic(), topicPartition.partition())) + .collect(Collectors.toSet()); + + return new ActiveStandbyEntity( + streamsMetadata.stateStoreNames(), + activePartitions, + streamsMetadata.standbyStateStoreNames(), + standByPartitions); + } + } + + private HostInfo asHostInfo(final KsqlHostInfo ksqlHostInfo) { + return new HostInfo(ksqlHostInfo.host(), ksqlHostInfo.port()); + } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/HeartbeatResource.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/HeartbeatResource.java index dbabb2f6a065..7599d3b9abd0 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/HeartbeatResource.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/HeartbeatResource.java @@ -17,10 +17,10 @@ import io.confluent.ksql.rest.entity.HeartbeatMessage; import io.confluent.ksql.rest.entity.HeartbeatResponse; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.entity.Versions; import io.confluent.ksql.rest.server.HeartbeatAgent; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.Produces; @@ -49,9 +49,10 @@ public Response registerHeartbeat(final HeartbeatMessage request) { } private void handleHeartbeat(final HeartbeatMessage request) { - final KsqlHostEntity ksqlHostEntity = request.getHostInfo(); - final KsqlHost ksqlHost = new KsqlHost(ksqlHostEntity.getHost(), ksqlHostEntity.getPort()); + final KsqlHostInfoEntity ksqlHostInfoEntity = request.getHostInfo(); + final KsqlHostInfo ksqlHostInfo = new KsqlHostInfo( + ksqlHostInfoEntity.getHost(), ksqlHostInfoEntity.getPort()); final long timestamp = request.getTimestamp(); - heartbeatAgent.receiveHeartbeat(ksqlHost, timestamp); + heartbeatAgent.receiveHeartbeat(ksqlHostInfo, timestamp); } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisher.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisher.java index 32b17005d39d..55e3bc457dce 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisher.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisher.java @@ -20,8 +20,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import io.confluent.ksql.GenericRow; -import io.confluent.ksql.KsqlExecutionContext; -import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.parser.tree.Query; import io.confluent.ksql.rest.entity.StreamedRow; import io.confluent.ksql.rest.entity.TableRowsEntity; @@ -36,27 +34,16 @@ class PullQueryPublisher implements Flow.Publisher> { - private final KsqlEngine ksqlEngine; private final ServiceContext serviceContext; private final ConfiguredStatement query; - private final TheQueryExecutor pullQueryExecutor; - - PullQueryPublisher( - final KsqlEngine ksqlEngine, - final ServiceContext serviceContext, - final ConfiguredStatement query - ) { - this(ksqlEngine, serviceContext, query, PullQueryExecutor::execute); - } + private final PullQueryExecutor pullQueryExecutor; @VisibleForTesting PullQueryPublisher( - final KsqlEngine ksqlEngine, final ServiceContext serviceContext, final ConfiguredStatement query, - final TheQueryExecutor pullQueryExecutor + final PullQueryExecutor pullQueryExecutor ) { - this.ksqlEngine = requireNonNull(ksqlEngine, "ksqlEngine"); this.serviceContext = requireNonNull(serviceContext, "serviceContext"); this.query = requireNonNull(query, "query"); this.pullQueryExecutor = requireNonNull(pullQueryExecutor, "pullQueryExecutor"); @@ -66,7 +53,7 @@ class PullQueryPublisher implements Flow.Publisher> { public synchronized void subscribe(final Subscriber> subscriber) { final PullQuerySubscription subscription = new PullQuerySubscription( subscriber, - () -> pullQueryExecutor.execute(query, ksqlEngine, serviceContext) + () -> pullQueryExecutor.execute(query, serviceContext) ); subscriber.onSubscribe(subscription); @@ -121,13 +108,4 @@ private static GenericRow toGenericRow(final List values) { return new GenericRow().appendAll(values); } } - - interface TheQueryExecutor { - - TableRowsEntity execute( - ConfiguredStatement statement, - KsqlExecutionContext executionContext, - ServiceContext serviceContext - ); - } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java index 09bb237b8316..e0b64742218a 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java @@ -81,6 +81,7 @@ public class StreamedQueryResource implements KsqlConfigurable { private final Optional authorizationValidator; private final Errors errorHandler; private KsqlConfig ksqlConfig; + private final PullQueryExecutor pullQueryExecutor; public StreamedQueryResource( final KsqlEngine ksqlEngine, @@ -89,7 +90,8 @@ public StreamedQueryResource( final Duration commandQueueCatchupTimeout, final ActivenessRegistrar activenessRegistrar, final Optional authorizationValidator, - final Errors errorHandler + final Errors errorHandler, + final PullQueryExecutor pullQueryExecutor ) { this( ksqlEngine, @@ -99,12 +101,15 @@ public StreamedQueryResource( commandQueueCatchupTimeout, activenessRegistrar, authorizationValidator, - errorHandler + errorHandler, + pullQueryExecutor ); } @VisibleForTesting + // CHECKSTYLE_RULES.OFF: ParameterNumberCheck StreamedQueryResource( + // CHECKSTYLE_RULES.OFF: ParameterNumberCheck final KsqlEngine ksqlEngine, final StatementParser statementParser, final CommandQueue commandQueue, @@ -112,7 +117,8 @@ public StreamedQueryResource( final Duration commandQueueCatchupTimeout, final ActivenessRegistrar activenessRegistrar, final Optional authorizationValidator, - final Errors errorHandler + final Errors errorHandler, + final PullQueryExecutor pullQueryExecutor ) { this.ksqlEngine = Objects.requireNonNull(ksqlEngine, "ksqlEngine"); this.statementParser = Objects.requireNonNull(statementParser, "statementParser"); @@ -125,7 +131,8 @@ public StreamedQueryResource( this.activenessRegistrar = Objects.requireNonNull(activenessRegistrar, "activenessRegistrar"); this.authorizationValidator = authorizationValidator; - this.errorHandler = Objects.requireNonNull(errorHandler, "errorHandler");; + this.errorHandler = Objects.requireNonNull(errorHandler, "errorHandler"); + this.pullQueryExecutor = Objects.requireNonNull(pullQueryExecutor, "pullQueryExecutor"); } @Override @@ -230,10 +237,10 @@ private Response handlePullQuery( final Map streamsProperties ) { final ConfiguredStatement configured = - ConfiguredStatement.of(statement, streamsProperties, ksqlConfig); + ConfiguredStatement.of(statement,streamsProperties, ksqlConfig); - final TableRowsEntity entity = PullQueryExecutor - .execute(configured, ksqlEngine, serviceContext); + final TableRowsEntity entity = pullQueryExecutor + .execute(configured, serviceContext); final StreamedRow header = StreamedRow.header(entity.getQueryId(), entity.getSchema()); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpoint.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpoint.java index 0b380f96abba..3ecb6c0a1eed 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpoint.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpoint.java @@ -31,6 +31,7 @@ import io.confluent.ksql.rest.entity.Versions; import io.confluent.ksql.rest.server.StatementParser; import io.confluent.ksql.rest.server.computation.CommandQueue; +import io.confluent.ksql.rest.server.execution.PullQueryExecutor; import io.confluent.ksql.rest.server.services.RestServiceContextFactory; import io.confluent.ksql.rest.server.services.RestServiceContextFactory.DefaultServiceContextFactory; import io.confluent.ksql.rest.server.services.RestServiceContextFactory.UserServiceContextFactory; @@ -90,7 +91,7 @@ public class WSQueryEndpoint { private final ListeningScheduledExecutorService exec; private final ActivenessRegistrar activenessRegistrar; private final QueryPublisher pushQueryPublisher; - private final QueryPublisher pullQueryPublisher; + private final IPullQueryPublisher pullQueryPublisher; private final PrintTopicPublisher topicPublisher; private final Duration commandQueueCatchupTimeout; private final Optional authorizationValidator; @@ -100,6 +101,7 @@ public class WSQueryEndpoint { private final ServerState serverState; private final Errors errorHandler; private final Supplier schemaRegistryClientFactory; + private final PullQueryExecutor pullQueryExecutor; private WebSocketSubscriber subscriber; private KsqlSecurityContext securityContext; @@ -119,7 +121,8 @@ public WSQueryEndpoint( final Errors errorHandler, final KsqlSecurityExtension securityExtension, final ServerState serverState, - final Supplier schemaRegistryClientFactory + final Supplier schemaRegistryClientFactory, + final PullQueryExecutor pullQueryExecutor ) { this(ksqlConfig, mapper, @@ -138,7 +141,9 @@ public WSQueryEndpoint( RestServiceContextFactory::create, RestServiceContextFactory::create, serverState, - schemaRegistryClientFactory); + schemaRegistryClientFactory, + pullQueryExecutor + ); } // CHECKSTYLE_RULES.OFF: ParameterNumberCheck @@ -151,7 +156,7 @@ public WSQueryEndpoint( final CommandQueue commandQueue, final ListeningScheduledExecutorService exec, final QueryPublisher pushQueryPublisher, - final QueryPublisher pullQueryPublisher, + final IPullQueryPublisher pullQueryPublisher, final PrintTopicPublisher topicPublisher, final ActivenessRegistrar activenessRegistrar, final Duration commandQueueCatchupTimeout, @@ -161,7 +166,8 @@ public WSQueryEndpoint( final UserServiceContextFactory serviceContextFactory, final DefaultServiceContextFactory defaultServiceContextFactory, final ServerState serverState, - final Supplier schemaRegistryClientFactory + final Supplier schemaRegistryClientFactory, + final PullQueryExecutor pullQueryExecutor ) { this.ksqlConfig = Objects.requireNonNull(ksqlConfig, "ksqlConfig"); this.mapper = Objects.requireNonNull(mapper, "mapper"); @@ -188,6 +194,7 @@ public WSQueryEndpoint( this.errorHandler = Objects.requireNonNull(errorHandler, "errorHandler"); this.schemaRegistryClientFactory = Objects.requireNonNull(schemaRegistryClientFactory, "schemaRegistryClientFactory"); + this.pullQueryExecutor = Objects.requireNonNull(pullQueryExecutor, "pullQueryExecutor"); } @SuppressWarnings("unused") @@ -392,17 +399,24 @@ private void handleQuery(final RequestContext info, final Query query) { final ConfiguredStatement configured = ConfiguredStatement.of(statement, clientLocalProperties, ksqlConfig); - final QueryPublisher queryPublisher = query.isPullQuery() - ? pullQueryPublisher - : pushQueryPublisher; - - queryPublisher.start( - ksqlEngine, - info.securityContext.getServiceContext(), - exec, - configured, - streamSubscriber - ); + if (query.isPullQuery()) { + pullQueryPublisher.start( + ksqlEngine, + info.securityContext.getServiceContext(), + exec, + configured, + streamSubscriber, + pullQueryExecutor + ); + } else { + pushQueryPublisher.start( + ksqlEngine, + info.securityContext.getServiceContext(), + exec, + configured, + streamSubscriber + ); + } } private void handlePrintTopic(final RequestContext info, final PrintTopic printTopic) { @@ -453,10 +467,15 @@ private static void startPullQueryPublisher( final ServiceContext serviceContext, final ListeningScheduledExecutorService ignored, final ConfiguredStatement query, - final WebSocketSubscriber streamSubscriber + final WebSocketSubscriber streamSubscriber, + final PullQueryExecutor pullQueryExecutor + ) { - new PullQueryPublisher(ksqlEngine, serviceContext, query) - .subscribe(streamSubscriber); + new PullQueryPublisher( + serviceContext, + query, + pullQueryExecutor + ).subscribe(streamSubscriber); } private static void startPrintPublisher( @@ -481,6 +500,19 @@ void start( } + interface IPullQueryPublisher { + + void start( + KsqlEngine ksqlEngine, + ServiceContext serviceContext, + ListeningScheduledExecutorService exec, + ConfiguredStatement query, + WebSocketSubscriber subscriber, + PullQueryExecutor pullQueryExecutor); + + } + + interface PrintTopicPublisher { void start( diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/DefaultKsqlClient.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/DefaultKsqlClient.java index 2363547a47d4..9748b128f5d6 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/DefaultKsqlClient.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/DefaultKsqlClient.java @@ -28,11 +28,11 @@ import io.confluent.ksql.rest.client.RestResponse; import io.confluent.ksql.rest.entity.ClusterStatusResponse; import io.confluent.ksql.rest.entity.KsqlEntityList; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.entity.LagReportingMessage; import io.confluent.ksql.rest.entity.StreamedRow; import io.confluent.ksql.services.SimpleKsqlClient; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import java.net.URI; import java.util.List; import java.util.Optional; @@ -70,10 +70,7 @@ public RestResponse makeKsqlRequest( final KsqlTarget target = sharedClient .target(serverEndPoint); - return authHeader - .map(target::authorizationHeader) - .orElse(target) - .postKsqlRequest(sql, Optional.empty()); + return getTarget(target, authHeader).postKsqlRequest(sql, Optional.empty()); } @Override @@ -84,9 +81,7 @@ public RestResponse> makeQueryRequest( final KsqlTarget target = sharedClient .target(serverEndPoint); - final RestResponse resp = authHeader - .map(target::authorizationHeader) - .orElse(target) + final RestResponse resp = getTarget(target, authHeader) .postQueryRequest(sql, Optional.empty()); if (resp.isErroneous()) { @@ -106,15 +101,13 @@ public RestResponse> makeQueryRequest( @Override public void makeAsyncHeartbeatRequest( final URI serverEndPoint, - final KsqlHost host, + final KsqlHostInfo host, final long timestamp) { final KsqlTarget target = sharedClient .target(serverEndPoint); - authHeader - .map(target::authorizationHeader) - .orElse(target) - .postAsyncHeartbeatRequest(new KsqlHostEntity(host.host(), host.port()), timestamp); + getTarget(target, authHeader) + .postAsyncHeartbeatRequest(new KsqlHostInfoEntity(host.host(), host.port()), timestamp); } @Override @@ -122,10 +115,7 @@ public RestResponse makeClusterStatusRequest(final URI se final KsqlTarget target = sharedClient .target(serverEndPoint); - return authHeader - .map(target::authorizationHeader) - .orElse(target) - .getClusterStatus(); + return getTarget(target, authHeader).getClusterStatus(); } @Override @@ -136,9 +126,12 @@ public void makeAsyncLagReportRequest( final KsqlTarget target = sharedClient .target(serverEndPoint); - authHeader + getTarget(target, authHeader).postAsyncLagReportingRequest(lagReportingMessage); + } + + private KsqlTarget getTarget(final KsqlTarget target, final Optional authHeader) { + return authHeader .map(target::authorizationHeader) - .orElse(target) - .postAsyncLagReportingRequest(lagReportingMessage); + .orElse(target); } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/ServerInternalKsqlClient.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/ServerInternalKsqlClient.java index 9e5266d659a3..f4ff3f801d02 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/ServerInternalKsqlClient.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/ServerInternalKsqlClient.java @@ -27,7 +27,7 @@ import io.confluent.ksql.rest.server.resources.KsqlResource; import io.confluent.ksql.security.KsqlSecurityContext; import io.confluent.ksql.services.SimpleKsqlClient; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import java.net.URI; import java.util.Collections; import java.util.List; @@ -78,7 +78,7 @@ public RestResponse> makeQueryRequest( @Override public void makeAsyncHeartbeatRequest( final URI serverEndPoint, - final KsqlHost host, + final KsqlHostInfo host, final long timestamp ) { throw new UnsupportedOperationException(); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/HeartbeatAgentFunctionalTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/HeartbeatAgentFunctionalTest.java index da41841e41b1..b1b2fc692426 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/HeartbeatAgentFunctionalTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/HeartbeatAgentFunctionalTest.java @@ -15,6 +15,9 @@ package io.confluent.ksql.rest.integration; +import static io.confluent.ksql.rest.integration.HighAvailabilityTestUtil.sendHeartbeartsForWindowLength; +import static io.confluent.ksql.rest.integration.HighAvailabilityTestUtil.waitForClusterToBeDiscovered; +import static io.confluent.ksql.rest.integration.HighAvailabilityTestUtil.waitForRemoteServerToChangeStatus; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; @@ -22,7 +25,7 @@ import io.confluent.ksql.integration.IntegrationTestHarness; import io.confluent.ksql.integration.Retry; import io.confluent.ksql.rest.entity.ClusterStatusResponse; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.server.KsqlRestConfig; import io.confluent.ksql.rest.server.TestKsqlRestApp; import io.confluent.ksql.serde.FormatFactory; @@ -45,9 +48,8 @@ public class HeartbeatAgentFunctionalTest { 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.kstreamName(); - - private static final KsqlHostEntity host0 = new KsqlHostEntity("localhost",8088); - private static final KsqlHostEntity host1 = new KsqlHostEntity("localhost",8089); + private static final KsqlHostInfoEntity host0 = new KsqlHostInfoEntity("localhost", 8088); + private static final KsqlHostInfoEntity host1 = new KsqlHostInfoEntity("localhost", 8089); private static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); private static final TestKsqlRestApp REST_APP_0 = TestKsqlRestApp .builder(TEST_HARNESS::kafkaBootstrapServers) @@ -105,13 +107,13 @@ public void tearDown() { @Test(timeout = 60000) public void shouldMarkServersAsUp() { // Given: - HighAvailabilityTestUtil.waitForClusterToBeDiscovered(REST_APP_0, 2); - HighAvailabilityTestUtil.waitForRemoteServerToChangeStatus( + waitForClusterToBeDiscovered(REST_APP_0, 2); + waitForRemoteServerToChangeStatus( REST_APP_0, host1, HighAvailabilityTestUtil::remoteServerIsDown); // When: - HighAvailabilityTestUtil.sendHeartbeartsForWindowLength(REST_APP_0, host1, 3000); - final ClusterStatusResponse clusterStatusResponseUp = HighAvailabilityTestUtil.waitForRemoteServerToChangeStatus( + sendHeartbeartsForWindowLength(REST_APP_0, host1, 3000); + final ClusterStatusResponse clusterStatusResponseUp = waitForRemoteServerToChangeStatus( REST_APP_0, host1, HighAvailabilityTestUtil::remoteServerIsUp); // Then: @@ -122,10 +124,10 @@ public void shouldMarkServersAsUp() { @Test(timeout = 60000) public void shouldMarkRemoteServerAsDown() { // Given: - HighAvailabilityTestUtil.waitForClusterToBeDiscovered(REST_APP_0, 2); + waitForClusterToBeDiscovered(REST_APP_0, 2); // When: - ClusterStatusResponse clusterStatusResponse = HighAvailabilityTestUtil.waitForRemoteServerToChangeStatus( + ClusterStatusResponse clusterStatusResponse = waitForRemoteServerToChangeStatus( REST_APP_0, host1, HighAvailabilityTestUtil::remoteServerIsDown); // Then: @@ -136,11 +138,11 @@ public void shouldMarkRemoteServerAsDown() { @Test(timeout = 60000) public void shouldMarkRemoteServerAsUpThenDownThenUp() { // Given: - HighAvailabilityTestUtil.waitForClusterToBeDiscovered(REST_APP_0, 2); - HighAvailabilityTestUtil.sendHeartbeartsForWindowLength(REST_APP_0, host1, 3000); + waitForClusterToBeDiscovered(REST_APP_0, 2); + sendHeartbeartsForWindowLength(REST_APP_0, host1, 3000); // When: - final ClusterStatusResponse clusterStatusResponseUp1 = HighAvailabilityTestUtil.waitForRemoteServerToChangeStatus( + final ClusterStatusResponse clusterStatusResponseUp1 = waitForRemoteServerToChangeStatus( REST_APP_0, host1, HighAvailabilityTestUtil::remoteServerIsUp); // Then: @@ -148,7 +150,7 @@ public void shouldMarkRemoteServerAsUpThenDownThenUp() { assertThat(clusterStatusResponseUp1.getClusterStatus().get(host1).getHostAlive(), is(true)); // When: - ClusterStatusResponse clusterStatusResponseDown = HighAvailabilityTestUtil.waitForRemoteServerToChangeStatus( + ClusterStatusResponse clusterStatusResponseDown = waitForRemoteServerToChangeStatus( REST_APP_0, host1, HighAvailabilityTestUtil::remoteServerIsDown); // Then: @@ -156,8 +158,8 @@ public void shouldMarkRemoteServerAsUpThenDownThenUp() { assertThat(clusterStatusResponseDown.getClusterStatus().get(host1).getHostAlive(), is(false)); // When : - HighAvailabilityTestUtil.sendHeartbeartsForWindowLength(REST_APP_0, host1, 3000); - ClusterStatusResponse clusterStatusResponseUp2 = HighAvailabilityTestUtil.waitForRemoteServerToChangeStatus( + sendHeartbeartsForWindowLength(REST_APP_0, host1, 3000); + ClusterStatusResponse clusterStatusResponseUp2 = waitForRemoteServerToChangeStatus( REST_APP_0, host1, HighAvailabilityTestUtil::remoteServerIsUp); // Then: diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/HighAvailabilityTestUtil.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/HighAvailabilityTestUtil.java index 4fbd1399bdec..e0d407a3df5f 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/HighAvailabilityTestUtil.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/HighAvailabilityTestUtil.java @@ -19,31 +19,31 @@ import io.confluent.ksql.rest.client.RestResponse; import io.confluent.ksql.rest.entity.ClusterStatusResponse; import io.confluent.ksql.rest.entity.HostStatusEntity; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.server.TestKsqlRestApp; +import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.function.BiFunction; +import java.util.stream.Collectors; class HighAvailabilityTestUtil { static ClusterStatusResponse sendClusterStatusRequest(final TestKsqlRestApp restApp) { - try (final KsqlRestClient restClient = restApp.buildKsqlClient()) { - final RestResponse res = restClient.makeClusterStatusRequest(); if (res.isErroneous()) { throw new AssertionError("Erroneous result: " + res.getErrorMessage()); } - return res.getResponse(); } } static void sendHeartbeartsForWindowLength( final TestKsqlRestApp receiverApp, - final KsqlHostEntity sender, + final KsqlHostInfoEntity sender, final long window ) { long start = System.currentTimeMillis(); @@ -59,8 +59,8 @@ static void sendHeartbeartsForWindowLength( static ClusterStatusResponse waitForRemoteServerToChangeStatus( final TestKsqlRestApp restApp, - final KsqlHostEntity remoteServer, - final BiFunction, Boolean> function + final KsqlHostInfoEntity remoteServer, + final BiFunction, Boolean> function ) { while (true) { final ClusterStatusResponse clusterStatusResponse = sendClusterStatusRequest(restApp); @@ -91,14 +91,38 @@ static void waitForClusterToBeDiscovered( } } + static void waitForStreamsMetadataToInitialize( + final TestKsqlRestApp restApp, List hosts, String queryId + ) { + while (true) { + ClusterStatusResponse clusterStatusResponse = HighAvailabilityTestUtil.sendClusterStatusRequest(restApp); + List initialized = hosts.stream() + .filter(hostInfo -> Optional.ofNullable( + clusterStatusResponse + .getClusterStatus() + .get(hostInfo)) + .map(hostStatusEntity -> hostStatusEntity + .getActiveStandbyPerQuery() + .isEmpty()).isPresent()) + .collect(Collectors.toList()); + if(initialized.size() == hosts.size()) + break; + } + try { + Thread.sleep(200); + } catch (final Exception e) { + // Meh + } + } + static boolean remoteServerIsDown( - final KsqlHostEntity remoteServer, - final Map clusterStatus + final KsqlHostInfoEntity remoteServer, + final Map clusterStatus ) { if (!clusterStatus.containsKey(remoteServer)) { return true; } - for( Entry entry: clusterStatus.entrySet()) { + for( Entry entry: clusterStatus.entrySet()) { if (entry.getKey().getPort() == remoteServer.getPort() && !entry.getValue().getHostAlive()) { return true; @@ -108,10 +132,10 @@ static boolean remoteServerIsDown( } static boolean remoteServerIsUp( - final KsqlHostEntity remoteServer, - final Map clusterStatus + final KsqlHostInfoEntity remoteServer, + final Map clusterStatus ) { - for( Entry entry: clusterStatus.entrySet()) { + for( Entry entry: clusterStatus.entrySet()) { if (entry.getKey().getPort() == remoteServer.getPort() && entry.getValue().getHostAlive()) { return true; @@ -122,15 +146,14 @@ static boolean remoteServerIsUp( private static boolean allServersDiscovered( final int numServers, - final Map clusterStatus + final Map clusterStatus ) { - return clusterStatus.size() >= numServers; } private static void sendHeartbeatRequest( final TestKsqlRestApp restApp, - final KsqlHostEntity hostInfoEntity, + final KsqlHostInfoEntity hostInfoEntity, final long timestamp ) { @@ -138,4 +161,6 @@ private static void sendHeartbeatRequest( restClient.makeAsyncHeartbeatRequest(hostInfoEntity, timestamp); } } -} \ No newline at end of file + +} + diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/LagReportingAgentFunctionalTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/LagReportingAgentFunctionalTest.java index 4ae86fc71617..089487c0cd75 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/LagReportingAgentFunctionalTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/LagReportingAgentFunctionalTest.java @@ -10,7 +10,7 @@ import io.confluent.ksql.rest.client.RestResponse; import io.confluent.ksql.rest.entity.ClusterStatusResponse; import io.confluent.ksql.rest.entity.HostStoreLags; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.entity.LagInfoEntity; import io.confluent.ksql.rest.entity.QueryStateStoreId; import io.confluent.ksql.rest.entity.StateStoreLags; @@ -64,8 +64,8 @@ public class LagReportingAgentFunctionalTest { "_confluent-ksql-default_query_CTAS_USER_LATEST_VIEWTIME_5", "Aggregate-Aggregate-Materialize"); - private static final KsqlHostEntity HOST0 = new KsqlHostEntity("localhost", 8088); - private static final KsqlHostEntity HOST1 = new KsqlHostEntity("localhost", 8089); + private static final KsqlHostInfoEntity HOST0 = new KsqlHostInfoEntity("localhost", 8088); + private static final KsqlHostInfoEntity HOST1 = new KsqlHostInfoEntity("localhost", 8089); private static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); private static final TestKsqlRestApp REST_APP_0 = TestKsqlRestApp .builder(TEST_HARNESS::kafkaBootstrapServers) diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/PullQueryRoutingFunctionalTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/PullQueryRoutingFunctionalTest.java new file mode 100644 index 000000000000..12abae485be1 --- /dev/null +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/PullQueryRoutingFunctionalTest.java @@ -0,0 +1,402 @@ +/* + * 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.rest.integration; + +import static io.confluent.ksql.rest.integration.HighAvailabilityTestUtil.sendHeartbeartsForWindowLength; +import static io.confluent.ksql.rest.integration.HighAvailabilityTestUtil.waitForClusterToBeDiscovered; +import static io.confluent.ksql.rest.integration.HighAvailabilityTestUtil.waitForRemoteServerToChangeStatus; +import static io.confluent.ksql.rest.integration.HighAvailabilityTestUtil.waitForStreamsMetadataToInitialize; +import static io.confluent.ksql.util.KsqlConfig.KSQL_STREAMS_PREFIX; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import io.confluent.common.utils.IntegrationTest; +import io.confluent.ksql.integration.IntegrationTestHarness; +import io.confluent.ksql.integration.Retry; +import io.confluent.ksql.name.ColumnName; +import io.confluent.ksql.rest.entity.ActiveStandbyEntity; +import io.confluent.ksql.rest.entity.ClusterStatusResponse; +import io.confluent.ksql.rest.entity.KsqlEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; +import io.confluent.ksql.rest.entity.StreamedRow; +import io.confluent.ksql.rest.server.KsqlRestConfig; +import io.confluent.ksql.rest.server.TestKsqlRestApp; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.PhysicalSchema; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.serde.Format; +import io.confluent.ksql.serde.SerdeOption; +import io.confluent.ksql.test.util.KsqlIdentifierTestUtil; +import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.Pair; +import io.confluent.ksql.util.UserDataProvider; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Pattern; +import kafka.zookeeper.ZooKeeperClientException; +import org.apache.kafka.streams.StreamsConfig; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.rules.RuleChain; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; + +/** + * Test to ensure pull queries route across multiple KSQL nodes correctly. + * + *

For tests on general syntax and handled see RestQueryTranslationTest's + * materialized-aggregate-static-queries.json + */ +@SuppressWarnings("OptionalGetWithoutIsPresent") +@Category({IntegrationTest.class}) +public class PullQueryRoutingFunctionalTest { + + private static final TemporaryFolder TMP = new TemporaryFolder(); + + static { + try { + TMP.create(); + } catch (final IOException e) { + throw new AssertionError("Failed to init TMP", e); + } + } + + private static final Pattern QUERY_ID_PATTERN = Pattern.compile("with query ID: (\\S+)"); + private static final KsqlHostInfoEntity host0 = new KsqlHostInfoEntity("localhost", 8088); + private static final KsqlHostInfoEntity host1 = new KsqlHostInfoEntity("localhost", 8089); + private static final KsqlHostInfoEntity host2 = new KsqlHostInfoEntity("localhost", 8087); + private static final String USER_TOPIC = "user_topic"; + private static final String USERS_STREAM = "users"; + private static final UserDataProvider USER_PROVIDER = new UserDataProvider(); + private static final Format VALUE_FORMAT = Format.JSON; + private static final int HEADER = 1; + private static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); + private static final int BASE_TIME = 1_000_000; + private final static String KEY = Iterables.get(USER_PROVIDER.data().keySet(), 0); + private String output; + private String QUERY_ID; + private String sql; + + private static final PhysicalSchema AGGREGATE_SCHEMA = PhysicalSchema.from( + LogicalSchema.builder() + .valueColumn(ColumnName.of("COUNT"), SqlTypes.BIGINT) + .build(), + SerdeOption.none() + ); + + private static final Map COMMON_CONFIG = ImmutableMap.builder() + .put(KSQL_STREAMS_PREFIX + StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1) + .put(KsqlRestConfig.KSQL_HEARTBEAT_ENABLE_CONFIG, true) + .put(KsqlRestConfig.KSQL_HEARTBEAT_SEND_INTERVAL_MS_CONFIG, 600000) + .put(KsqlRestConfig.KSQL_HEARTBEAT_CHECK_INTERVAL_MS_CONFIG, 200) + .put(KsqlRestConfig.KSQL_HEARTBEAT_DISCOVER_CLUSTER_MS_CONFIG, 2000) + .put(KsqlConfig.KSQL_QUERY_PULL_ENABLE_STANDBY_READS, true) + .put(KsqlConfig.KSQL_STREAMS_PREFIX + "num.standby.replicas", 1) + .put(KsqlConfig.KSQL_SHUTDOWN_TIMEOUT_MS_CONFIG, 1000) + .build(); + + @Rule + public final TestKsqlRestApp REST_APP_0 = TestKsqlRestApp + .builder(TEST_HARNESS::kafkaBootstrapServers) + .withProperty(KSQL_STREAMS_PREFIX + StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1) + .withProperty(KSQL_STREAMS_PREFIX + StreamsConfig.STATE_DIR_CONFIG, getNewStateDir()) + .withProperty(KsqlRestConfig.LISTENERS_CONFIG, "http://localhost:8088") + .withProperty(KsqlRestConfig.ADVERTISED_LISTENER_CONFIG, "http://localhost:8088") + .withProperties(COMMON_CONFIG) + .build(); + + @Rule + public final TestKsqlRestApp REST_APP_1 = TestKsqlRestApp + .builder(TEST_HARNESS::kafkaBootstrapServers) + .withProperty(KSQL_STREAMS_PREFIX + StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1) + .withProperty(KSQL_STREAMS_PREFIX + StreamsConfig.STATE_DIR_CONFIG, getNewStateDir()) + .withProperty(KsqlRestConfig.LISTENERS_CONFIG, "http://localhost:8089") + .withProperty(KsqlRestConfig.ADVERTISED_LISTENER_CONFIG, "http://localhost:8089") + .withProperties(COMMON_CONFIG) + .build(); + + @Rule + public final TestKsqlRestApp REST_APP_2 = TestKsqlRestApp + .builder(TEST_HARNESS::kafkaBootstrapServers) + .withProperty(KSQL_STREAMS_PREFIX + StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1) + .withProperty(KSQL_STREAMS_PREFIX + StreamsConfig.STATE_DIR_CONFIG, getNewStateDir()) + .withProperty(KsqlRestConfig.LISTENERS_CONFIG, "http://localhost:8087") + .withProperty(KsqlRestConfig.ADVERTISED_LISTENER_CONFIG, "http://localhost:8087") + .withProperties(COMMON_CONFIG) + .build(); + + @ClassRule + public static final RuleChain CHAIN = RuleChain + .outerRule(Retry.of(3, ZooKeeperClientException.class, 3, TimeUnit.SECONDS)) + .around(TEST_HARNESS); + + @Rule + public final Timeout timeout = Timeout.builder() + .withTimeout(1, TimeUnit.MINUTES) + .withLookingForStuckThread(true) + .build(); + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @BeforeClass + public static void setUpClass() { + //Create topic with 1 partition to control who is active and standby + TEST_HARNESS.ensureTopics(1, USER_TOPIC); + + final AtomicLong timestampSupplier = new AtomicLong(BASE_TIME); + + TEST_HARNESS.produceRows( + USER_TOPIC, + USER_PROVIDER, + VALUE_FORMAT, + timestampSupplier::getAndIncrement + ); + } + + @Before + public void setUp() { + //Create stream + makeAdminRequest( + REST_APP_0, + "CREATE STREAM " + USERS_STREAM + + " (" + USER_PROVIDER.ksqlSchemaString() + ")" + + " WITH (" + + " kafka_topic='" + USER_TOPIC + "', " + + " key='" + USER_PROVIDER.key() + "', " + + " value_format='" + VALUE_FORMAT + "'" + + ");" + ); + //Create table + output = KsqlIdentifierTestUtil.uniqueIdentifierName(); + sql = "SELECT * FROM " + output + " WHERE ROWKEY = '" + KEY + "';"; + List res = makeAdminRequestWithResponse( + REST_APP_0, + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(1) AS COUNT FROM " + USERS_STREAM + + " GROUP BY " + USER_PROVIDER.key() + ";" + ); + QUERY_ID = extractQueryId(res.get(0).toString()); + QUERY_ID = QUERY_ID.substring(0, QUERY_ID.length()-1); + waitForTableRows(); + waitForStreamsMetadataToInitialize( + REST_APP_0, ImmutableList.of(host0, host1, host2), QUERY_ID); + } + + @After + public void cleanUp() { + REST_APP_0.closePersistentQueries(); + REST_APP_0.dropSourcesExcept(); + } + + @AfterClass + public static void classTearDown() { + TMP.delete(); + } + + @Test + public void shouldQueryActiveWhenActiveAliveQueryIssuedToStandby() { + // Given: + ClusterFormation clusterFormation = findClusterFormation(REST_APP_0, REST_APP_1, REST_APP_2); + waitForClusterToBeDiscovered(clusterFormation.standBy.right, 3); + sendHeartbeartsForWindowLength( + clusterFormation.standBy.right, clusterFormation.active.left, 2000); + waitForRemoteServerToChangeStatus( + clusterFormation.standBy.right, clusterFormation.active.left, HighAvailabilityTestUtil::remoteServerIsUp); + + // When: + final List rows_0 = makePullQueryRequest(clusterFormation.standBy.right, sql); + + // Then: + assertThat(rows_0, hasSize(HEADER + 1)); + assertThat(rows_0.get(1).getRow(), is(not(Optional.empty()))); + assertThat(rows_0.get(1).getRow().get().values(), is(ImmutableList.of(KEY, BASE_TIME, 1))); + } + + + @Test + public void shouldQueryActiveWhenActiveAliveStandbyDeadQueryIssuedToRouter() { + // Given: + ClusterFormation clusterFormation = findClusterFormation(REST_APP_0, REST_APP_1, REST_APP_2); + waitForClusterToBeDiscovered(clusterFormation.router.right, 3); + sendHeartbeartsForWindowLength( + clusterFormation.router.right, clusterFormation.active.left, 2000); + waitForRemoteServerToChangeStatus( + clusterFormation.router.right, + clusterFormation.active.left, + HighAvailabilityTestUtil::remoteServerIsUp); + waitForRemoteServerToChangeStatus( + clusterFormation.router.right, + clusterFormation.standBy.left, + HighAvailabilityTestUtil::remoteServerIsDown); + + // When: + final List rows_0 = makePullQueryRequest(clusterFormation.router.right, sql); + + // Then: + assertThat(rows_0, hasSize(HEADER + 1)); + assertThat(rows_0.get(1).getRow(), is(not(Optional.empty()))); + assertThat(rows_0.get(1).getRow().get().values(), is(ImmutableList.of(KEY, BASE_TIME, 1))); + } + + @Test + public void shouldQueryStandbyWhenActiveDeadStandbyAliveQueryIssuedToRouter() { + // Given: + ClusterFormation clusterFormation = findClusterFormation(REST_APP_0, REST_APP_1, REST_APP_2); + waitForClusterToBeDiscovered(clusterFormation.router.right, 3); + sendHeartbeartsForWindowLength( + clusterFormation.router.right, clusterFormation.standBy.left, 2000); + waitForRemoteServerToChangeStatus( + clusterFormation.router.right, + clusterFormation.standBy.left, + HighAvailabilityTestUtil::remoteServerIsUp); + + // When: + final List rows_0 = makePullQueryRequest(clusterFormation.router.right, sql); + + // Then: + assertThat(rows_0, hasSize(HEADER + 1)); + assertThat(rows_0.get(1).getRow(), is(not(Optional.empty()))); + assertThat(rows_0.get(1).getRow().get().values(), is(ImmutableList.of(KEY, BASE_TIME, 1))); + } + + private List makePullQueryRequest( + final TestKsqlRestApp target, + final String sql + ) { + return RestIntegrationTestUtil.makeQueryRequest(target, sql, Optional.empty()); + } + + private static void makeAdminRequest(TestKsqlRestApp restApp, final String sql) { + RestIntegrationTestUtil.makeKsqlRequest(restApp, sql, Optional.empty()); + } + + private List makeAdminRequestWithResponse( + TestKsqlRestApp restApp, final String sql) { + return RestIntegrationTestUtil.makeKsqlRequest(restApp, sql, Optional.empty()); + } + + private ClusterFormation findClusterFormation( + TestKsqlRestApp restApp0, TestKsqlRestApp restApp1, TestKsqlRestApp restApp2) { + ClusterFormation clusterFormation = new ClusterFormation(); + ClusterStatusResponse clusterStatusResponse = HighAvailabilityTestUtil.sendClusterStatusRequest(restApp0); + ActiveStandbyEntity entity0 = clusterStatusResponse.getClusterStatus().get(host0) + .getActiveStandbyPerQuery().get(QUERY_ID); + ActiveStandbyEntity entity1 = clusterStatusResponse.getClusterStatus().get(host1) + .getActiveStandbyPerQuery().get(QUERY_ID); + + // find active + if(!entity0.getActiveStores().isEmpty() && !entity0.getActivePartitions().isEmpty()) { + clusterFormation.setActive(Pair.of(host0, restApp0)); + } + else if(!entity1.getActiveStores().isEmpty() && !entity1.getActivePartitions().isEmpty()) { + clusterFormation.setActive(Pair.of(host1, restApp1)); + } else { + clusterFormation.setActive(Pair.of(host2, restApp2)); + } + + //find standby + if(!entity0.getStandByStores().isEmpty() && !entity0.getStandByPartitions().isEmpty()) { + clusterFormation.setStandBy(Pair.of(host0, restApp0)); + } + else if(!entity1.getStandByStores().isEmpty() && !entity1.getStandByPartitions().isEmpty()) { + clusterFormation.setStandBy(Pair.of(host1, restApp1)); + } else { + clusterFormation.setStandBy(Pair.of(host2, restApp2)); + } + + //find router + if(entity0.getStandByStores().isEmpty() && entity0.getActiveStores().isEmpty()) { + clusterFormation.setRouter(Pair.of(host0, restApp0)); + } + else if(entity1.getStandByStores().isEmpty() && entity1.getActiveStores().isEmpty()) { + clusterFormation.setRouter(Pair.of(host1, restApp1)); + } else { + clusterFormation.setRouter(Pair.of(host2, restApp2)); + } + + return clusterFormation; + } + + static class ClusterFormation { + Pair active; + Pair standBy; + Pair router; + + ClusterFormation() { + } + + public void setActive(final Pair active) { + this.active = active; + } + + public void setStandBy(final Pair standBy) { + this.standBy = standBy; + } + + public void setRouter(final Pair router) { + this.router = router; + } + + public String toString() { + return new StringBuilder() + .append("Active = ").append(active.left) + .append(", Standby = ").append(standBy.left) + .append(", Router = ").append(router.left) + .toString(); + } + } + + private void waitForTableRows() { + TEST_HARNESS.verifyAvailableUniqueRows( + output.toUpperCase(), + USER_PROVIDER.data().size(), + VALUE_FORMAT, + AGGREGATE_SCHEMA + ); + } + + private String getNewStateDir() { + try { + return TMP.newFolder().getAbsolutePath(); + } catch (final IOException e) { + throw new AssertionError("Failed to create new state dir", e); + } + } + + private String extractQueryId(final String outputString) { + final java.util.regex.Matcher matcher = QUERY_ID_PATTERN.matcher(outputString); + assertThat("Could not find query id in: " + outputString, matcher.find()); + return matcher.group(1); + } +} + diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/ActiveHostFilterTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/ActiveHostFilterTest.java new file mode 100644 index 000000000000..6bf4d61fbcea --- /dev/null +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/ActiveHostFilterTest.java @@ -0,0 +1,56 @@ +/* + * Copyright 2019 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.rest.server; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + +import io.confluent.ksql.util.KsqlHostInfo; +import org.apache.kafka.streams.state.HostInfo; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class ActiveHostFilterTest { + + private KsqlHostInfo activeHost; + private KsqlHostInfo standByHost; + private HostInfo activeHostInfo; + private ActiveHostFilter activeHostFilter; + + @Before + public void setUp() { + activeHost = new KsqlHostInfo("activeHost", 2345); + activeHostInfo = new HostInfo("activeHost", 2345); + standByHost = new KsqlHostInfo("standby1", 1234); + activeHostFilter = new ActiveHostFilter(); + } + + @Test + public void shouldFilterActive() { + // Given: + + // When: + final boolean filterActive = activeHostFilter.filter(activeHostInfo, activeHost, "", -1); + final boolean filterStandby = activeHostFilter.filter(activeHostInfo, standByHost, "", -1); + + // Then: + assertThat(filterActive, is(true)); + assertThat(filterStandby, is(false)); + } +} diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/HeartbeatAgentTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/HeartbeatAgentTest.java index 78ff1c57d9e1..ec64c99934fe 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/HeartbeatAgentTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/HeartbeatAgentTest.java @@ -29,7 +29,7 @@ import io.confluent.ksql.rest.server.HeartbeatAgent.DiscoverClusterService; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.util.HostStatus; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import io.confluent.ksql.util.PersistentQueryMetadata; import java.util.List; import java.util.Map; @@ -59,8 +59,8 @@ public class HeartbeatAgentTest { private HeartbeatAgent heartbeatAgent; private HostInfo localHostInfo; private HostInfo remoteHostInfo; - private KsqlHost localHost; - private KsqlHost remoteHost; + private KsqlHostInfo localHost; + private KsqlHostInfo remoteHost; private List allMetadata0; private List allMetadata1; private static final String LOCALHOST_URL = "http://localhost:8088"; @@ -69,8 +69,8 @@ public class HeartbeatAgentTest { public void setUp() { localHostInfo = new HostInfo("localhost", 8088); remoteHostInfo = new HostInfo("localhost", 8089); - localHost = new KsqlHost ("localhost", 8088); - remoteHost = new KsqlHost("localhost", 8089); + localHost = new KsqlHostInfo("localhost", 8088); + remoteHost = new KsqlHostInfo("localhost", 8089); Builder builder = HeartbeatAgent.builder(); heartbeatAgent = builder @@ -78,7 +78,7 @@ public void setUp() { .heartbeatMissedThreshold(2) .build(ksqlEngine, serviceContext); heartbeatAgent.setLocalAddress(LOCALHOST_URL); - Map hostsStatus = ImmutableMap + Map hostsStatus = ImmutableMap .of(localHost, new HostStatus(true, 0L), remoteHost, new HostStatus(true, 0L)); heartbeatAgent.setHostsStatus(hostsStatus); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/LagReportingAgentTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/LagReportingAgentTest.java index a38a72994aa8..133ba0ea7fcf 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/LagReportingAgentTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/LagReportingAgentTest.java @@ -10,8 +10,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.engine.KsqlEngine; +import io.confluent.ksql.rest.entity.ActiveStandbyEntity; +import io.confluent.ksql.rest.entity.HostStatusEntity; import io.confluent.ksql.rest.entity.HostStoreLags; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.entity.LagInfoEntity; import io.confluent.ksql.rest.entity.LagReportingMessage; import io.confluent.ksql.rest.entity.QueryStateStoreId; @@ -21,7 +23,7 @@ import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.services.SimpleKsqlClient; import io.confluent.ksql.util.HostStatus; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import io.confluent.ksql.util.PersistentQueryMetadata; import java.net.URI; import java.time.Clock; @@ -38,30 +40,45 @@ public class LagReportingAgentTest { private static long TIME_NOW_MS = 100; private static final String LOCALHOST_URL = "http://localhost:8088"; - private static KsqlHostEntity LOCALHOST_INFO = new KsqlHostEntity("localhost", 8088); - private static KsqlHost HOST1 = new KsqlHost("host1", 1234); - private static KsqlHost HOST2 = new KsqlHost("host2", 1234); - private static KsqlHostEntity HOST_ENTITY1 = new KsqlHostEntity("host1", 1234); - private static KsqlHostEntity HOST_ENTITY2 = new KsqlHostEntity("host2", 1234); + private static KsqlHostInfoEntity LOCALHOST_INFO = new KsqlHostInfoEntity("localhost", 8088); + private static KsqlHostInfo HOST1 = new KsqlHostInfo("host1", 1234); + private static KsqlHostInfo HOST2 = new KsqlHostInfo("host2", 1234); + private static KsqlHostInfoEntity HOST_ENTITY1 = new KsqlHostInfoEntity("host1", 1234); + private static KsqlHostInfoEntity HOST_ENTITY2 = new KsqlHostInfoEntity("host2", 1234); + private static KsqlHostInfo HI1 = new KsqlHostInfo("host1", 1234); + private static KsqlHostInfo HI2 = new KsqlHostInfo("host2", 1234); + private static final HostStoreLags EMPTY_HOST_STORE_LAGS = + new HostStoreLags(ImmutableMap.of(), 0); + private static final ImmutableMap EMPTY_ACTIVE_STANDBY_PER_QUERY = + ImmutableMap.of(); private static HostStatus HOST1_STATUS_ALIVE = new HostStatus(true, 0L); private static HostStatus HOST2_STATUS_ALIVE = new HostStatus(true, 0L); private static HostStatus HOST1_STATUS_DEAD = new HostStatus(false, 0L); private static HostStatus HOST2_STATUS_DEAD = new HostStatus(false, 0L); - - private static ImmutableMap HOSTS_ALIVE - = ImmutableMap.builder() + private static HostStatusEntity HOST1_STATUS_ALIVE_ENTITY = new HostStatusEntity( + true, 0L, EMPTY_ACTIVE_STANDBY_PER_QUERY, EMPTY_HOST_STORE_LAGS); + private static HostStatusEntity HOST2_STATUS_ALIVE_ENTITY = new HostStatusEntity( + true, 0L, EMPTY_ACTIVE_STANDBY_PER_QUERY, EMPTY_HOST_STORE_LAGS); + private static HostStatusEntity HOST1_STATUS_DEAD_ENTITY = new HostStatusEntity( + false, 0L, EMPTY_ACTIVE_STANDBY_PER_QUERY, EMPTY_HOST_STORE_LAGS); + private static HostStatusEntity HOST2_STATUS_DEAD_ENTITY = new HostStatusEntity( + false, 0L, EMPTY_ACTIVE_STANDBY_PER_QUERY, EMPTY_HOST_STORE_LAGS); + + + private static ImmutableMap HOSTS_ALIVE + = ImmutableMap.builder() .put(HOST1, HOST1_STATUS_ALIVE) .put(HOST2, HOST2_STATUS_ALIVE) .build(); - private static ImmutableMap HOSTS_HOST1_DEAD - = ImmutableMap.builder() + private static ImmutableMap HOSTS_HOST1_DEAD + = ImmutableMap.builder() .put(HOST1, HOST1_STATUS_DEAD) .put(HOST2, HOST2_STATUS_ALIVE) .build(); - private static ImmutableMap HOSTS_HOST2_DEAD - = ImmutableMap.builder() + private static ImmutableMap HOSTS_HOST2_DEAD + = ImmutableMap.builder() .put(HOST1, HOST1_STATUS_ALIVE) .put(HOST2, HOST2_STATUS_DEAD) .build(); @@ -227,7 +244,7 @@ public void shouldReceiveLags_listAllCurrentPositions() { lagReportingAgent.onHostStatusUpdated(HOSTS_ALIVE); // Then: - ImmutableMap allLags = lagReportingAgent.getAllLags(); + ImmutableMap allLags = lagReportingAgent.getAllLags(); LagInfoEntity lag = allLags.get(HOST_ENTITY1).getStateStoreLags(QUERY_STORE_A) .flatMap(s -> s.getLagByPartition(1)).get(); assertEquals(M1_A1_CUR, lag.getCurrentOffsetPosition()); @@ -296,7 +313,7 @@ public void shouldSendLags() { } private LagReportingMessage hostLag( - KsqlHostEntity host, + KsqlHostInfoEntity host, ImmutableMap lagMap, long lastUpdateMs) { return new LagReportingMessage(host, new HostStoreLags(lagMap, lastUpdateMs)); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/LivenessFilterTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/LivenessFilterTest.java new file mode 100644 index 000000000000..ed4a2c7c8043 --- /dev/null +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/LivenessFilterTest.java @@ -0,0 +1,145 @@ +/* + * Copyright 2019 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.rest.server; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.util.HostStatus; +import io.confluent.ksql.util.KsqlHostInfo; +import java.util.Map; +import java.util.Optional; +import org.apache.kafka.streams.state.HostInfo; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class LivenessFilterTest { + + @Mock + private HeartbeatAgent heartbeatAgent; + + private Optional optionalHeartbeatAgent; + private Map allHostsStatus; + private KsqlHostInfo activeHost; + private KsqlHostInfo standByHost1; + private KsqlHostInfo standByHost2; + private HostInfo activeHostInfo; + private LivenessFilter livenessFilter; + + private static final HostStatus HOST_ALIVE = new HostStatus(true, 0L); + private static final HostStatus HOST_DEAD = new HostStatus(false, 0L); + + @Before + public void setUp() { + activeHost = new KsqlHostInfo("activeHost", 2345); + activeHostInfo = new HostInfo("activeHost", 2345); + standByHost1 = new KsqlHostInfo("standby1", 1234); + standByHost2 = new KsqlHostInfo("standby2", 5678); + optionalHeartbeatAgent = Optional.of(heartbeatAgent); + livenessFilter = new LivenessFilter(optionalHeartbeatAgent); + } + + @Test + public void shouldFilterActiveAlive() { + // Given: + allHostsStatus = ImmutableMap.of( + activeHost, HOST_ALIVE, + standByHost1, HOST_DEAD, + standByHost2, HOST_DEAD + ); + when(heartbeatAgent.getHostsStatus()).thenReturn(allHostsStatus); + + // When: + final boolean filterActive = livenessFilter.filter(activeHostInfo, activeHost, "", -1); + final boolean filterStandby1 = livenessFilter.filter(activeHostInfo, standByHost1, "", -1); + final boolean filterStandby2 = livenessFilter.filter(activeHostInfo, standByHost2, "", -1); + + // Then: + assertThat(filterActive, is(true)); + assertThat(filterStandby1, is(false)); + assertThat(filterStandby2, is(false)); + } + + @Test + public void shouldFilterStandbyAlive() { + // Given: + allHostsStatus = ImmutableMap.of( + activeHost, HOST_DEAD, + standByHost1, HOST_ALIVE, + standByHost2, HOST_DEAD + ); + when(heartbeatAgent.getHostsStatus()).thenReturn(allHostsStatus); + + // When: + final boolean filterActive = livenessFilter.filter(activeHostInfo, activeHost, "", -1); + final boolean filterStandby1 = livenessFilter.filter(activeHostInfo, standByHost1, "", -1); + final boolean filterStandby2 = livenessFilter.filter(activeHostInfo, standByHost2, "", -1); + + // Then: + assertThat(filterActive, is(false)); + assertThat(filterStandby1, is(true)); + assertThat(filterStandby2, is(false)); + } + + @Test + public void shouldFilterAllAlive() { + // Given: + allHostsStatus = ImmutableMap.of( + activeHost, HOST_ALIVE, + standByHost1, HOST_ALIVE, + standByHost2, HOST_ALIVE + ); + when(heartbeatAgent.getHostsStatus()).thenReturn(allHostsStatus); + + // When: + final boolean filterActive = livenessFilter.filter(activeHostInfo, activeHost, "", -1); + final boolean filterStandby1 = livenessFilter.filter(activeHostInfo, standByHost1, "", -1); + final boolean filterStandby2 = livenessFilter.filter(activeHostInfo, standByHost2, "", -1); + + // Then: + assertThat(filterActive, is(true)); + assertThat(filterStandby1, is(true)); + assertThat(filterStandby2, is(true)); + } + + @Test + public void shouldFilterAllDead() { + // Given: + allHostsStatus = ImmutableMap.of( + activeHost, HOST_DEAD, + standByHost1, HOST_DEAD, + standByHost2, HOST_DEAD + ); + when(heartbeatAgent.getHostsStatus()).thenReturn(allHostsStatus); + + // When: + final boolean filterActive = livenessFilter.filter(activeHostInfo, activeHost, "", -1); + final boolean filterStandby1 = livenessFilter.filter(activeHostInfo, standByHost1, "", -1); + final boolean filterStandby2 = livenessFilter.filter(activeHostInfo, standByHost2, "", -1); + + // Then: + assertThat(filterActive, is(false)); + assertThat(filterStandby1, is(false)); + assertThat(filterStandby2, is(false)); + } + +} diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/PullQueryExecutorTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/PullQueryExecutorTest.java index c1356758e561..028dcb1b133d 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/PullQueryExecutorTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/PullQueryExecutorTest.java @@ -24,15 +24,18 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.tree.Query; +import io.confluent.ksql.execution.streams.RoutingFilters; import io.confluent.ksql.rest.server.TemporaryEngine; import io.confluent.ksql.rest.server.resources.KsqlRestException; import io.confluent.ksql.rest.server.validation.CustomValidators; import io.confluent.ksql.statement.ConfiguredStatement; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; +import java.util.Optional; import org.eclipse.jetty.http.HttpStatus.Code; import org.junit.Rule; import org.junit.Test; @@ -62,18 +65,15 @@ public void shouldThrowExceptionIfConfigDisabled() { ImmutableMap.of(), engine.getKsqlConfig() ); + PullQueryExecutor pullQueryExecutor = new PullQueryExecutor( + engine.getEngine(), Optional.empty(), new RoutingFilters(ImmutableList.of())); // Then: expectedException.expect(KsqlException.class); expectedException.expectMessage(containsString("Pull queries are disabled")); // When: - PullQueryExecutor.execute( - query, - ImmutableMap.of(), - engine.getEngine(), - engine.getServiceContext() - ); + pullQueryExecutor.execute(query, engine.getServiceContext()); } } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/ClusterStatusResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/ClusterStatusResourceTest.java index f0d19af21e40..79b78c404020 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/ClusterStatusResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/ClusterStatusResourceTest.java @@ -18,12 +18,10 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; -import static org.mockito.Mockito.when; -import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.rest.entity.ClusterStatusResponse; import io.confluent.ksql.rest.server.HeartbeatAgent; -import io.confluent.ksql.rest.server.LagReportingAgent; import java.util.Optional; import javax.ws.rs.core.Response; import org.junit.Before; @@ -38,16 +36,16 @@ public class ClusterStatusResourceTest { @Mock private HeartbeatAgent heartbeatAgent; @Mock - private LagReportingAgent lagReportingAgent; - + private KsqlEngine ksqlEngine; private ClusterStatusResource clusterStatusResource; @Before public void setUp() { - clusterStatusResource = new ClusterStatusResource(heartbeatAgent, - Optional.of(lagReportingAgent)); - when(lagReportingAgent.getAllLags()).thenReturn(ImmutableMap.of()); + clusterStatusResource = new ClusterStatusResource( + ksqlEngine, + heartbeatAgent, + Optional.empty()); } @Test diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/HeartbeatResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/HeartbeatResourceTest.java index 08541d0252a3..c9db133c8b85 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/HeartbeatResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/HeartbeatResourceTest.java @@ -21,7 +21,7 @@ import io.confluent.ksql.rest.entity.HeartbeatMessage; import io.confluent.ksql.rest.entity.HeartbeatResponse; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.server.HeartbeatAgent; import javax.ws.rs.core.Response; import org.junit.Before; @@ -45,7 +45,7 @@ public void setUp() { @Test public void shouldSendHeartbeat() { // When: - final HeartbeatMessage request = new HeartbeatMessage(new KsqlHostEntity("localhost", 8080), 1); + final HeartbeatMessage request = new HeartbeatMessage(new KsqlHostInfoEntity("localhost", 8080), 1); final Response response = heartbeatResource.registerHeartbeat(request); // Then: diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisherTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisherTest.java index 0d7e3ca85f53..0d2bdb92e9ce 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisherTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisherTest.java @@ -28,9 +28,9 @@ import io.confluent.ksql.parser.tree.Query; import io.confluent.ksql.rest.entity.StreamedRow; import io.confluent.ksql.rest.entity.TableRowsEntity; +import io.confluent.ksql.rest.server.execution.PullQueryExecutor; import io.confluent.ksql.rest.server.resources.streaming.Flow.Subscriber; import io.confluent.ksql.rest.server.resources.streaming.Flow.Subscription; -import io.confluent.ksql.rest.server.resources.streaming.PullQueryPublisher.TheQueryExecutor; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.services.ServiceContext; @@ -64,7 +64,7 @@ public class PullQueryPublisherTest { @Mock private Subscriber> subscriber; @Mock - private TheQueryExecutor pullQueryExecutor; + private PullQueryExecutor pullQueryExecutor; @Mock private TableRowsEntity entity; @Captor @@ -75,10 +75,12 @@ public class PullQueryPublisherTest { @Before public void setUp() { - publisher = new PullQueryPublisher(engine, serviceContext, statement, pullQueryExecutor); - - when(pullQueryExecutor.execute(any(), any(), any())).thenReturn(entity); + publisher = new PullQueryPublisher( + serviceContext, + statement, + pullQueryExecutor); + when(pullQueryExecutor.execute(any(), any())).thenReturn(entity); when(entity.getSchema()).thenReturn(SCHEMA); doAnswer(callRequestAgain()).when(subscriber).onNext(any()); @@ -102,7 +104,7 @@ public void shouldRunQueryWithCorrectParams() { subscription.request(1); // Then: - verify(pullQueryExecutor).execute(statement, engine, serviceContext); + verify(pullQueryExecutor).execute(statement, serviceContext); } @Test @@ -115,7 +117,7 @@ public void shouldOnlyExecuteOnce() { // Then: verify(subscriber).onNext(any()); - verify(pullQueryExecutor).execute(statement, engine, serviceContext); + verify(pullQueryExecutor).execute(statement, serviceContext); } @Test @@ -149,9 +151,8 @@ public void shouldPassSchema() { public void shouldCallOnErrorOnFailure() { // Given: givenSubscribed(); - final Throwable e = new RuntimeException("Boom!"); - when(pullQueryExecutor.execute(any(), any(), any())).thenThrow(e); + when(pullQueryExecutor.execute(any(), any())).thenThrow(e); // When: subscription.request(1); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResourceTest.java index c8db22be6a0d..8bede0c303a1 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResourceTest.java @@ -36,6 +36,7 @@ import static org.mockito.Mockito.when; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.confluent.ksql.GenericRow; @@ -53,8 +54,10 @@ import io.confluent.ksql.rest.entity.KsqlErrorMessage; import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.entity.StreamedRow; +import io.confluent.ksql.execution.streams.RoutingFilters; import io.confluent.ksql.rest.server.StatementParser; import io.confluent.ksql.rest.server.computation.CommandQueue; +import io.confluent.ksql.rest.server.execution.PullQueryExecutor; import io.confluent.ksql.rest.server.resources.KsqlRestException; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; @@ -149,11 +152,13 @@ public class StreamedQueryResourceTest { private KsqlAuthorizationValidator authorizationValidator; @Mock private Errors errorsHandler; + private StreamedQueryResource testResource; private PreparedStatement invalid; private PreparedStatement query; private PreparedStatement print; private KsqlSecurityContext securityContext; + private PullQueryExecutor pullQueryExecutor; @Before public void setup() { @@ -170,6 +175,8 @@ public void setup() { securityContext = new KsqlSecurityContext(Optional.empty(), serviceContext); + pullQueryExecutor = new PullQueryExecutor( + mockKsqlEngine, Optional.empty(), new RoutingFilters(ImmutableList.of())); testResource = new StreamedQueryResource( mockKsqlEngine, mockStatementParser, @@ -178,7 +185,8 @@ public void setup() { COMMAND_QUEUE_CATCHUP_TIMOEUT, activenessRegistrar, Optional.of(authorizationValidator), - errorsHandler + errorsHandler, + pullQueryExecutor ); testResource.configure(VALID_CONFIG); @@ -204,7 +212,8 @@ public void shouldThrowOnHandleStatementIfNotConfigured() { COMMAND_QUEUE_CATCHUP_TIMOEUT, activenessRegistrar, Optional.of(authorizationValidator), - errorsHandler + errorsHandler, + pullQueryExecutor ); // Then: diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpointTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpointTest.java index 3f18d2d328ce..4be64c1ba604 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpointTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpointTest.java @@ -52,6 +52,8 @@ import io.confluent.ksql.rest.entity.Versions; import io.confluent.ksql.rest.server.StatementParser; import io.confluent.ksql.rest.server.computation.CommandQueue; +import io.confluent.ksql.rest.server.execution.PullQueryExecutor; +import io.confluent.ksql.rest.server.resources.streaming.WSQueryEndpoint.IPullQueryPublisher; import io.confluent.ksql.rest.server.resources.streaming.WSQueryEndpoint.PrintTopicPublisher; import io.confluent.ksql.rest.server.resources.streaming.WSQueryEndpoint.QueryPublisher; import io.confluent.ksql.rest.server.services.RestServiceContextFactory.DefaultServiceContextFactory; @@ -139,7 +141,7 @@ public class WSQueryEndpointTest { @Mock private QueryPublisher pushQueryPublisher; @Mock - private QueryPublisher pullQueryPublisher; + private IPullQueryPublisher pullQueryPublisher; @Mock private PrintTopicPublisher topicPublisher; @Mock @@ -164,6 +166,8 @@ public class WSQueryEndpointTest { private Errors errorsHandler; @Mock private DefaultServiceContextFactory defaultServiceContextProvider; + @Mock + private PullQueryExecutor pullQueryExecutor; @Captor private ArgumentCaptor closeReasonCaptor; private Query query; @@ -207,7 +211,8 @@ public void setUp() { serviceContextFactory, defaultServiceContextProvider, serverState, - schemaRegistryClientSupplier + schemaRegistryClientSupplier, + pullQueryExecutor ); } @@ -439,7 +444,8 @@ public void shouldHandlePullQuery() { eq(serviceContext), eq(exec), eq(configuredStatement), - any()); + any(), + eq(pullQueryExecutor)); } @Test diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java index 83d56ac02efb..7655066f48ca 100644 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java +++ b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java @@ -25,7 +25,7 @@ import io.confluent.ksql.rest.entity.CommandStatuses; import io.confluent.ksql.rest.entity.HealthCheckResponse; import io.confluent.ksql.rest.entity.KsqlEntityList; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.entity.ServerInfo; import java.io.Closeable; import java.io.InputStream; @@ -91,7 +91,7 @@ public RestResponse getServerHealth() { } public Future makeAsyncHeartbeatRequest( - final KsqlHostEntity host, + final KsqlHostInfoEntity host, final long timestamp ) { return target().postAsyncHeartbeatRequest(host, timestamp); diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java index e945d23e81b2..bad19fe77ba7 100644 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java +++ b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java @@ -24,7 +24,7 @@ import io.confluent.ksql.rest.entity.HealthCheckResponse; import io.confluent.ksql.rest.entity.HeartbeatMessage; import io.confluent.ksql.rest.entity.KsqlEntityList; -import io.confluent.ksql.rest.entity.KsqlHostEntity; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.entity.LagReportingMessage; import io.confluent.ksql.rest.entity.ServerInfo; @@ -83,7 +83,7 @@ public RestResponse getServerHealth() { } public Future postAsyncHeartbeatRequest( - final KsqlHostEntity host, + final KsqlHostInfoEntity host, final long timestamp ) { return postAsync( diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/ActiveStandbyEntity.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/ActiveStandbyEntity.java new file mode 100644 index 000000000000..a3d990f82631 --- /dev/null +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/ActiveStandbyEntity.java @@ -0,0 +1,97 @@ +/* + * 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.rest.entity; + +import static java.util.Objects.requireNonNull; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableSet; +import com.google.errorprone.annotations.Immutable; +import java.util.Objects; +import java.util.Set; + +@Immutable +@JsonIgnoreProperties(ignoreUnknown = true) +public class ActiveStandbyEntity { + + private final ImmutableSet activeStores; + private final ImmutableSet activePartitions; + private final ImmutableSet standByStores; + private final ImmutableSet standByPartitions; + + @JsonCreator + public ActiveStandbyEntity( + @JsonProperty("activeStores") final Set activeStores, + @JsonProperty("activePartitions") final Set activePartitions, + @JsonProperty("standByStores") final Set standByStores, + @JsonProperty("standByPartitions") final Set standByPartitions + ) { + this.activeStores = ImmutableSet.copyOf(requireNonNull(activeStores)); + this.activePartitions = ImmutableSet.copyOf(requireNonNull(activePartitions)); + this.standByStores = ImmutableSet.copyOf(requireNonNull(standByStores)); + this.standByPartitions = ImmutableSet.copyOf(requireNonNull(standByPartitions)); + } + + public Set getActiveStores() { + return activeStores; + } + + public Set getStandByStores() { + return standByStores; + } + + public Set getActivePartitions() { + return activePartitions; + } + + public Set getStandByPartitions() { + return standByPartitions; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final ActiveStandbyEntity that = (ActiveStandbyEntity) o; + return Objects.equals(activeStores, that.activeStores) + && Objects.equals(standByStores, that.standByStores) + && Objects.equals(activePartitions, that.activePartitions) + && Objects.equals(standByPartitions, that.standByPartitions); + } + + @Override + public int hashCode() { + return Objects.hash(activeStores, standByStores, activePartitions, standByPartitions); + } + + @Override + public String toString() { + return "ActiveStandbyEntity{" + + " Active stores = " + activeStores + + ", Active partitions = " + activePartitions + + ", Standby stores = " + standByStores + + ", Standby partitions = " + standByPartitions + + "}"; + } +} diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/ClusterStatusResponse.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/ClusterStatusResponse.java index 5e5cf990682f..0787d5489d63 100644 --- a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/ClusterStatusResponse.java +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/ClusterStatusResponse.java @@ -29,15 +29,16 @@ @Immutable public final class ClusterStatusResponse { - private final ImmutableMap clusterStatus; + private final ImmutableMap clusterStatus; @JsonCreator public ClusterStatusResponse( - @JsonProperty("clusterStatus") final Map clusterStatus) { + @JsonProperty("clusterStatus") + final Map clusterStatus) { this.clusterStatus = ImmutableMap.copyOf(requireNonNull(clusterStatus, "status")); } - public Map getClusterStatus() { + public Map getClusterStatus() { return clusterStatus; } @@ -59,4 +60,9 @@ public boolean equals(final Object o) { public int hashCode() { return Objects.hash(clusterStatus); } + + @Override + public String toString() { + return "ClusterStatus = " + clusterStatus; + } } diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/HeartbeatMessage.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/HeartbeatMessage.java index a7e79997d816..0a0775c221d0 100644 --- a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/HeartbeatMessage.java +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/HeartbeatMessage.java @@ -25,17 +25,17 @@ @JsonSubTypes({}) public class HeartbeatMessage { - private final KsqlHostEntity hostInfo; + private final KsqlHostInfoEntity hostInfo; private final long timestamp; @JsonCreator - public HeartbeatMessage(@JsonProperty("hostInfo") final KsqlHostEntity hostInfo, + public HeartbeatMessage(@JsonProperty("hostInfo") final KsqlHostInfoEntity hostInfo, @JsonProperty("timestamp") final long timestamp) { this.hostInfo = hostInfo; this.timestamp = timestamp; } - public KsqlHostEntity getHostInfo() { + public KsqlHostInfoEntity getHostInfo() { return hostInfo; } diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/HostStatusEntity.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/HostStatusEntity.java index e207cdb6dbf9..65d7d7078940 100644 --- a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/HostStatusEntity.java +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/HostStatusEntity.java @@ -18,7 +18,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; import com.google.errorprone.annotations.Immutable; +import java.util.Map; import java.util.Objects; @Immutable @@ -27,16 +29,21 @@ public class HostStatusEntity { private final boolean hostAlive; private final long lastStatusUpdateMs; + private final ImmutableMap activeStandbyPerQuery; private final HostStoreLags hostStoreLags; @JsonCreator public HostStatusEntity( @JsonProperty("hostAlive") final boolean hostAlive, @JsonProperty("lastStatusUpdateMs") final long lastStatusUpdateMs, + @JsonProperty("activeStandbyPerQuery") + final Map activeStandbyPerQuery, @JsonProperty("hostStoreLags") final HostStoreLags hostStoreLags ) { this.hostAlive = hostAlive; this.lastStatusUpdateMs = lastStatusUpdateMs; + this.activeStandbyPerQuery = ImmutableMap.copyOf(Objects.requireNonNull( + activeStandbyPerQuery, "activeStandbyPerQuery")); this.hostStoreLags = Objects.requireNonNull(hostStoreLags, "hostStoreLags"); } @@ -48,6 +55,10 @@ public long getLastStatusUpdateMs() { return lastStatusUpdateMs; } + public ImmutableMap getActiveStandbyPerQuery() { + return activeStandbyPerQuery; + } + public HostStoreLags getHostStoreLags() { return hostStoreLags; } @@ -65,12 +76,13 @@ public boolean equals(final Object o) { final HostStatusEntity that = (HostStatusEntity) o; return hostAlive == that.hostAlive && lastStatusUpdateMs == that.lastStatusUpdateMs + && Objects.equals(activeStandbyPerQuery, that.activeStandbyPerQuery) && Objects.equals(hostStoreLags, that.hostStoreLags); } @Override public int hashCode() { - return Objects.hash(hostAlive, lastStatusUpdateMs, hostStoreLags); + return Objects.hash(hostAlive, lastStatusUpdateMs, activeStandbyPerQuery, hostStoreLags); } @Override @@ -78,6 +90,8 @@ public String toString() { return "HostStatusEntity{" + "hostAlive=" + hostAlive + ", lastStatusUpdateMs=" + lastStatusUpdateMs + + ", activeStandbyPerQuery=" + activeStandbyPerQuery + + ", hostStoreLags=" + hostStoreLags + '}'; } } diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlHostEntity.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlHostInfoEntity.java similarity index 88% rename from ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlHostEntity.java rename to ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlHostInfoEntity.java index 06853bddb5d5..d691a3efac0e 100644 --- a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlHostEntity.java +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlHostInfoEntity.java @@ -20,17 +20,17 @@ import com.fasterxml.jackson.annotation.JsonValue; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.util.KsqlException; -import io.confluent.ksql.util.KsqlHost; +import io.confluent.ksql.util.KsqlHostInfo; import java.util.Objects; @Immutable @JsonIgnoreProperties(ignoreUnknown = true) -public class KsqlHostEntity { +public class KsqlHostInfoEntity { private final String host; private final int port; - public KsqlHostEntity( + public KsqlHostInfoEntity( final String host, final int port ) { @@ -39,7 +39,7 @@ public KsqlHostEntity( } @JsonCreator - public KsqlHostEntity(final String serializedPair) { + public KsqlHostInfoEntity(final String serializedPair) { final String [] parts = serializedPair.split(":"); if (parts.length != 2) { throw new KsqlException("Invalid host info. Expected format: :, but was " @@ -64,8 +64,8 @@ public int getPort() { return port; } - public KsqlHost toKsqlHost() { - return new KsqlHost(host, port); + public KsqlHostInfo toKsqlHost() { + return new KsqlHostInfo(host, port); } @Override @@ -78,7 +78,7 @@ public boolean equals(final Object o) { return false; } - final KsqlHostEntity that = (KsqlHostEntity) o; + final KsqlHostInfoEntity that = (KsqlHostInfoEntity) o; return Objects.equals(host, that.host) && port == that.port; } diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/LagReportingMessage.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/LagReportingMessage.java index d9927a164050..3c69fd192cc3 100644 --- a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/LagReportingMessage.java +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/LagReportingMessage.java @@ -25,19 +25,19 @@ @JsonIgnoreProperties(ignoreUnknown = true) public class LagReportingMessage { - private final KsqlHostEntity ksqlHost; + private final KsqlHostInfoEntity ksqlHost; private final HostStoreLags hostStoreLags; @JsonCreator public LagReportingMessage( - @JsonProperty("ksqlHost") final KsqlHostEntity ksqlHost, + @JsonProperty("ksqlHost") final KsqlHostInfoEntity ksqlHost, @JsonProperty("hostStoreLags") final HostStoreLags hostStoreLags ) { this.ksqlHost = Objects.requireNonNull(ksqlHost, "hostInfo"); this.hostStoreLags = Objects.requireNonNull(hostStoreLags, "hostStoreLags"); } - public KsqlHostEntity getKsqlHost() { + public KsqlHostInfoEntity getKsqlHost() { return ksqlHost; } diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/TopicPartitionEntity.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/TopicPartitionEntity.java new file mode 100644 index 000000000000..ef628ee48168 --- /dev/null +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/TopicPartitionEntity.java @@ -0,0 +1,75 @@ +/* + * 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.rest.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.errorprone.annotations.Immutable; +import java.util.Objects; + +@Immutable +@JsonIgnoreProperties(ignoreUnknown = true) +public class TopicPartitionEntity { + + private final String topic; + private final int partition; + + @JsonCreator + public TopicPartitionEntity( + @JsonProperty("topic") final String topic, + @JsonProperty("partition") final int partition + ) { + this.topic = topic; + this.partition = partition; + } + + public String getTopic() { + return topic; + } + + public int getPartition() { + return partition; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final TopicPartitionEntity that = (TopicPartitionEntity) o; + return topic.equals(that.topic) + && partition == that.partition; + } + + @Override + public int hashCode() { + return Objects.hash(topic, partition); + } + + @Override + public String toString() { + return "TopicPartition{" + + "topic=" + topic + + ", partition=" + partition + + '}'; + } +} diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/RoutingFilter.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/RoutingFilter.java new file mode 100644 index 000000000000..b45971ae8b0f --- /dev/null +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/RoutingFilter.java @@ -0,0 +1,34 @@ +/*o + * 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.execution.streams; + +import io.confluent.ksql.util.KsqlHostInfo; +import org.apache.kafka.streams.state.HostInfo; + +/** + * Used to filter ksql hosts based on criteria specified in implementing classes. + * One such example is a filter that checks whether hosts are alive or dead as determined + * by the heartbeat agent. + */ +public interface RoutingFilter { + + boolean filter( + HostInfo activeHost, + KsqlHostInfo hostInfo, + String storeName, + int partition); + +} diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/RoutingFilters.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/RoutingFilters.java new file mode 100644 index 000000000000..7ad4c995b0db --- /dev/null +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/RoutingFilters.java @@ -0,0 +1,40 @@ +/* + * Copyright 2019 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.execution.streams; + +import com.google.common.collect.ImmutableList; +import io.confluent.ksql.util.KsqlHostInfo; +import java.util.Objects; +import org.apache.kafka.streams.state.HostInfo; + +public final class RoutingFilters implements RoutingFilter { + + private final ImmutableList routingFilters; + + public RoutingFilters(final ImmutableList routingFilters) { + this.routingFilters = Objects.requireNonNull(routingFilters, "routingFilters"); + } + + public boolean filter( + final HostInfo activeHost, + final KsqlHostInfo host, + final String storeName, + final int partition + ) { + return routingFilters.stream() + .allMatch(f -> f.filter(activeHost, host, storeName, partition)); + } +} diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/Locator.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/Locator.java index c9c59d24cedf..b059116fb93d 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/Locator.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/Locator.java @@ -15,7 +15,9 @@ package io.confluent.ksql.execution.streams.materialization; +import io.confluent.ksql.execution.streams.RoutingFilter; import java.net.URI; +import java.util.List; import java.util.Optional; import org.apache.kafka.connect.data.Struct; @@ -28,16 +30,18 @@ public interface Locator { /** - * Locate which KSQL node stores the supplied {@code key}. + * Locate which KSQL nodes store the supplied {@code key}. * *

Implementations are free to return {@link Optional#empty()} if the location is not known at * this time. * * @param key the required key. - * @return the owning node, if known. + * @return the list of nodes, that can potentially serve the key. */ - Optional locate(Struct key); - + List locate( + Struct key, + RoutingFilter routingFilters + ); interface KsqlNode { diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocator.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocator.java index d52bc0519dbc..ef7f28c8f0a6 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocator.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocator.java @@ -17,22 +17,33 @@ import static java.util.Objects.requireNonNull; +import com.google.common.annotations.VisibleForTesting; import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.execution.streams.RoutingFilter; import io.confluent.ksql.execution.streams.materialization.Locator; +import io.confluent.ksql.execution.streams.materialization.MaterializationException; +import io.confluent.ksql.util.KsqlHostInfo; import java.net.URI; import java.net.URL; -import java.util.Optional; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyQueryMetadata; import org.apache.kafka.streams.state.HostInfo; -import org.apache.kafka.streams.state.StreamsMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Kafka Streams implementation of {@link Locator}. */ final class KsLocator implements Locator { + private static final Logger LOG = LoggerFactory.getLogger(KsLocator.class); private final String stateStoreName; private final KafkaStreams kafkaStreams; private final Serializer keySerializer; @@ -51,27 +62,58 @@ final class KsLocator implements Locator { } @Override - @SuppressWarnings("deprecation") - public Optional locate(final Struct key) { - final StreamsMetadata metadata = kafkaStreams - .metadataForKey(stateStoreName, key, keySerializer); - - if (metadata == StreamsMetadata.NOT_AVAILABLE) { - return Optional.empty(); + public List locate( + final Struct key, + final RoutingFilter routingFilters + ) { + final KeyQueryMetadata metadata = kafkaStreams + .queryMetadataForKey(stateStoreName, key, keySerializer); + + // Fail fast if Streams not ready. Let client handle it + if (metadata == KeyQueryMetadata.NOT_AVAILABLE) { + LOG.debug("KeyQueryMetadata not available for state store {} and key {}", + stateStoreName, key); + throw new MaterializationException(String.format( + "KeyQueryMetadata not available for state store %s and key %s", stateStoreName, key)); } - final HostInfo hostInfo = metadata.hostInfo(); - return Optional.of(asNode(hostInfo)); + final HostInfo activeHost = metadata.getActiveHost(); + final Set standByHosts = metadata.getStandbyHosts(); + LOG.debug("Before filtering: Active host {} , standby hosts {}", activeHost, standByHosts); + + final Stream active = Stream.of(asKsqlHost(activeHost)); + final Stream standby = standByHosts + .stream() + .map(this::asKsqlHost); + final Stream hostStream = Stream.concat(active, standby); + + // Filter out hosts based on active and liveness filters. + // The list is ordered by routing preference: active node is first, then standby nodes. + // If heartbeat is not enabled, all hosts are considered alive. + final List filteredHosts = hostStream + .filter(hostInfo -> routingFilters.filter( + activeHost, hostInfo, stateStoreName, metadata.getPartition())) + .map(this::asNode) + .collect(Collectors.toList()); + + LOG.debug("Filtered and ordered hosts: {}", filteredHosts); + return filteredHosts; + } + + @VisibleForTesting + KsqlHostInfo asKsqlHost(final HostInfo hostInfo) { + return new KsqlHostInfo(hostInfo.host(), hostInfo.port()); } - private KsqlNode asNode(final HostInfo hostInfo) { + @VisibleForTesting + KsqlNode asNode(final KsqlHostInfo host) { return new Node( - isLocalHost(hostInfo), - buildLocation(hostInfo) + isLocalHost(host), + buildLocation(host) ); } - private boolean isLocalHost(final HostInfo hostInfo) { + private boolean isLocalHost(final KsqlHostInfo hostInfo) { if (hostInfo.port() != localHost.getPort()) { return false; } @@ -80,7 +122,7 @@ private boolean isLocalHost(final HostInfo hostInfo) { || hostInfo.host().equalsIgnoreCase("localhost"); } - private URI buildLocation(final HostInfo remoteInfo) { + private URI buildLocation(final KsqlHostInfo remoteInfo) { try { return new URL( localHost.getProtocol(), @@ -114,5 +156,34 @@ public boolean isLocal() { public URI location() { return location; } + + @Override + public String toString() { + return "Node{" + + "local = " + local + + ", location = " + location + + "}"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final Node that = (Node) o; + return local == that.local + && location.equals(that.location); + } + + @Override + public int hashCode() { + return Objects.hash(local, location); + } + } } diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsStateStore.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsStateStore.java index b8c38ee3c350..c338fd1f2816 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsStateStore.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsStateStore.java @@ -72,9 +72,16 @@ T store(final QueryableStoreType queryableStoreType) { awaitRunning(); try { - - return kafkaStreams - .store(StoreQueryParams.fromNameAndType(stateStoreName, queryableStoreType)); + if (ksqlConfig.getBoolean(KsqlConfig.KSQL_QUERY_PULL_ENABLE_STANDBY_READS)) { + // True flag allows queries on standby and replica state stores + return kafkaStreams.store( + StoreQueryParams.fromNameAndType(stateStoreName, queryableStoreType) + .enableStaleStores()); + } else { + // False flag allows queries only on active state store + return kafkaStreams.store( + StoreQueryParams.fromNameAndType(stateStoreName, queryableStoreType)); + } } catch (final Exception e) { final State state = kafkaStreams.state(); if (state != State.RUNNING) { diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocatorTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocatorTest.java index 3cd96cd5fd40..3c6250ea8b65 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocatorTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocatorTest.java @@ -16,27 +16,40 @@ package io.confluent.ksql.execution.streams.materialization.ks; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.is; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.google.common.testing.NullPointerTester; import com.google.common.testing.NullPointerTester.Visibility; +import io.confluent.ksql.execution.streams.RoutingFilter; +import io.confluent.ksql.execution.streams.RoutingFilters; import io.confluent.ksql.execution.streams.materialization.Locator.KsqlNode; +import io.confluent.ksql.execution.streams.materialization.MaterializationException; +import io.confluent.ksql.util.HostStatus; +import io.confluent.ksql.util.KsqlHostInfo; import java.net.MalformedURLException; import java.net.URI; import java.net.URL; +import java.util.Collections; +import java.util.List; import java.util.Optional; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyQueryMetadata; import org.apache.kafka.streams.state.HostInfo; -import org.apache.kafka.streams.state.StreamsMetadata; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; @@ -51,20 +64,65 @@ public class KsLocatorTest { @Mock private KafkaStreams kafkaStreams; @Mock + private KeyQueryMetadata keyQueryMetadata; + @Mock private Serializer keySerializer; @Mock - private HostInfo hostInfo; - + private RoutingFilter livenessFilter; + @Mock + private RoutingFilter activeFilter; + + private KsqlHostInfo activeHost; + private KsqlHostInfo standByHost1; + private KsqlHostInfo standByHost2; + private HostInfo activeHostInfo; + private HostInfo standByHostInfo1; + private HostInfo standByHostInfo2; private KsLocator locator; + private KsqlNode activeNode; + private KsqlNode standByNode1; + private KsqlNode standByNode2; + private RoutingFilters routingStandbyFilters; + private RoutingFilters routingActiveFilters; + private static final HostStatus HOST_ALIVE = new HostStatus(true, 0L); + private static final HostStatus HOST_DEAD = new HostStatus(false, 0L); + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); @Before public void setUp() { locator = new KsLocator(STORE_NAME, kafkaStreams, keySerializer, LOCAL_HOST_URL); - givenOwnerMetadata(Optional.empty()); - - when(hostInfo.host()).thenReturn("remoteHost"); - when(hostInfo.port()).thenReturn(2345); + activeHost = new KsqlHostInfo("remoteHost", 2345); + activeHostInfo = new HostInfo("remoteHost", 2345); + standByHost1 = new KsqlHostInfo("standby1", 1234); + standByHostInfo1 = new HostInfo("standby1", 1234); + standByHost2 = new KsqlHostInfo("standby2", 5678); + standByHostInfo2 = new HostInfo("standby2", 5678); + + activeNode = locator.asNode(activeHost); + standByNode1 = locator.asNode(standByHost1); + standByNode2 = locator.asNode(standByHost2); + + routingStandbyFilters = new RoutingFilters(ImmutableList.of(livenessFilter)); + routingActiveFilters = new RoutingFilters(ImmutableList.of(activeFilter, livenessFilter)); + + // Only active serves query + when(activeFilter.filter(eq(activeHostInfo), eq(activeHost), anyString(), anyInt())) + .thenReturn(true); + when(activeFilter.filter(eq(activeHostInfo), eq(standByHost1), anyString(), anyInt())) + .thenReturn(false); + when(activeFilter.filter(eq(activeHostInfo), eq(standByHost2), anyString(), anyInt())) + .thenReturn(false); + + // Heartbeat not enabled, all hosts alive + when(livenessFilter.filter(eq(activeHostInfo), eq(activeHost), anyString(), anyInt())) + .thenReturn(true); + when(livenessFilter.filter(eq(activeHostInfo), eq(standByHost1), anyString(), anyInt())) + .thenReturn(true); + when(livenessFilter.filter(eq(activeHostInfo), eq(standByHost2), anyString(), anyInt())) + .thenReturn(true); } @Test @@ -77,125 +135,205 @@ public void shouldThrowNPEs() { } @Test - @SuppressWarnings("deprecation") - public void shouldRequestMetadata() { - // When: - locator.locate(SOME_KEY); - - // Then: - verify(kafkaStreams).metadataForKey(STORE_NAME, SOME_KEY, keySerializer); - } - - @Test - public void shouldReturnEmptyIfOwnerNotKnown() { + public void shouldThrowIfMetadataNotAvailable() { // Given: - givenOwnerMetadata(Optional.empty()); + getEmtpyMetadata(); - // When: - final Optional result = locator.locate(SOME_KEY); + // Expect: + expectedException.expect(MaterializationException.class); + expectedException.expectMessage( + "KeyQueryMetadata not available for state store someStoreName and key Struct{}"); - // Then: - assertThat(result, is(Optional.empty())); + // When: + locator.locate(SOME_KEY, routingActiveFilters); } @Test public void shouldReturnOwnerIfKnown() { // Given: - givenOwnerMetadata(Optional.of(hostInfo)); + getActiveAndStandbyMetadata(); // When: - final Optional result = locator.locate(SOME_KEY); + final List result = locator.locate(SOME_KEY, routingActiveFilters); // Then: - final Optional url = result.map(KsqlNode::location); + final Optional url = result.stream().findFirst().map(KsqlNode::location); assertThat(url.map(URI::getScheme), is(Optional.of(LOCAL_HOST_URL.getProtocol()))); - assertThat(url.map(URI::getHost), is(Optional.of(hostInfo.host()))); - assertThat(url.map(URI::getPort), is(Optional.of(hostInfo.port()))); + assertThat(url.map(URI::getHost), is(Optional.of(activeHost.host()))); + assertThat(url.map(URI::getPort), is(Optional.of(activeHost.port()))); assertThat(url.map(URI::getPath), is(Optional.of("/"))); } @Test public void shouldReturnLocalOwnerIfSameAsSuppliedLocalHost() { // Given: - when(hostInfo.host()).thenReturn(LOCAL_HOST_URL.getHost()); - when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort()); - givenOwnerMetadata(Optional.of(hostInfo)); + final HostInfo localHostInfo = new HostInfo(LOCAL_HOST_URL.getHost(), LOCAL_HOST_URL.getPort()); + final KsqlHostInfo localHost = locator.asKsqlHost(localHostInfo); + getActiveAndStandbyMetadata(localHostInfo); + when(activeFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); + when(livenessFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); // When: - final Optional result = locator.locate(SOME_KEY); + final List result = locator.locate(SOME_KEY, routingActiveFilters); // Then: - assertThat(result.map(KsqlNode::isLocal), is(Optional.of(true))); + assertThat(result.stream().findFirst().map(KsqlNode::isLocal), is(Optional.of(true))); } @Test public void shouldReturnLocalOwnerIfExplicitlyLocalHostOnSamePortAsSuppliedLocalHost() { // Given: - when(hostInfo.host()).thenReturn("LocalHOST"); - when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort()); - givenOwnerMetadata(Optional.of(hostInfo)); + final HostInfo localHostInfo = new HostInfo("LocalHOST", LOCAL_HOST_URL.getPort()); + final KsqlHostInfo localHost = locator.asKsqlHost(localHostInfo); + getActiveAndStandbyMetadata(localHostInfo); + when(activeFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); + when(livenessFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); // When: - final Optional result = locator.locate(SOME_KEY); + final List result = locator.locate(SOME_KEY, routingActiveFilters); // Then: - assertThat(result.map(KsqlNode::isLocal), is(Optional.of(true))); + assertThat(result.stream().findFirst().map(KsqlNode::isLocal), is(Optional.of(true))); } @Test public void shouldReturnRemoteOwnerForDifferentHost() { // Given: - when(hostInfo.host()).thenReturn("different"); - when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort()); - givenOwnerMetadata(Optional.of(hostInfo)); + final HostInfo localHostInfo = new HostInfo("different", LOCAL_HOST_URL.getPort()); + final KsqlHostInfo localHost = locator.asKsqlHost(localHostInfo); + getActiveAndStandbyMetadata(localHostInfo); + when(activeFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); + when(livenessFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); // When: - final Optional result = locator.locate(SOME_KEY); + final List result = locator.locate(SOME_KEY, routingActiveFilters); // Then: - assertThat(result.map(KsqlNode::isLocal), is(Optional.of(false))); + assertThat(result.stream().findFirst().map(KsqlNode::isLocal), is(Optional.of(false))); } @Test public void shouldReturnRemoteOwnerForDifferentPort() { // Given: - when(hostInfo.host()).thenReturn(LOCAL_HOST_URL.getHost()); - when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort() + 1); - givenOwnerMetadata(Optional.of(hostInfo)); + final HostInfo localHostInfo = new HostInfo(LOCAL_HOST_URL.getHost(), LOCAL_HOST_URL.getPort()+1); + final KsqlHostInfo localHost = locator.asKsqlHost(localHostInfo); + getActiveAndStandbyMetadata(localHostInfo); + when(activeFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); + when(livenessFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); // When: - final Optional result = locator.locate(SOME_KEY); + final List result = locator.locate(SOME_KEY, routingActiveFilters); // Then: - assertThat(result.map(KsqlNode::isLocal), is(Optional.of(false))); + assertThat(result.stream().findFirst().map(KsqlNode::isLocal), is(Optional.of(false))); } @Test public void shouldReturnRemoteOwnerForDifferentPortOnLocalHost() { // Given: - when(hostInfo.host()).thenReturn("LOCALhost"); - when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort() + 1); - givenOwnerMetadata(Optional.of(hostInfo)); + final HostInfo localHostInfo = new HostInfo("LOCALhost", LOCAL_HOST_URL.getPort()+1); + final KsqlHostInfo localHost = locator.asKsqlHost(localHostInfo); + getActiveAndStandbyMetadata(localHostInfo); + when(activeFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); + when(livenessFilter.filter(eq(localHostInfo), eq(localHost), anyString(), anyInt())) + .thenReturn(true); + + // When: + final List result = locator.locate(SOME_KEY, routingActiveFilters); + + // Then: + assertThat(result.stream().findFirst().map(KsqlNode::isLocal), is(Optional.of(false))); + } + + @Test + public void shouldReturnActiveWhenRoutingStandbyNotEnabledHeartBeatNotEnabled() { + // Given: + getActiveAndStandbyMetadata(); // When: - final Optional result = locator.locate(SOME_KEY); + final List result = locator.locate(SOME_KEY, routingActiveFilters); // Then: - assertThat(result.map(KsqlNode::isLocal), is(Optional.of(false))); + assertThat(result.size(), is(1)); + assertThat(result.stream().findFirst().get(), is(activeNode)); } - @SuppressWarnings({"unchecked", "deprecation"}) - private void givenOwnerMetadata(final Optional hostInfo) { - final StreamsMetadata metadata = hostInfo - .map(hi -> { - final StreamsMetadata md = mock(StreamsMetadata.class); - when(md.hostInfo()).thenReturn(hostInfo.get()); - return md; - }) - .orElse(StreamsMetadata.NOT_AVAILABLE); + @Test + public void shouldReturnActiveAndStandBysWhenRoutingStandbyEnabledHeartBeatNotEnabled() { + // Given: + getActiveAndStandbyMetadata(); + + // When: + final List result = locator.locate(SOME_KEY, routingStandbyFilters); + + // Then: + assertThat(result.size(), is(3)); + assertThat(result.stream().findFirst().get(), is(activeNode)); + assertThat(result, containsInAnyOrder(activeNode, standByNode1, standByNode2)); + } + + @Test + public void shouldReturnStandBysWhenActiveDown() { + // Given: + getActiveAndStandbyMetadata(); + when(livenessFilter.filter(eq(activeHostInfo), eq(activeHost), anyString(), anyInt())) + .thenReturn(false); + + // When: + final List result = locator.locate(SOME_KEY, routingStandbyFilters); + + // Then: + assertThat(result.size(), is(2)); + assertThat(result, containsInAnyOrder(standByNode1, standByNode2)); + } + + @Test + public void shouldReturnOneStandByWhenActiveAndOtherStandByDown() { + // Given: + getActiveAndStandbyMetadata(); + when(livenessFilter.filter(eq(activeHostInfo), eq(activeHost), anyString(), anyInt())) + .thenReturn(false); + when(livenessFilter.filter(eq(activeHostInfo), eq(standByHost1), anyString(), anyInt())) + .thenReturn(false); + + // When: + final List result = locator.locate(SOME_KEY, routingStandbyFilters); + + // Then: + assertThat(result.size(), is(1)); + assertThat(result.stream().findFirst().get(), is(standByNode2)); + } + + @SuppressWarnings("unchecked") + private void getEmtpyMetadata() { + when(kafkaStreams.queryMetadataForKey(any(), any(), any(Serializer.class))) + .thenReturn(KeyQueryMetadata.NOT_AVAILABLE); + } + + @SuppressWarnings("unchecked") + private void getActiveAndStandbyMetadata() { + when(keyQueryMetadata.getActiveHost()).thenReturn(activeHostInfo); + when(keyQueryMetadata.getStandbyHosts()).thenReturn(ImmutableSet.of( + standByHostInfo1, standByHostInfo2)); + when(kafkaStreams.queryMetadataForKey(any(), any(), any(Serializer.class))) + .thenReturn(keyQueryMetadata); + } - when(kafkaStreams.metadataForKey(any(), any(), any(Serializer.class))) - .thenReturn(metadata); + @SuppressWarnings("unchecked") + private void getActiveAndStandbyMetadata(final HostInfo activeHostInfo) { + when(keyQueryMetadata.getActiveHost()).thenReturn(activeHostInfo); + when(keyQueryMetadata.getStandbyHosts()).thenReturn(Collections.emptySet()); + when(kafkaStreams.queryMetadataForKey(any(), any(), any(Serializer.class))) + .thenReturn(keyQueryMetadata); } private static URL localHost() { diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsStateStoreTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsStateStoreTest.java index 46d41e5f8b84..efd7a6399af9 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsStateStoreTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsStateStoreTest.java @@ -142,7 +142,6 @@ public void shouldThrowIfNotRunningAfterFailedToGetStore() { when(kafkaStreams.state()) .thenReturn(State.RUNNING) .thenReturn(State.NOT_RUNNING); - when(kafkaStreams.store(any())).thenThrow(new IllegalStateException()); // When: