From c3a70c9774a6ebe6c7bf6a0cb3bf8a8cd3f7724e Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Thu, 8 Nov 2018 11:18:01 -0800 Subject: [PATCH 01/13] return command topic offset with success response from KsqlResource. --- .../ksql/cli/console/ConsoleTest.java | 2 +- .../ksql/rest/entity/CommandStatusEntity.java | 24 ++++++++--- .../rest/server/computation/CommandStore.java | 5 ++- .../computation/QueuedCommandStatus.java | 11 +++++ .../rest/server/resources/KsqlResource.java | 3 +- .../server/computation/CommandStoreTest.java | 43 ++++++++++++++----- .../server/resources/KsqlResourceTest.java | 3 +- 7 files changed, 70 insertions(+), 21 deletions(-) diff --git a/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java b/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java index 139d6d3a6436..bd97d24c9473 100644 --- a/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java +++ b/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java @@ -127,7 +127,7 @@ public void testPrintKSqlEntityList() throws IOException { for (int i = 0; i < 5; i++) { final KsqlEntityList entityList = new KsqlEntityList(ImmutableList.of( - new CommandStatusEntity("e", "topic/1/create", "SUCCESS", "Success Message"), + new CommandStatusEntity("e", "topic/1/create", "SUCCESS", "Success Message", "0"), new PropertiesList("e", properties, Collections.emptyList(), Collections.emptyList()), new Queries("e", queries), new SourceDescriptionEntity( diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java index fccb1b4900e9..e94c6d483ece 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java @@ -27,27 +27,32 @@ public class CommandStatusEntity extends KsqlEntity { private final CommandId commandId; private final CommandStatus commandStatus; + private final long commandOffset; public CommandStatusEntity( final String statementText, final CommandId commandId, - final CommandStatus commandStatus + final CommandStatus commandStatus, + final long commandOffset ) { super(statementText); this.commandId = commandId; this.commandStatus = commandStatus; + this.commandOffset = commandOffset; } public CommandStatusEntity( final String statementText, final String commandId, final String status, - final String message + final String message, + final String commandOffset ) { this( statementText, CommandId.fromString(commandId), - new CommandStatus(CommandStatus.Status.valueOf(status), message) + new CommandStatus(CommandStatus.Status.valueOf(status), message), + Long.parseLong(commandOffset) ); } @@ -58,7 +63,8 @@ public CommandStatusEntity(final Map properties) { (String) properties.get("statementText"), (String) properties.get("commandId"), (String) ((Map) properties.get("commandStatus")).get("status"), - (String) ((Map) properties.get("commandStatus")).get("message") + (String) ((Map) properties.get("commandStatus")).get("message"), + String.valueOf(properties.get("commandOffset")) ); } @@ -71,6 +77,10 @@ public CommandStatus getCommandStatus() { return commandStatus; } + public long getCommandOffset() { + return commandOffset; + } + @Override public boolean equals(final Object o) { if (this == o) { @@ -81,12 +91,13 @@ public boolean equals(final Object o) { } final CommandStatusEntity that = (CommandStatusEntity) o; return Objects.equals(getCommandId(), that.getCommandId()) - && Objects.equals(getCommandStatus(), that.getCommandStatus()); + && Objects.equals(getCommandStatus(), that.getCommandStatus()) + && (getCommandOffset() == that.getCommandOffset()); } @Override public int hashCode() { - return Objects.hash(getCommandId(), getCommandStatus()); + return Objects.hash(getCommandId(), getCommandStatus(), getCommandOffset()); } @Override @@ -94,6 +105,7 @@ public String toString() { return "CommandStatusEntity{" + "commandId=" + commandId + ", commandStatus=" + commandStatus + + ", commandOffset=" + String.valueOf(commandOffset) + '}'; } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 2dccdf98e59b..9fbc389429fb 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -34,6 +34,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; @@ -118,7 +119,9 @@ public QueuedCommandStatus enqueueCommand( } ); try { - commandProducer.send(new ProducerRecord<>(commandTopic, commandId, command)).get(); + final RecordMetadata recordMetadata = + commandProducer.send(new ProducerRecord<>(commandTopic, commandId, command)).get(); + status.setCommandOffset(recordMetadata.offset()); } catch (final Exception e) { commandStatusMap.remove(commandId); throw new KsqlException( diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java index c11a1f23dfdf..51a6a9e78a8e 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java @@ -28,15 +28,18 @@ public class QueuedCommandStatus { private static final CommandStatus INITIAL_STATUS = new CommandStatus( CommandStatus.Status.QUEUED, "Statement written to command topic"); + private static final long INITIAL_OFFSET = -1L; private final CommandId commandId; private volatile CommandStatus commandStatus; private final CompletableFuture future; + private long commandOffset; public QueuedCommandStatus(final CommandId commandId) { this.commandId = Objects.requireNonNull(commandId); this.commandStatus = INITIAL_STATUS; this.future = new CompletableFuture<>(); + this.commandOffset = INITIAL_OFFSET; } public CommandId getCommandId() { @@ -47,6 +50,10 @@ public CommandStatus getStatus() { return commandStatus; } + public long getCommandOffset() { + return commandOffset; + } + public CommandStatus tryWaitForFinalStatus(final Duration timeout) throws InterruptedException { try { @@ -66,4 +73,8 @@ public void setFinalStatus(final CommandStatus status) { setStatus(status); future.complete(status); } + + public void setCommandOffset(final long offset) { + commandOffset = offset; + } } \ No newline at end of file diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index 512506556905..86ea5946d8a1 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -339,7 +339,8 @@ private CommandStatusEntity distributeStatement( statementText, queuedCommandStatus.getCommandId(), queuedCommandStatus.tryWaitForFinalStatus( - Duration.ofMillis(distributedCommandResponseTimeout)) + Duration.ofMillis(distributedCommandResponseTimeout)), + queuedCommandStatus.getCommandOffset() ); } catch (final Exception e) { throw new RuntimeException(e); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index e9eefffbfefc..18ff88e69281 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -62,6 +62,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.RecordBatch; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; @@ -87,6 +88,8 @@ public class CommandStoreTest { private final Command command = new Command(statementText, Collections.emptyMap(), Collections.emptyMap()); private final Node node = mock(Node.class); + private final RecordMetadata recordMetadata = new RecordMetadata( + new TopicPartition("topic", 0), 0, 0, RecordBatch.NO_TIMESTAMP, 0L, 0, 0); @Test public void shouldHaveAllCreateCommandsInOrder() { @@ -120,12 +123,13 @@ public void shouldHaveAllCreateCommandsInOrder() { } @Test - public void shouldFailEnqueueIfCommandWithSameIdRegistered() { + public void shouldFailEnqueueIfCommandWithSameIdRegistered() throws InterruptedException, ExecutionException { final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); // Given: expect(commandProducer.send(anyObject())).andReturn(future); - replay(commandProducer); + expect(future.get()).andReturn(recordMetadata); + replay(commandProducer, future); commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); try { @@ -139,13 +143,14 @@ public void shouldFailEnqueueIfCommandWithSameIdRegistered() { } @Test - public void shouldCleanupCommandStatusOnProduceError() { + public void shouldCleanupCommandStatusOnProduceError() throws InterruptedException, ExecutionException { final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); // Given: expect(commandProducer.send(anyObject())).andThrow(new RuntimeException("oops")).times(1); expect(commandProducer.send(anyObject())).andReturn(future).times(1); - replay(commandProducer); + expect(future.get()).andReturn(recordMetadata); + replay(commandProducer, future); try { commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); fail("enqueueCommand should have raised an exception"); @@ -173,8 +178,7 @@ public void shouldEnqueueNewAfterHandlingExistingCommand() throws InterruptedExc } ).times(1); expect(commandProducer.send(anyObject(ProducerRecord.class))).andReturn(future); - future.get(); - expectLastCall().andStubReturn(null); + expect(future.get()).andReturn(recordMetadata).times(2); replay(future, commandProducer); commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); @@ -203,8 +207,7 @@ public void shouldRegisterBeforeDistributeAndReturnStatusOnGetNewCommands() return future; } ); - future.get(); - expectLastCall().andReturn(null); + expect(future.get()).andReturn(recordMetadata); replay(future, commandProducer); // When: @@ -273,11 +276,9 @@ public void shouldDistributeCommand() throws ExecutionException, InterruptedExce final Statement statement = mock(Statement.class); final Capture> recordCapture = Capture.newInstance(); - final Future future = mock(Future.class); expect(commandProducer.send(capture(recordCapture))).andReturn(future); - future.get(); - expectLastCall().andReturn(null); + expect(future.get()).andReturn(recordMetadata); replay(commandProducer, future); final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); @@ -292,6 +293,26 @@ public void shouldDistributeCommand() throws ExecutionException, InterruptedExce assertThat(record.value().getOriginalProperties(), equalTo(ksqlConfig.getAllConfigPropsWithSecretsObfuscated())); } + @Test + public void shouldIncludeTopicOffsetInSuccessfulQueuedCommandStatus() + throws InterruptedException, ExecutionException { + // Given: + final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); + + expect(commandProducer.send(anyObject(ProducerRecord.class))).andReturn(future); + expect(future.get()).andReturn(recordMetadata); + replay(commandProducer, future); + + // When: + final QueuedCommandStatus commandStatus = + commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); + + // Then: + assertThat(commandStatus.getCommandOffset(), equalTo(recordMetadata.offset())); + + verify(commandProducer, future); + } + private void setupConsumerToReturnCommand(final CommandId commandId, final Command command) { reset(commandConsumer); expect(commandConsumer.poll(anyObject(Duration.class))).andReturn( diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index ec46ab8b5147..49fe6b1a8c74 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -159,6 +159,7 @@ public void shouldInstantRegisterTopic() { // Given: final QueuedCommandStatus cmdStatus = new QueuedCommandStatus(new CommandId("TABLE", "orders", "CREATE")); + cmdStatus.setCommandOffset(0); givenCommandStore(mockCommandStore -> EasyMock.expect(mockCommandStore.enqueueCommand( @@ -174,7 +175,7 @@ public void shouldInstantRegisterTopic() { // Then: final CommandStatusEntity expectedCommandStatusEntity = new CommandStatusEntity( "REGISTER TOPIC FOO WITH (kafka_topic='bar', value_format='json');", - cmdStatus.getCommandId(), cmdStatus.getStatus()); + cmdStatus.getCommandId(), cmdStatus.getStatus(), 0); assertThat(result, is(expectedCommandStatusEntity)); } From 72418583b696f101710be42ffbce47499836bbeb Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Wed, 14 Nov 2018 11:11:24 -0800 Subject: [PATCH 02/13] accept offset to wait for in KsqlRequest. --- .../ksql/rest/client/KsqlRestClient.java | 6 +- .../ksql/rest/entity/KsqlRequest.java | 15 ++- .../rest/server/computation/CommandStore.java | 40 +++++++ .../computation/ReplayableCommandQueue.java | 3 + .../rest/server/resources/KsqlResource.java | 38 +++++++ .../ksql/rest/entity/KsqlRequestTest.java | 90 ++++++++++++--- .../ksql/rest/integration/RestApiTest.java | 10 +- .../server/computation/CommandStoreTest.java | 106 +++++++++++++++++- .../rest/server/computation/RecoveryTest.java | 8 +- .../server/resources/KsqlResourceTest.java | 77 ++++++++++++- .../resources/StreamedQueryResourceTest.java | 6 +- .../streaming/WSQueryEndpointTest.java | 11 +- 12 files changed, 369 insertions(+), 41 deletions(-) diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java index 7f317a739793..35a23a7e184e 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java @@ -124,7 +124,7 @@ public RestResponse getServerInfo() { } public RestResponse makeKsqlRequest(final String ksql) { - final KsqlRequest jsonRequest = new KsqlRequest(ksql, localProperties.toMap()); + final KsqlRequest jsonRequest = new KsqlRequest(ksql, localProperties.toMap(), null); return postRequest("ksql", jsonRequest, true, r -> r.readEntity(KsqlEntityList.class)); } @@ -137,12 +137,12 @@ public RestResponse makeStatusRequest(final String commandId) { } public RestResponse makeQueryRequest(final String ksql) { - final KsqlRequest jsonRequest = new KsqlRequest(ksql, localProperties.toMap()); + final KsqlRequest jsonRequest = new KsqlRequest(ksql, localProperties.toMap(), null); return postRequest("query", jsonRequest, false, QueryStream::new); } public RestResponse makePrintTopicRequest(final String ksql) { - final KsqlRequest jsonRequest = new KsqlRequest(ksql, localProperties.toMap()); + final KsqlRequest jsonRequest = new KsqlRequest(ksql, localProperties.toMap(), null); return postRequest("query", jsonRequest, false, r -> (InputStream)r.getEntity()); } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java index c801490c03c5..a65cbb89e79c 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.Optional; @JsonIgnoreProperties(ignoreUnknown = true) @JsonSubTypes({}) @@ -35,16 +36,19 @@ public class KsqlRequest { private final String ksql; private final Map streamsProperties; + private final Optional commandOffset; @JsonCreator public KsqlRequest( @JsonProperty("ksql") final String ksql, - @JsonProperty("streamsProperties") final Map streamsProperties + @JsonProperty("streamsProperties") final Map streamsProperties, + @JsonProperty("commandOffset") final Long commandOffset ) { this.ksql = ksql == null ? "" : ksql; this.streamsProperties = streamsProperties == null ? Collections.emptyMap() : Collections.unmodifiableMap(new HashMap<>(streamsProperties)); + this.commandOffset = commandOffset == null ? Optional.empty() : Optional.of(commandOffset); } public String getKsql() { @@ -55,6 +59,10 @@ public Map getStreamsProperties() { return coerceTypes(streamsProperties); } + public Optional getCommandOffset() { + return commandOffset; + } + @Override public boolean equals(final Object o) { if (this == o) { @@ -67,12 +75,13 @@ public boolean equals(final Object o) { final KsqlRequest that = (KsqlRequest) o; return Objects.equals(getKsql(), that.getKsql()) - && Objects.equals(getStreamsProperties(), that.getStreamsProperties()); + && Objects.equals(getStreamsProperties(), that.getStreamsProperties()) + && Objects.equals(getCommandOffset(), that.getCommandOffset()); } @Override public int hashCode() { - return Objects.hash(getKsql(), getStreamsProperties()); + return Objects.hash(getKsql(), getStreamsProperties(), getCommandOffset()); } private static Map coerceTypes(final Map streamsProperties) { diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 9fbc389429fb..4dbc7b5a72a9 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -21,14 +21,18 @@ import io.confluent.ksql.parser.tree.Statement; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.Pair; import java.io.Closeable; import java.time.Duration; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -45,7 +49,9 @@ * the beginning until now, or any new messages since then), and writing to it. */ +// CHECKSTYLE_RULES.OFF: ClassDataAbstractionCoupling public class CommandStore implements ReplayableCommandQueue, Closeable { + // CHECKSTYLE_RULES.ON: ClassDataAbstractionCoupling private static final Logger log = LoggerFactory.getLogger(CommandStore.class); @@ -56,6 +62,7 @@ public class CommandStore implements ReplayableCommandQueue, Closeable { private final Producer commandProducer; private final CommandIdAssigner commandIdAssigner; private final Map commandStatusMap; + private List, Long>> commandOffsetFutures; public CommandStore( final String commandTopic, @@ -68,6 +75,7 @@ public CommandStore( this.commandProducer = commandProducer; this.commandIdAssigner = commandIdAssigner; this.commandStatusMap = Maps.newConcurrentMap(); + this.commandOffsetFutures = new ArrayList<>(); commandConsumer.assign(Collections.singleton(new TopicPartition(commandTopic, 0))); } @@ -142,6 +150,8 @@ public QueuedCommandStatus enqueueCommand( * @return The commands that have been polled from the command topic */ public List getNewCommands() { + completeSatisfiedOffsetFutures(); + final List queuedCommands = Lists.newArrayList(); commandConsumer.poll(Duration.ofMillis(Long.MAX_VALUE)).forEach( c -> { @@ -189,6 +199,25 @@ public List getRestoreCommands() { return restoreCommands; } + public CompletableFuture getConsumerPositionFuture(final long offset) { + final CompletableFuture future = new CompletableFuture<>(); + + final long consumerPosition = getConsumerPosition(); + if (consumerPosition > offset) { + future.complete(null); + } else { + commandOffsetFutures.add(Pair.of(future, offset)); + } + return future; + } + + // returns the next offset to be consumed + private long getConsumerPosition() { + final Collection cmdTopicPartitions = getTopicPartitionsForTopic(commandTopic); + // NOTE: assumes there's only one partition + return commandConsumer.position(cmdTopicPartitions.iterator().next()); + } + private Collection getTopicPartitionsForTopic(final String topic) { final List partitionInfoList = commandConsumer.partitionsFor(topic); @@ -198,4 +227,15 @@ private Collection getTopicPartitionsForTopic(final String topic } return result; } + + private void completeSatisfiedOffsetFutures() { + final long offset = getConsumerPosition(); + commandOffsetFutures.stream() + .filter(futureOffsetPair -> futureOffsetPair.getRight() < offset) + .map(futureOffsetPair -> futureOffsetPair.getLeft()) + .forEach(future -> future.complete(null)); + commandOffsetFutures = commandOffsetFutures.stream() + .filter(futureOffsetPair -> !futureOffsetPair.getLeft().isDone()) + .collect(Collectors.toList()); + } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java index 7fb84b10b921..eaed7f732c0e 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java @@ -22,6 +22,7 @@ import java.io.Closeable; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; public interface ReplayableCommandQueue extends Closeable { QueuedCommandStatus enqueueCommand( @@ -34,4 +35,6 @@ QueuedCommandStatus enqueueCommand( List getNewCommands(); List getRestoreCommands(); + + CompletableFuture getConsumerPositionFuture(long offset); } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index 86ea5946d8a1..f1f790bf301c 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -116,6 +116,11 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -154,11 +159,19 @@ public KsqlResource( this.registerKsqlStatementTasks(); } + // CHECKSTYLE_RULES.OFF: NPathComplexity @POST public Response handleKsqlStatements(final KsqlRequest request) { + // CHECKSTYLE_RULES.ON: NPathComplexity final List parsedStatements; final KsqlEntityList result = new KsqlEntityList(); + try { + waitForCommandOffset(request.getCommandOffset()); + } catch (final RuntimeException e) { + return Errors.serverErrorForStatement(e, request.getKsql(), result); + } + try { parsedStatements = ksqlEngine.parseStatements(request.getKsql()); } catch (final ParseFailedException e) { @@ -192,6 +205,31 @@ public Response handleKsqlStatements(final KsqlRequest request) { return Response.ok(result).build(); } + private void waitForCommandOffset(final Optional commandOffset) { + if (commandOffset.isPresent()) { + final long offset = Objects.requireNonNull(commandOffset.get(), "commandOffset is null"); + final CompletableFuture future = + replayableCommandQueue.getConsumerPositionFuture(offset); + + try { + future.get(distributedCommandResponseTimeout, TimeUnit.MILLISECONDS); + } catch (final ExecutionException e) { + throw new RuntimeException( + "Error waiting for command offset of " + String.valueOf(offset), e.getCause()); + } catch (final InterruptedException e) { + throw new RuntimeException( + "Interrupted while waiting for command offset of " + String.valueOf(offset), e); + } catch (final TimeoutException e) { + throw new RuntimeException( + String.format( + "Timeout reached while waiting for command offset of %d. (Timeout: %d ms)", + offset, + distributedCommandResponseTimeout), + e); + } + } + } + // CHECKSTYLE_RULES.OFF: CyclomaticComplexity private void validateStatement( final KsqlEntityList entities, final String statementText, final Statement statement, diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/KsqlRequestTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/KsqlRequestTest.java index 194e14c016c1..7ccd06a9515c 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/KsqlRequestTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/KsqlRequestTest.java @@ -25,6 +25,7 @@ import static org.hamcrest.Matchers.nullValue; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; import com.google.common.collect.ImmutableMap; import com.google.common.testing.EqualsTester; import io.confluent.ksql.util.KsqlConfig; @@ -32,6 +33,8 @@ import java.io.IOException; import java.util.Collections; import java.util.Map; +import java.util.Optional; +import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -45,24 +48,50 @@ public class KsqlRequestTest { + "\"streamsProperties\":{" + "\"" + KsqlConfig.KSQL_SERVICE_ID_CONFIG + "\":\"some-service-id\"" + "}}"; + private static final String A_JSON_REQUEST_WITH_OFFSET = "{" + + "\"ksql\":\"sql\"," + + "\"streamsProperties\":{" + + "\"" + KsqlConfig.KSQL_SERVICE_ID_CONFIG + "\":\"some-service-id\"" + + "}," + + "\"commandOffset\":2}"; + private static final String A_JSON_REQUEST_WITH_NULL_OFFSET = "{" + + "\"ksql\":\"sql\"," + + "\"streamsProperties\":{" + + "\"" + KsqlConfig.KSQL_SERVICE_ID_CONFIG + "\":\"some-service-id\"" + + "}," + + "\"commandOffset\":null}"; private static final ImmutableMap SOME_PROPS = ImmutableMap.of( KsqlConfig.KSQL_SERVICE_ID_CONFIG, "some-service-id" ); + private static final long SOME_OFFSET = 2L; - private static final KsqlRequest A_REQUEST = new KsqlRequest("sql", SOME_PROPS); + private static final KsqlRequest A_REQUEST = new KsqlRequest("sql", SOME_PROPS, null); + private static final KsqlRequest A_REQUEST_WITH_OFFSET = + new KsqlRequest("sql", SOME_PROPS, SOME_OFFSET); + + @BeforeClass + public static void setUpClass() { + OBJECT_MAPPER.registerModule(new Jdk8Module()); + } @Rule public final ExpectedException expectedException = ExpectedException.none(); @Test public void shouldHandleNullStatement() { - assertThat(new KsqlRequest(null, SOME_PROPS).getKsql(), is("")); + assertThat(new KsqlRequest(null, SOME_PROPS, SOME_OFFSET).getKsql(), is("")); } @Test public void shouldHandleNullProps() { - assertThat(new KsqlRequest("sql", null).getStreamsProperties(), is(Collections.emptyMap())); + assertThat(new KsqlRequest("sql", null, SOME_OFFSET).getStreamsProperties(), + is(Collections.emptyMap())); + } + + @Test + public void shouldHandleNullOffset() { + assertThat(new KsqlRequest("sql", SOME_PROPS, null).getCommandOffset(), is(Optional.empty())); } @Test @@ -74,21 +103,50 @@ public void shouldDeserializeFromJson() { assertThat(request, is(A_REQUEST)); } + @Test + public void shouldDeserializeFromJsonWithOffset() { + // When: + final KsqlRequest request = deserialize(A_JSON_REQUEST_WITH_OFFSET); + + // Then: + assertThat(request, is(A_REQUEST_WITH_OFFSET)); + } + + @Test + public void shouldDeserializeFromJsonWithNullOffset() { + // When: + final KsqlRequest request = deserialize(A_JSON_REQUEST_WITH_NULL_OFFSET); + + // Then: + assertThat(request, is(A_REQUEST)); + } + @Test public void shouldSerializeToJson() { // When: final String jsonRequest = serialize(A_REQUEST); // Then: - assertThat(jsonRequest, is(A_JSON_REQUEST)); + assertThat(jsonRequest, is(A_JSON_REQUEST_WITH_NULL_OFFSET)); + } + + @Test + public void shouldSerializeToJsonWithOffset() { + // When: + final String jsonRequest = serialize(A_REQUEST_WITH_OFFSET); + + // Then: + assertThat(jsonRequest, is(A_JSON_REQUEST_WITH_OFFSET)); } @Test public void shouldImplementHashCodeAndEqualsCorrectly() { new EqualsTester() - .addEqualityGroup(new KsqlRequest("sql", SOME_PROPS), new KsqlRequest("sql", SOME_PROPS)) - .addEqualityGroup(new KsqlRequest("different-sql", SOME_PROPS)) - .addEqualityGroup(new KsqlRequest("sql", ImmutableMap.of())) + .addEqualityGroup(new KsqlRequest("sql", SOME_PROPS, SOME_OFFSET), + new KsqlRequest("sql", SOME_PROPS, SOME_OFFSET)) + .addEqualityGroup(new KsqlRequest("different-sql", SOME_PROPS, SOME_OFFSET)) + .addEqualityGroup(new KsqlRequest("sql", ImmutableMap.of(), SOME_OFFSET)) + .addEqualityGroup(new KsqlRequest("sql", SOME_PROPS, null)) .testEquals(); } @@ -111,9 +169,12 @@ public void shouldHandleShortProperties() { @Test public void shouldThrowOnInvalidPropertyValue() { // Given: - final KsqlRequest request = new KsqlRequest("sql", ImmutableMap.of( - SINK_NUMBER_OF_REPLICAS_PROPERTY, "not-parsable" - )); + final KsqlRequest request = new KsqlRequest( + "sql", + ImmutableMap.of( + SINK_NUMBER_OF_REPLICAS_PROPERTY, "not-parsable" + ), + null); expectedException.expect(KsqlException.class); expectedException.expectMessage(containsString(SINK_NUMBER_OF_REPLICAS_PROPERTY)); @@ -126,9 +187,12 @@ public void shouldThrowOnInvalidPropertyValue() { @Test public void shouldHandleNullPropertyValue() { // Given: - final KsqlRequest request = new KsqlRequest("sql", Collections.singletonMap( - KSQL_TRANSIENT_QUERY_NAME_PREFIX_CONFIG, null - )); + final KsqlRequest request = new KsqlRequest( + "sql", + Collections.singletonMap( + KSQL_TRANSIENT_QUERY_NAME_PREFIX_CONFIG, null + ), + null); // When: final Map props = request.getStreamsProperties(); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java index 264c560614fb..2da59a10dc57 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java @@ -103,9 +103,8 @@ public void setUp() { @Test public void shouldExecuteStreamingQueryWithV1ContentType() { - final KsqlRequest request = new KsqlRequest(String.format("SELECT * from %s;", - PAGE_VIEW_STREAM), - Collections.emptyMap()); + final KsqlRequest request = new KsqlRequest( + String.format("SELECT * from %s;", PAGE_VIEW_STREAM), Collections.emptyMap(), null); try (final Response response = restClient.target(serverAddress) .path("query") .request(Versions.KSQL_V1_JSON) @@ -117,9 +116,8 @@ public void shouldExecuteStreamingQueryWithV1ContentType() { @Test public void shouldExecuteStreamingQueryWithJsonContentType() { - final KsqlRequest request = new KsqlRequest(String.format("SELECT * from %s;", - PAGE_VIEW_STREAM), - Collections.emptyMap()); + final KsqlRequest request = new KsqlRequest( + String.format("SELECT * from %s;", PAGE_VIEW_STREAM), Collections.emptyMap(), null); try (final Response response = restClient.target(serverAddress) .path("query") .request(MediaType.APPLICATION_JSON_TYPE) diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index 18ff88e69281..3a18d21a36c1 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -19,7 +19,6 @@ import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.capture; import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.mock; import static org.easymock.EasyMock.niceMock; import static org.easymock.EasyMock.replay; @@ -49,6 +48,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.stream.Collectors; @@ -227,6 +227,7 @@ public void shouldFilterNullCommands() { final ConsumerRecords records = buildRecords( id, null, id, command); + expectConsumerToReturnPartitionInfo(); expect(commandConsumer.poll(anyObject())).andReturn(records); replay(commandConsumer); @@ -248,11 +249,7 @@ public void shouldFilterNullPriorCommand() { final ConsumerRecords records = buildRecords( id, null, id, command); - expect(commandConsumer.partitionsFor(COMMAND_TOPIC)).andStubReturn( - ImmutableList.of( - new PartitionInfo(COMMAND_TOPIC, 0, node, new Node[]{node}, new Node[]{node}) - ) - ); + expectConsumerToReturnPartitionInfo(); expect(commandConsumer.poll(anyObject())).andReturn(records); expect(commandConsumer.poll(anyObject())).andReturn(ConsumerRecords.empty()); replay(commandConsumer); @@ -313,14 +310,101 @@ public void shouldIncludeTopicOffsetInSuccessfulQueuedCommandStatus() verify(commandProducer, future); } + @Test + public void shouldReturnCompletedFutureIfOffsetReached() { + // Given: + givenCmdConsumerAtPosition(1); + + final CommandStore commandStore = createCommandStore(); + + // When: + final CompletableFuture future = commandStore.getConsumerPositionFuture(0); + + // Then: + assertFutureIsCompleted(future); + verify(commandConsumer); + } + + @Test + public void shouldReturnUncompletedFutureIfOffsetNotReached() { + // Given: + givenCmdConsumerAtPosition(2); + + final CommandStore commandStore = createCommandStore(); + + // When: + final CompletableFuture future = commandStore.getConsumerPositionFuture(2); + + // Then: + assertFutureIsNotCompleted(future); + verify(commandConsumer); + } + + @Test + public void shouldCompleteFutureWhenOffsetIsReached() { + // Given: + final CommandStore commandStore = createCommandStore(); + + givenCmdConsumerAtPosition(0); + final CompletableFuture future = commandStore.getConsumerPositionFuture(2); + givenCmdConsumerAtPosition(3, true); + + // When: + commandStore.getNewCommands(); + + // Then: + assertFutureIsCompleted(future); + verify(commandConsumer); + } + + @Test + public void shouldNotCompleteFutureWhenOffsetIsNotReached() { + // Given: + final CommandStore commandStore = createCommandStore(); + + givenCmdConsumerAtPosition(0); + final CompletableFuture future = commandStore.getConsumerPositionFuture(2); + givenCmdConsumerAtPosition(2, true); + + // When: + commandStore.getNewCommands(); + + // Then: + assertFutureIsNotCompleted(future); + verify(commandConsumer); + } + private void setupConsumerToReturnCommand(final CommandId commandId, final Command command) { reset(commandConsumer); expect(commandConsumer.poll(anyObject(Duration.class))).andReturn( buildRecords(commandId, command) ).times(1); + expectConsumerToReturnPartitionInfo(); + replay(commandConsumer); + } + + private void givenCmdConsumerAtPosition(long position) { + givenCmdConsumerAtPosition(position, false); + } + + private void givenCmdConsumerAtPosition(long position, boolean poll) { + reset(commandConsumer); + expectConsumerToReturnPartitionInfo(); + expect(commandConsumer.position(anyObject(TopicPartition.class))).andReturn(position); + if (poll) { + expect(commandConsumer.poll(anyObject())) + .andReturn(new ConsumerRecords<>(Collections.emptyMap())); + } replay(commandConsumer); } + private void expectConsumerToReturnPartitionInfo() { + expect(commandConsumer.partitionsFor(COMMAND_TOPIC)) + .andReturn(ImmutableList.of( + new PartitionInfo(COMMAND_TOPIC, 0, node, new Node[]{node}, new Node[]{node}) + )); + } + private CommandStore createCommandStoreThatAssignsSameId(final CommandId commandId) { final CommandIdAssigner commandIdAssigner = mock(CommandIdAssigner.class); expect(commandIdAssigner.getCommandId(anyObject())).andStubAnswer( @@ -366,4 +450,14 @@ private ConsumerRecords buildRecords(final Object ...args) { ) ); } + + private void assertFutureIsCompleted(CompletableFuture future) { + assertThat(future.isDone(), is(true)); + assertThat(future.isCancelled(), is(false)); + assertThat(future.isCompletedExceptionally(), is(false)); + } + + private void assertFutureIsNotCompleted(CompletableFuture future) { + assertThat(future.isDone(), is(false)); + } } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index 60a9626dd120..89d9afde5e41 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -67,6 +67,7 @@ import java.util.Map.Entry; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import javax.ws.rs.core.Response; import org.apache.kafka.connect.data.Schema; @@ -145,6 +146,11 @@ public List getRestoreCommands() { return restoreCommands; } + @Override + public CompletableFuture getConsumerPositionFuture(final long offset) { + return null; + } + @Override public void close() { } @@ -192,7 +198,7 @@ void executeCommands() { void submitCommands(final String ...statements) { for (final String statement : statements) { final Response response = ksqlResource.handleKsqlStatements( - new KsqlRequest(statement, Collections.emptyMap())); + new KsqlRequest(statement, Collections.emptyMap(), null)); assertThat(response.getStatus(), equalTo(200)); executeCommands(); } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 49fe6b1a8c74..3cc46fceb9af 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -89,9 +89,13 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import java.util.function.Consumer; import java.util.stream.Collectors; import javax.ws.rs.core.Response; +import junit.framework.AssertionFailedError; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.Serdes; @@ -107,6 +111,7 @@ import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; +import org.junit.runners.model.TestTimedOutException; @SuppressWarnings("unchecked") @RunWith(EasyMockRunner.class) @@ -538,7 +543,7 @@ public void shouldListPropertiesWithOverrides() { // When: final PropertiesList props = makeSingleRequest( - new KsqlRequest("list properties;", overrides), PropertiesList.class); + new KsqlRequest("list properties;", overrides, null), PropertiesList.class); // Then: assertThat(props.getProperties().get("ksql.streams.auto.offset.reset"), is("latest")); @@ -690,6 +695,67 @@ public void shouldNotIncludeSslPropertiesInListPropertiesOutput() { not(hasItems(KsqlConfig.SSL_CONFIG_NAMES.toArray(new String[0])))); } + @Test + public void shouldNotWaitIfNoCommandTopicOffsetSpecified() { + // Given: + final String sql = "LIST REGISTERED TOPICS;"; + final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), null); + + EasyMock.expect(commandStore.getConsumerPositionFuture(EasyMock.anyLong())) + .andThrow(new AssertionFailedError()).anyTimes(); + EasyMock.replay(commandStore); + + // When: + makeSingleRequest(request, KsqlTopicsList.class); + + // Then: + EasyMock.verify(commandStore); + } + + @Test + public void shouldNotWaitIfCommandTopicOffsetReached() + throws InterruptedException, ExecutionException, TimeoutException { + // Given: + final String sql = "LIST REGISTERED TOPICS;"; + final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), 2L); + + final CompletableFuture future = EasyMock.mock(CompletableFuture.class); + EasyMock.expect(future.get(EasyMock.anyLong(), EasyMock.anyObject())).andReturn(null); + EasyMock.expect(commandStore.getConsumerPositionFuture(EasyMock.anyLong())) + .andReturn(future); + EasyMock.replay(future, commandStore); + + // When: + makeSingleRequest(request, KsqlTopicsList.class); + + // Then: + EasyMock.verify(future, commandStore); + } + + @Test + public void shouldReturn5xxIfTimeoutWhileWaitingForCommandTopicOffset() + throws InterruptedException, ExecutionException, TimeoutException { + // Given: + final String sql = "LIST REGISTERED TOPICS;"; + final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), 2L); + + final CompletableFuture future = EasyMock.mock(CompletableFuture.class); + EasyMock.expect(future.get(EasyMock.anyLong(), EasyMock.anyObject())) + .andThrow(new TimeoutException()); + EasyMock.expect(commandStore.getConsumerPositionFuture(EasyMock.anyLong())) + .andReturn(future); + EasyMock.replay(future, commandStore); + + // When: + final KsqlErrorMessage result = makeFailingRequest(request, Code.INTERNAL_SERVER_ERROR); + + // Then: + assertThat(result.getErrorCode(), is(Errors.ERROR_CODE_SERVER_ERROR)); + assertThat(result.getMessage(), + containsString("Timeout reached while waiting for command offset")); + EasyMock.verify(future, commandStore); + } + @SuppressWarnings("SameParameterValue") private SourceInfo.Table sourceTable(final String name) { final KsqlTable table = (KsqlTable) ksqlResource @@ -744,9 +810,12 @@ private List createRunningQueries( } private KsqlErrorMessage makeFailingRequest(final String ksql, final Code errorCode) { + return makeFailingRequest(new KsqlRequest(ksql, Collections.emptyMap(), null), errorCode); + } + + private KsqlErrorMessage makeFailingRequest(final KsqlRequest ksqlRequest, final Code errorCode) { try { - final Response response = ksqlResource.handleKsqlStatements( - new KsqlRequest(ksql, Collections.emptyMap())); + final Response response = ksqlResource.handleKsqlStatements(ksqlRequest); assertThat(response.getStatus(), is(errorCode.getCode())); assertThat(response.getEntity(), instanceOf(KsqlErrorMessage.class)); return (KsqlErrorMessage) response.getEntity(); @@ -758,7 +827,7 @@ private KsqlErrorMessage makeFailingRequest(final String ksql, final Code errorC private T makeSingleRequest( final String sql, final Class expectedEntityType) { - return makeSingleRequest(new KsqlRequest(sql, Collections.emptyMap()), expectedEntityType); + return makeSingleRequest(new KsqlRequest(sql, Collections.emptyMap(), null), expectedEntityType); } private T makeSingleRequest( diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java index f5edc4ad670b..6d66494f1dd4 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java @@ -91,7 +91,7 @@ public void shouldReturn400OnBadStatement() throws Exception { ksqlConfig, mockKsqlEngine, mockStatementParser, DISCONNECT_CHECK_INTERVAL); final Response response = - testResource.streamQuery(new KsqlRequest(queryString, Collections.emptyMap())); + testResource.streamQuery(new KsqlRequest(queryString, Collections.emptyMap(), null)); assertThat(response.getStatus(), equalTo(Response.Status.BAD_REQUEST.getStatusCode())); assertThat(response.getEntity(), instanceOf(KsqlErrorMessage.class)); final KsqlErrorMessage errorMessage = (KsqlErrorMessage)response.getEntity(); @@ -122,7 +122,7 @@ public void shouldReturn400OnBuildMultipleQueriesError() throws Exception { ksqlConfig, mockKsqlEngine, mockStatementParser, DISCONNECT_CHECK_INTERVAL); final Response response = - testResource.streamQuery(new KsqlRequest(queryString, Collections.emptyMap())); + testResource.streamQuery(new KsqlRequest(queryString, Collections.emptyMap(), null)); assertThat(response.getStatus(), equalTo(Response.Status.BAD_REQUEST.getStatusCode())); assertThat(response.getEntity(), instanceOf(KsqlErrorMessage.class)); final KsqlErrorMessage errorMessage = (KsqlErrorMessage)response.getEntity(); @@ -210,7 +210,7 @@ public void run() { mockKsqlConfig, mockKsqlEngine, mockStatementParser, DISCONNECT_CHECK_INTERVAL); final Response response = - testResource.streamQuery(new KsqlRequest(queryString, requestStreamsProperties)); + testResource.streamQuery(new KsqlRequest(queryString, requestStreamsProperties, null)); final PipedOutputStream responseOutputStream = new EOFPipedOutputStream(); final PipedInputStream responseInputStream = new PipedInputStream(responseOutputStream, 1); final StreamingOutput responseStream = (StreamingOutput) response.getEntity(); 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 6d1df72c159a..05923f5d3773 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 @@ -10,6 +10,7 @@ import static org.mockito.Mockito.when; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap.Builder; @@ -39,6 +40,7 @@ import javax.websocket.CloseReason.CloseCodes; import javax.websocket.Session; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; @@ -53,10 +55,10 @@ public class WSQueryEndpointTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final KsqlRequest VALID_REQUEST = new KsqlRequest("test-sql", - ImmutableMap.of(KsqlConfig.KSQL_SERVICE_ID_CONFIG, "test-id")); + ImmutableMap.of(KsqlConfig.KSQL_SERVICE_ID_CONFIG, "test-id"), null); private static final KsqlRequest ANOTHER_REQUEST = new KsqlRequest("other-sql", - ImmutableMap.of()); + ImmutableMap.of(), null); private static final String VALID_VERSION = Versions.KSQL_V1_WS; private static final String[] NO_VERSION_PROPERTY = null; @@ -88,6 +90,11 @@ public class WSQueryEndpointTest { private Query query; private WSQueryEndpoint wsQueryEndpoint; + @BeforeClass + public static void setUpClass() { + OBJECT_MAPPER.registerModule(new Jdk8Module()); + } + @Before public void setUp() { query = new Query(queryBody, Optional.empty()); From baffaf87d6d26e1fa3b3b05925ed332c1ed1aca7 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Tue, 27 Nov 2018 13:14:03 -0800 Subject: [PATCH 03/13] address comments. --- .../ksql/cli/console/ConsoleTest.java | 8 +- .../ksql/rest/entity/CommandStatusEntity.java | 37 ++- .../ksql/rest/entity/KsqlRequest.java | 2 +- .../ksql/rest/server/KsqlRestApplication.java | 5 +- .../server/computation/CommandRunner.java | 4 + .../computation/CommandStatusFuture.java | 67 ++++++ .../rest/server/computation/CommandStore.java | 82 ++++--- .../server/computation/OffsetFutureStore.java | 41 ++++ .../server/computation/QueuedCommand.java | 6 +- .../computation/QueuedCommandStatus.java | 58 +---- .../computation/ReplayableCommandQueue.java | 4 +- .../server/computation/StatementExecutor.java | 22 +- .../ksql/rest/server/resources/Errors.java | 11 + .../rest/server/resources/KsqlResource.java | 36 +-- .../streaming/StreamedQueryResource.java | 10 + .../resources/streaming/WSQueryEndpoint.java | 27 ++- .../ksql/rest/util/CommandStoreUtil.java | 51 ++++ .../rest/entity/CommandStatusEntityTest.java | 58 +++++ .../server/computation/CommandStoreTest.java | 224 +++++++++--------- .../computation/OffsetFutureStoreTest.java | 101 ++++++++ .../rest/server/computation/RecoveryTest.java | 6 +- .../computation/StatementExecutorTest.java | 8 +- .../server/resources/KsqlResourceTest.java | 66 +++--- .../resources/StreamedQueryResourceTest.java | 65 +++-- .../streaming/WSQueryEndpointTest.java | 76 +++++- .../ksql/rest/util/CommandStoreUtilTest.java | 83 +++++++ 26 files changed, 797 insertions(+), 361 deletions(-) create mode 100644 ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java create mode 100644 ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/OffsetFutureStore.java create mode 100644 ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java create mode 100644 ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java create mode 100644 ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/OffsetFutureStoreTest.java create mode 100644 ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java diff --git a/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java b/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java index bd97d24c9473..0e8da810ee89 100644 --- a/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java +++ b/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java @@ -27,6 +27,7 @@ import io.confluent.ksql.TestTerminal; import io.confluent.ksql.cli.console.Console.NoOpRowCaptor; import io.confluent.ksql.rest.entity.ArgumentInfo; +import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.entity.CommandStatusEntity; import io.confluent.ksql.rest.entity.EntityQueryId; import io.confluent.ksql.rest.entity.ExecutionPlan; @@ -49,6 +50,7 @@ import io.confluent.ksql.rest.entity.StreamsList; import io.confluent.ksql.rest.entity.TablesList; import io.confluent.ksql.rest.entity.TopicDescription; +import io.confluent.ksql.rest.server.computation.CommandId; import io.confluent.ksql.rest.util.EntityUtil; import io.confluent.ksql.serde.DataSource; import io.confluent.ksql.util.SchemaUtil; @@ -127,7 +129,11 @@ public void testPrintKSqlEntityList() throws IOException { for (int i = 0; i < 5; i++) { final KsqlEntityList entityList = new KsqlEntityList(ImmutableList.of( - new CommandStatusEntity("e", "topic/1/create", "SUCCESS", "Success Message", "0"), + new CommandStatusEntity( + "e", + CommandId.fromString("topic/1/create"), + new CommandStatus(CommandStatus.Status.SUCCESS, "Success Message"), + 0), new PropertiesList("e", properties, Collections.emptyList(), Collections.emptyList()), new Queries("e", queries), new SourceDescriptionEntity( diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java index e94c6d483ece..0de83ff8d1bd 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java @@ -18,6 +18,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.confluent.ksql.rest.server.computation.CommandId; import java.util.Map; @@ -41,30 +42,20 @@ public CommandStatusEntity( this.commandOffset = commandOffset; } + @JsonCreator public CommandStatusEntity( - final String statementText, - final String commandId, - final String status, - final String message, - final String commandOffset + @JsonProperty("statementText") final String statementText, + @JsonProperty("commandId") final String commandId, + @JsonProperty("commandStatus") final Map commandStatus, + @JsonProperty("commandOffset") final Long commandOffset ) { this( statementText, CommandId.fromString(commandId), - new CommandStatus(CommandStatus.Status.valueOf(status), message), - Long.parseLong(commandOffset) - ); - } - - @SuppressWarnings("unchecked") // needs investigating - @JsonCreator - public CommandStatusEntity(final Map properties) { - this( - (String) properties.get("statementText"), - (String) properties.get("commandId"), - (String) ((Map) properties.get("commandStatus")).get("status"), - (String) ((Map) properties.get("commandStatus")).get("message"), - String.valueOf(properties.get("commandOffset")) + new CommandStatus( + CommandStatus.Status.valueOf((String) commandStatus.get("status")), + (String) commandStatus.get("message")), + commandOffset == null ? -1 : commandOffset ); } @@ -90,9 +81,9 @@ public boolean equals(final Object o) { return false; } final CommandStatusEntity that = (CommandStatusEntity) o; - return Objects.equals(getCommandId(), that.getCommandId()) - && Objects.equals(getCommandStatus(), that.getCommandStatus()) - && (getCommandOffset() == that.getCommandOffset()); + return Objects.equals(commandId, that.commandId) + && Objects.equals(commandStatus, that.commandStatus) + && (commandOffset == that.commandOffset); } @Override @@ -105,7 +96,7 @@ public String toString() { return "CommandStatusEntity{" + "commandId=" + commandId + ", commandStatus=" + commandStatus - + ", commandOffset=" + String.valueOf(commandOffset) + + ", commandOffset=" + commandOffset + '}'; } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java index a65cbb89e79c..78acdba0fcb5 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java @@ -48,7 +48,7 @@ public KsqlRequest( this.streamsProperties = streamsProperties == null ? Collections.emptyMap() : Collections.unmodifiableMap(new HashMap<>(streamsProperties)); - this.commandOffset = commandOffset == null ? Optional.empty() : Optional.of(commandOffset); + this.commandOffset = Optional.ofNullable(commandOffset); } public String getKsql() { 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 049c44a8f2ff..fc99ae21995e 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 @@ -262,7 +262,9 @@ public T getEndpointInstance(final Class endpointClass) { JsonMapper.INSTANCE.mapper, statementParser, ksqlEngine, - exec + commandRunner.getCommandStore(), + exec, + config.getLong(KsqlRestConfig.DISTRIBUTED_COMMAND_RESPONSE_TIMEOUT_MS_CONFIG) ); } @@ -366,6 +368,7 @@ public static KsqlRestApplication buildApplication( ksqlConfig, ksqlEngine, statementParser, + commandStore, Duration.ofMillis( restConfig.getLong(KsqlRestConfig.STREAMED_QUERY_DISCONNECT_CHECK_MS_CONFIG)) ); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index b8cccba0ab4c..3b764f6f3337 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -86,6 +86,10 @@ public void close() { } } + public ReplayableCommandQueue getCommandStore() { + return commandStore; + } + void fetchAndRunCommands() { final List commands = commandStore.getNewCommands(); log.trace("Found {} new writes to command topic", commands.size()); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java new file mode 100644 index 000000000000..db82140269c0 --- /dev/null +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java @@ -0,0 +1,67 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.computation; + +import io.confluent.ksql.rest.entity.CommandStatus; +import java.time.Duration; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class CommandStatusFuture { + private static final CommandStatus INITIAL_STATUS = new CommandStatus( + CommandStatus.Status.QUEUED, "Statement written to command topic"); + + private final CommandId commandId; + private volatile CommandStatus commandStatus; + private final CompletableFuture future; + + public CommandStatusFuture(final CommandId commandId) { + this.commandId = Objects.requireNonNull(commandId, "commandId cannot be null"); + this.commandStatus = INITIAL_STATUS; + this.future = new CompletableFuture<>(); + } + + public CommandId getCommandId() { + return commandId; + } + + public CommandStatus getStatus() { + return commandStatus; + } + + public CommandStatus tryWaitForFinalStatus(final Duration timeout) throws InterruptedException { + try { + return future.get(timeout.toMillis(), TimeUnit.MILLISECONDS); + } catch (final ExecutionException e) { + throw new RuntimeException("Error executing command " + commandId, e.getCause()); + } catch (final TimeoutException e) { + return commandStatus; + } + } + + public void setStatus(final CommandStatus status) { + this.commandStatus = Objects.requireNonNull(status); + } + + public void setFinalStatus(final CommandStatus status) { + setStatus(status); + future.complete(status); + } +} diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 4dbc7b5a72a9..24c3d5057b90 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -21,10 +21,8 @@ import io.confluent.ksql.parser.tree.Statement; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; -import io.confluent.ksql.util.Pair; import java.io.Closeable; import java.time.Duration; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -32,7 +30,9 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -58,26 +58,39 @@ public class CommandStore implements ReplayableCommandQueue, Closeable { private static final Duration POLLING_TIMEOUT_FOR_COMMAND_TOPIC = Duration.ofMillis(5000); private final String commandTopic; + private final TopicPartition topicPartition; private final Consumer commandConsumer; private final Producer commandProducer; private final CommandIdAssigner commandIdAssigner; - private final Map commandStatusMap; - private List, Long>> commandOffsetFutures; + private final Map commandStatusMap; + private final OffsetFutureStore offsetFutureStore; public CommandStore( final String commandTopic, final Consumer commandConsumer, final Producer commandProducer, final CommandIdAssigner commandIdAssigner + ) { + this( + commandTopic,commandConsumer, commandProducer, commandIdAssigner, new OffsetFutureStore()); + } + + CommandStore( + final String commandTopic, + final Consumer commandConsumer, + final Producer commandProducer, + final CommandIdAssigner commandIdAssigner, + final OffsetFutureStore offsetFutureStore ) { this.commandTopic = commandTopic; + this.topicPartition = new TopicPartition(commandTopic, 0); this.commandConsumer = commandConsumer; this.commandProducer = commandProducer; this.commandIdAssigner = commandIdAssigner; this.commandStatusMap = Maps.newConcurrentMap(); - this.commandOffsetFutures = new ArrayList<>(); + this.offsetFutureStore = offsetFutureStore; - commandConsumer.assign(Collections.singleton(new TopicPartition(commandTopic, 0))); + commandConsumer.assign(Collections.singleton(topicPartition)); } /** @@ -110,12 +123,11 @@ public QueuedCommandStatus enqueueCommand( statementString, overwriteProperties, ksqlConfig.getAllConfigPropsWithSecretsObfuscated()); - final QueuedCommandStatus status = new QueuedCommandStatus(commandId); - this.commandStatusMap.compute( + final CommandStatusFuture statusFuture = this.commandStatusMap.compute( commandId, (k, v) -> { if (v == null) { - return status; + return new CommandStatusFuture(commandId); } // We should fail registration if a future is already registered, to prevent // a caller from receiving a future for a different statement. @@ -129,7 +141,7 @@ public QueuedCommandStatus enqueueCommand( try { final RecordMetadata recordMetadata = commandProducer.send(new ProducerRecord<>(commandTopic, commandId, command)).get(); - status.setCommandOffset(recordMetadata.offset()); + return new QueuedCommandStatus(recordMetadata.offset(), statusFuture); } catch (final Exception e) { commandStatusMap.remove(commandId); throw new KsqlException( @@ -141,7 +153,6 @@ public QueuedCommandStatus enqueueCommand( e ); } - return status; } /** @@ -199,23 +210,34 @@ public List getRestoreCommands() { return restoreCommands; } - public CompletableFuture getConsumerPositionFuture(final long offset) { - final CompletableFuture future = new CompletableFuture<>(); - - final long consumerPosition = getConsumerPosition(); + @Override + public void ensureConsumedUpThrough(final long offset, final long timeout) + throws TimeoutException { + final long consumerPosition = getNextConsumerOffset(); if (consumerPosition > offset) { - future.complete(null); - } else { - commandOffsetFutures.add(Pair.of(future, offset)); + return; + } + + final CompletableFuture future = offsetFutureStore.getFutureForOffset(offset); + try { + future.get(timeout, TimeUnit.MILLISECONDS); + } catch (final ExecutionException e) { + throw new RuntimeException( + "Error waiting for command offset of " + offset, e.getCause()); + } catch (final InterruptedException e) { + throw new RuntimeException( + "Interrupted while waiting for command offset of " + offset, e); + } catch (final TimeoutException e) { + throw new TimeoutException( + String.format( + "Timeout reached while waiting for command offset of %d. (Timeout: %d ms)", + offset, + timeout)); } - return future; } - // returns the next offset to be consumed - private long getConsumerPosition() { - final Collection cmdTopicPartitions = getTopicPartitionsForTopic(commandTopic); - // NOTE: assumes there's only one partition - return commandConsumer.position(cmdTopicPartitions.iterator().next()); + private long getNextConsumerOffset() { + return commandConsumer.position(topicPartition); } private Collection getTopicPartitionsForTopic(final String topic) { @@ -229,13 +251,7 @@ private Collection getTopicPartitionsForTopic(final String topic } private void completeSatisfiedOffsetFutures() { - final long offset = getConsumerPosition(); - commandOffsetFutures.stream() - .filter(futureOffsetPair -> futureOffsetPair.getRight() < offset) - .map(futureOffsetPair -> futureOffsetPair.getLeft()) - .forEach(future -> future.complete(null)); - commandOffsetFutures = commandOffsetFutures.stream() - .filter(futureOffsetPair -> !futureOffsetPair.getLeft().isDone()) - .collect(Collectors.toList()); + final long consumerPosition = getNextConsumerOffset(); + offsetFutureStore.completeFuturesUpToOffset(consumerPosition); } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/OffsetFutureStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/OffsetFutureStore.java new file mode 100644 index 000000000000..8bc6f79d109b --- /dev/null +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/OffsetFutureStore.java @@ -0,0 +1,41 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.computation; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; + +public class OffsetFutureStore { + private final ConcurrentHashMap> offsetFutures; + + public OffsetFutureStore() { + offsetFutures = new ConcurrentHashMap<>(8, 0.9f, 1); + } + + public CompletableFuture getFutureForOffset(final long offset) { + return offsetFutures.computeIfAbsent(offset, k -> new CompletableFuture<>()); + } + + public void completeFuturesUpToOffset(final long offset) { + offsetFutures.keySet().stream() + .filter(k -> k < offset) + .forEach(k -> { + offsetFutures.get(k).complete(null); + offsetFutures.remove(k); + }); + } +} diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommand.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommand.java index e633fb671596..2c7c8192069d 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommand.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommand.java @@ -22,11 +22,11 @@ public class QueuedCommand { private final CommandId commandId; private final Command command; - private final Optional status; + private final Optional status; public QueuedCommand(final CommandId commandId, final Command command, - final Optional status) { + final Optional status) { this.commandId = Objects.requireNonNull(commandId); this.command = Objects.requireNonNull(command); this.status = Objects.requireNonNull(status); @@ -40,7 +40,7 @@ public CommandId getCommandId() { return commandId; } - public Optional getStatus() { + public Optional getStatus() { return status; } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java index 51a6a9e78a8e..88cd8b86e72b 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java @@ -19,62 +19,30 @@ import io.confluent.ksql.rest.entity.CommandStatus; import java.time.Duration; -import java.util.Objects; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; public class QueuedCommandStatus { - private static final CommandStatus INITIAL_STATUS = new CommandStatus( - CommandStatus.Status.QUEUED, "Statement written to command topic"); - private static final long INITIAL_OFFSET = -1L; + private final CommandStatusFuture commandStatusFuture; + private final long commandOffset; - private final CommandId commandId; - private volatile CommandStatus commandStatus; - private final CompletableFuture future; - private long commandOffset; - - public QueuedCommandStatus(final CommandId commandId) { - this.commandId = Objects.requireNonNull(commandId); - this.commandStatus = INITIAL_STATUS; - this.future = new CompletableFuture<>(); - this.commandOffset = INITIAL_OFFSET; + public QueuedCommandStatus( + final long commandOffset, final CommandStatusFuture commandStatusFuture) { + this.commandOffset = commandOffset; + this.commandStatusFuture = commandStatusFuture; } - public CommandId getCommandId() { - return commandId; + public CommandStatus getStatus() { + return commandStatusFuture.getStatus(); } - public CommandStatus getStatus() { - return commandStatus; + public CommandId getCommandId() { + return commandStatusFuture.getCommandId(); } public long getCommandOffset() { return commandOffset; } - public CommandStatus tryWaitForFinalStatus(final Duration timeout) - throws InterruptedException { - try { - return future.get(timeout.toMillis(), TimeUnit.MILLISECONDS); - } catch (final ExecutionException e) { - throw new RuntimeException("Error executing command " + commandId, e.getCause()); - } catch (final TimeoutException e) { - return commandStatus; - } - } - - public void setStatus(final CommandStatus status) { - this.commandStatus = Objects.requireNonNull(status); - } - - public void setFinalStatus(final CommandStatus status) { - setStatus(status); - future.complete(status); - } - - public void setCommandOffset(final long offset) { - commandOffset = offset; + public CommandStatus tryWaitForFinalStatus(final Duration timeout) throws InterruptedException { + return commandStatusFuture.tryWaitForFinalStatus(timeout); } -} \ No newline at end of file +} diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java index eaed7f732c0e..2f213afa5e2f 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java @@ -22,7 +22,7 @@ import java.io.Closeable; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; public interface ReplayableCommandQueue extends Closeable { QueuedCommandStatus enqueueCommand( @@ -36,5 +36,5 @@ QueuedCommandStatus enqueueCommand( List getRestoreCommands(); - CompletableFuture getConsumerPositionFuture(long offset); + void ensureConsumedUpThrough(long offset, long timeout) throws TimeoutException; } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/StatementExecutor.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/StatementExecutor.java index 126706f521fd..b7a234f8de77 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/StatementExecutor.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/StatementExecutor.java @@ -126,17 +126,17 @@ public Optional getStatus(final CommandId statementId) { } public void putStatus(final CommandId commandId, - final Optional queuedCommandStatus, + final Optional commandStatusFuture, final CommandStatus status) { statusStore.put(commandId, status); - queuedCommandStatus.ifPresent(s -> s.setStatus(status)); + commandStatusFuture.ifPresent(s -> s.setStatus(status)); } public void putFinalStatus(final CommandId commandId, - final Optional queuedCommandStatus, + final Optional commandStatusFuture, final CommandStatus status) { statusStore.put(commandId, status); - queuedCommandStatus.ifPresent(s -> s.setFinalStatus(status)); + commandStatusFuture.ifPresent(s -> s.setFinalStatus(status)); } /** @@ -149,7 +149,7 @@ public void putFinalStatus(final CommandId commandId, void handleStatementWithTerminatedQueries( final Command command, final CommandId commandId, - final Optional queuedCommandStatus, + final Optional commandStatusFuture, final Mode mode ) { try { @@ -157,23 +157,23 @@ void handleStatementWithTerminatedQueries( maybeTerminateQueryForLegacyDropCommand(commandId, command); putStatus( commandId, - queuedCommandStatus, + commandStatusFuture, new CommandStatus(CommandStatus.Status.PARSING, "Parsing statement")); final Statement statement = statementParser.parseSingleStatement(statementString); putStatus( commandId, - queuedCommandStatus, + commandStatusFuture, new CommandStatus(CommandStatus.Status.EXECUTING, "Executing statement") ); executeStatement( - statement, command, commandId, queuedCommandStatus, mode); + statement, command, commandId, commandStatusFuture, mode); } catch (final KsqlException exception) { log.error("Failed to handle: " + command, exception); final CommandStatus errorStatus = new CommandStatus( CommandStatus.Status.ERROR, ExceptionUtil.stackTraceToString(exception) ); - putFinalStatus(commandId, queuedCommandStatus, errorStatus); + putFinalStatus(commandId, commandStatusFuture, errorStatus); } } @@ -181,7 +181,7 @@ private void executeStatement( final Statement statement, final Command command, final CommandId commandId, - final Optional queuedCommandStatus, + final Optional commandStatusFuture, final Mode mode ) { final String statementStr = command.getStatement(); @@ -222,7 +222,7 @@ private void executeStatement( CommandStatus.Status.SUCCESS, result != null ? result.getMessage() : successMessage ); - putFinalStatus(commandId, queuedCommandStatus, successStatus); + putFinalStatus(commandId, commandStatusFuture, successStatus); } private void handleRunScript(final Command command) { diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/Errors.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/Errors.java index fff9eb953926..6a9e03b02416 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/Errors.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/Errors.java @@ -20,6 +20,7 @@ import static javax.ws.rs.core.Response.Status.FORBIDDEN; import static javax.ws.rs.core.Response.Status.INTERNAL_SERVER_ERROR; import static javax.ws.rs.core.Response.Status.NOT_FOUND; +import static javax.ws.rs.core.Response.Status.SERVICE_UNAVAILABLE; import static javax.ws.rs.core.Response.Status.UNAUTHORIZED; import io.confluent.ksql.rest.entity.KsqlEntityList; @@ -40,6 +41,9 @@ public final class Errors { public static final int ERROR_CODE_NOT_FOUND = toErrorCode(NOT_FOUND.getStatusCode()); + public static final int ERROR_CODE_COMMAND_QUEUE_CATCHUP_TIMEOUT = + toErrorCode(SERVICE_UNAVAILABLE.getStatusCode()) + 1; + private Errors() { } @@ -109,4 +113,11 @@ public static Response serverErrorForStatement( .entity(new KsqlStatementErrorMessage(ERROR_CODE_SERVER_ERROR, t, statementText, entities)) .build(); } + + public static Response commandQueueCatchUpTimeout(final String msg) { + return Response + .status(SERVICE_UNAVAILABLE) + .entity(new KsqlErrorMessage(ERROR_CODE_COMMAND_QUEUE_CATCHUP_TIMEOUT, msg)) + .build(); + } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index f1f790bf301c..ba1b653fe626 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -97,6 +97,7 @@ import io.confluent.ksql.rest.server.KsqlRestApplication; import io.confluent.ksql.rest.server.computation.QueuedCommandStatus; import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; +import io.confluent.ksql.rest.util.CommandStoreUtil; import io.confluent.ksql.serde.DataSource; import io.confluent.ksql.util.AvroUtil; import io.confluent.ksql.util.KafkaConsumerGroupClient; @@ -116,11 +117,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -167,7 +163,10 @@ public Response handleKsqlStatements(final KsqlRequest request) { final KsqlEntityList result = new KsqlEntityList(); try { - waitForCommandOffset(request.getCommandOffset()); + CommandStoreUtil.httpWaitForCommandOffset( + replayableCommandQueue, request, distributedCommandResponseTimeout); + } catch (final KsqlRestException e) { + throw e; } catch (final RuntimeException e) { return Errors.serverErrorForStatement(e, request.getKsql(), result); } @@ -205,31 +204,6 @@ public Response handleKsqlStatements(final KsqlRequest request) { return Response.ok(result).build(); } - private void waitForCommandOffset(final Optional commandOffset) { - if (commandOffset.isPresent()) { - final long offset = Objects.requireNonNull(commandOffset.get(), "commandOffset is null"); - final CompletableFuture future = - replayableCommandQueue.getConsumerPositionFuture(offset); - - try { - future.get(distributedCommandResponseTimeout, TimeUnit.MILLISECONDS); - } catch (final ExecutionException e) { - throw new RuntimeException( - "Error waiting for command offset of " + String.valueOf(offset), e.getCause()); - } catch (final InterruptedException e) { - throw new RuntimeException( - "Interrupted while waiting for command offset of " + String.valueOf(offset), e); - } catch (final TimeoutException e) { - throw new RuntimeException( - String.format( - "Timeout reached while waiting for command offset of %d. (Timeout: %d ms)", - offset, - distributedCommandResponseTimeout), - e); - } - } - } - // CHECKSTYLE_RULES.OFF: CyclomaticComplexity private void validateStatement( final KsqlEntityList entities, final String statementText, final Statement statement, 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 68004d3fe60d..ed76d6e6436b 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 @@ -24,8 +24,10 @@ import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.entity.Versions; import io.confluent.ksql.rest.server.StatementParser; +import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; import io.confluent.ksql.rest.server.resources.Errors; import io.confluent.ksql.rest.server.resources.KsqlRestException; +import io.confluent.ksql.rest.util.CommandStoreUtil; import io.confluent.ksql.rest.util.JsonMapper; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; @@ -50,6 +52,7 @@ public class StreamedQueryResource { private final KsqlConfig ksqlConfig; private final KsqlEngine ksqlEngine; private final StatementParser statementParser; + private final ReplayableCommandQueue replayableCommandQueue; private final Duration disconnectCheckInterval; private final ObjectMapper objectMapper; @@ -57,11 +60,14 @@ public StreamedQueryResource( final KsqlConfig ksqlConfig, final KsqlEngine ksqlEngine, final StatementParser statementParser, + final ReplayableCommandQueue replayableCommandQueue, final Duration disconnectCheckInterval ) { this.ksqlConfig = ksqlConfig; this.ksqlEngine = ksqlEngine; this.statementParser = statementParser; + this.replayableCommandQueue = + Objects.requireNonNull(replayableCommandQueue, "replayableCommandQueue"); this.disconnectCheckInterval = Objects.requireNonNull(disconnectCheckInterval, "disconnectCheckInterval"); this.objectMapper = JsonMapper.INSTANCE.mapper; @@ -76,7 +82,11 @@ public Response streamQuery(final KsqlRequest request) throws Exception { } try { + CommandStoreUtil.httpWaitForCommandOffset( + replayableCommandQueue, request, disconnectCheckInterval.toMillis()); statement = statementParser.parseSingleStatement(ksql); + } catch (final KsqlRestException e) { + throw e; } catch (IllegalArgumentException | KsqlException e) { return Errors.badRequest(e); } 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 cf3a47f45540..1a6c540d72dc 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 @@ -28,6 +28,8 @@ import io.confluent.ksql.rest.entity.StreamedRow; import io.confluent.ksql.rest.entity.Versions; import io.confluent.ksql.rest.server.StatementParser; +import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; +import io.confluent.ksql.rest.util.CommandStoreUtil; import io.confluent.ksql.util.HandlerMaps; import io.confluent.ksql.util.HandlerMaps.ClassHandlerMap2; import io.confluent.ksql.util.KsqlConfig; @@ -35,6 +37,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeoutException; import javax.websocket.CloseReason; import javax.websocket.CloseReason.CloseCodes; import javax.websocket.EndpointConfig; @@ -63,9 +66,11 @@ public class WSQueryEndpoint { private final ObjectMapper mapper; private final StatementParser statementParser; private final KsqlEngine ksqlEngine; + private final ReplayableCommandQueue replayableCommandQueue; private final ListeningScheduledExecutorService exec; private final QueryPublisher queryPublisher; private final PrintTopicPublisher topicPublisher; + private final long timeout; private WebSocketSubscriber subscriber; @@ -74,10 +79,13 @@ public WSQueryEndpoint( final ObjectMapper mapper, final StatementParser statementParser, final KsqlEngine ksqlEngine, - final ListeningScheduledExecutorService exec + final ReplayableCommandQueue replayableCommandQueue, + final ListeningScheduledExecutorService exec, + final long commandQueueCatchupTimeout ) { - this(ksqlConfig, mapper, statementParser, ksqlEngine, exec, - WSQueryEndpoint::startQueryPublisher, WSQueryEndpoint::startPrintPublisher); + this(ksqlConfig, mapper, statementParser, ksqlEngine, replayableCommandQueue, exec, + WSQueryEndpoint::startQueryPublisher, WSQueryEndpoint::startPrintPublisher, + commandQueueCatchupTimeout); } WSQueryEndpoint( @@ -85,17 +93,22 @@ public WSQueryEndpoint( final ObjectMapper mapper, final StatementParser statementParser, final KsqlEngine ksqlEngine, + final ReplayableCommandQueue replayableCommandQueue, final ListeningScheduledExecutorService exec, final QueryPublisher queryPublisher, - final PrintTopicPublisher topicPublisher + final PrintTopicPublisher topicPublisher, + final long timeout ) { this.ksqlConfig = Objects.requireNonNull(ksqlConfig, "ksqlConfig"); this.mapper = Objects.requireNonNull(mapper, "mapper"); this.statementParser = Objects.requireNonNull(statementParser, "statementParser"); this.ksqlEngine = Objects.requireNonNull(ksqlEngine, "ksqlEngine"); + this.replayableCommandQueue = + Objects.requireNonNull(replayableCommandQueue, "replayableCommandQueue"); this.exec = Objects.requireNonNull(exec, "exec"); this.queryPublisher = Objects.requireNonNull(queryPublisher, "queryPublisher"); this.topicPublisher = Objects.requireNonNull(topicPublisher, "topicPublisher"); + this.timeout = timeout; } @SuppressWarnings("unused") @@ -107,12 +120,16 @@ public void onOpen(final Session session, final EndpointConfig unused) { validateVersion(session); final KsqlRequest request = parseRequest(session); + CommandStoreUtil.waitForCommandOffset(replayableCommandQueue, request, timeout); + final Statement statement = parseStatement(request); HANDLER_MAP .getOrDefault(statement.getClass(), WSQueryEndpoint::handleUnsupportedStatement) .handle(this, new SessionAndRequest(session, request), statement); - + } catch (final TimeoutException e) { + log.debug("Timeout while processing request", e); + SessionUtil.closeSilently(session, CloseCodes.TRY_AGAIN_LATER, e.getMessage()); } catch (final Exception e) { log.debug("Error processing request", e); SessionUtil.closeSilently(session, CloseCodes.CANNOT_ACCEPT, e.getMessage()); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java new file mode 100644 index 000000000000..3bd89646d6e3 --- /dev/null +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java @@ -0,0 +1,51 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.util; + +import io.confluent.ksql.rest.entity.KsqlRequest; +import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; +import io.confluent.ksql.rest.server.resources.Errors; +import io.confluent.ksql.rest.server.resources.KsqlRestException; +import java.util.Optional; +import java.util.concurrent.TimeoutException; + +public final class CommandStoreUtil { + private CommandStoreUtil() { + } + + public static void httpWaitForCommandOffset( + final ReplayableCommandQueue replayableCommandQueue, + final KsqlRequest request, + final long timeout) { + try { + waitForCommandOffset(replayableCommandQueue, request, timeout); + } catch (final TimeoutException e) { + throw new KsqlRestException(Errors.commandQueueCatchUpTimeout(e.getMessage())); + } + } + + public static void waitForCommandOffset( + final ReplayableCommandQueue replayableCommandQueue, + final KsqlRequest request, + final long timeout) throws TimeoutException { + final Optional commandOffset = request.getCommandOffset(); + if (commandOffset.isPresent()) { + final long offset = commandOffset.get(); + replayableCommandQueue.ensureConsumedUpThrough(offset, timeout); + } + } +} diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java new file mode 100644 index 000000000000..9368f64fbab9 --- /dev/null +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java @@ -0,0 +1,58 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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 org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.confluent.ksql.rest.server.computation.CommandId; +import org.junit.Test; + +public class CommandStatusEntityTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final String JSON_ENTITY = "{" + + "\"@type\":\"currentStatus\"," + + "\"statementText\":\"sql\"," + + "\"commandId\":\"topic/1/create\"," + + "\"commandStatus\":{" + + "\"status\":\"SUCCESS\"," + + "\"message\":\"some success message\"" + + "}," + + "\"commandOffset\":0" + + "}"; + + private static final CommandStatusEntity ENTITY = new CommandStatusEntity( + "sql", + CommandId.fromString("topic/1/create"), + new CommandStatus(CommandStatus.Status.SUCCESS, "some success message"), + 0); + + @Test + public void shouldSerializeToJson() throws Exception { + final String json = OBJECT_MAPPER.writeValueAsString(ENTITY); + assertThat(json, is(JSON_ENTITY)); + } + + @Test + public void shouldDeserializeFromJson() throws Exception { + final CommandStatusEntity entity = + OBJECT_MAPPER.readValue(JSON_ENTITY, CommandStatusEntity.class); + assertThat(entity, is(ENTITY)); + } +} \ No newline at end of file diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index 3a18d21a36c1..6a4989433876 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -19,6 +19,7 @@ import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.capture; import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.mock; import static org.easymock.EasyMock.niceMock; import static org.easymock.EasyMock.replay; @@ -49,8 +50,9 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -66,6 +68,7 @@ import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -77,20 +80,35 @@ public class CommandStoreTest { private static final String COMMAND_TOPIC = "command"; private static final KsqlConfig KSQL_CONFIG = new KsqlConfig(Collections.emptyMap()); private static final Map OVERRIDE_PROPERTIES = Collections.emptyMap(); + private static final long TIMEOUT = 1000; private final Consumer commandConsumer = niceMock(Consumer.class); private final Producer commandProducer = mock(Producer.class); + private CommandIdAssigner commandIdAssigner = + new CommandIdAssigner(new MetaStoreImpl(new InternalFunctionRegistry())); + private final OffsetFutureStore offsetFutureStore = mock(OffsetFutureStore.class); + private final CompletableFuture future = niceMock(CompletableFuture.class); private final String statementText = "test-statement"; private final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); private final Statement statement = mock(Statement.class); - private final Future future = niceMock(Future.class); + private final Future recordMetadataFuture = niceMock(Future.class); private final Command command = new Command(statementText, Collections.emptyMap(), Collections.emptyMap()); private final Node node = mock(Node.class); private final RecordMetadata recordMetadata = new RecordMetadata( new TopicPartition("topic", 0), 0, 0, RecordBatch.NO_TIMESTAMP, 0L, 0, 0); + private CommandStore commandStore; + + @Before + public void setUp() throws Exception { + expect(recordMetadataFuture.get()).andReturn(recordMetadata); + replay(recordMetadataFuture); + + setUpCommandStore(); + } + @Test public void shouldHaveAllCreateCommandsInOrder() { final CommandId createId = new CommandId(CommandId.Type.TABLE, "one", CommandId.Action.CREATE); @@ -115,21 +133,19 @@ public void shouldHaveAllCreateCommandsInOrder() { .andReturn(new ConsumerRecords<>(Collections.emptyMap())); EasyMock.replay(commandConsumer); - final CommandStore command = createCommandStore(); - final List> commands = getPriorCommands(command); + final List> commands = getPriorCommands(commandStore); assertThat(commands, equalTo(Arrays.asList(new Pair<>(createId, originalCommand), new Pair<>(dropId, dropCommand), new Pair<>(createId, latestCommand)))); } @Test - public void shouldFailEnqueueIfCommandWithSameIdRegistered() throws InterruptedException, ExecutionException { - final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); + public void shouldFailEnqueueIfCommandWithSameIdRegistered() { + givenCommandStoreThatAssignsSameId(commandId); // Given: - expect(commandProducer.send(anyObject())).andReturn(future); - expect(future.get()).andReturn(recordMetadata); - replay(commandProducer, future); + expect(commandProducer.send(anyObject())).andReturn(recordMetadataFuture); + replay(commandProducer); commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); try { @@ -143,14 +159,13 @@ public void shouldFailEnqueueIfCommandWithSameIdRegistered() throws InterruptedE } @Test - public void shouldCleanupCommandStatusOnProduceError() throws InterruptedException, ExecutionException { - final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); + public void shouldCleanupCommandStatusOnProduceError() { + givenCommandStoreThatAssignsSameId(commandId); // Given: expect(commandProducer.send(anyObject())).andThrow(new RuntimeException("oops")).times(1); - expect(commandProducer.send(anyObject())).andReturn(future).times(1); - expect(future.get()).andReturn(recordMetadata); - replay(commandProducer, future); + expect(commandProducer.send(anyObject())).andReturn(recordMetadataFuture).times(1); + replay(commandProducer); try { commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); fail("enqueueCommand should have raised an exception"); @@ -166,33 +181,33 @@ public void shouldCleanupCommandStatusOnProduceError() throws InterruptedExcepti } @Test - public void shouldEnqueueNewAfterHandlingExistingCommand() throws InterruptedException, ExecutionException { - final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); + public void shouldEnqueueNewAfterHandlingExistingCommand() throws Exception { + givenCommandStoreThatAssignsSameId(commandId); // Given: setupConsumerToReturnCommand(commandId, command); expect(commandProducer.send(anyObject(ProducerRecord.class))).andAnswer( () -> { commandStore.getNewCommands(); - return future; + return recordMetadataFuture; } ).times(1); - expect(commandProducer.send(anyObject(ProducerRecord.class))).andReturn(future); - expect(future.get()).andReturn(recordMetadata).times(2); - replay(future, commandProducer); + expect(commandProducer.send(anyObject(ProducerRecord.class))).andReturn(recordMetadataFuture); + reset(recordMetadataFuture); + expect(recordMetadataFuture.get()).andReturn(recordMetadata).times(2); + replay(recordMetadataFuture, commandProducer); commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); // When: commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); // Then: - verify(future, commandProducer, commandConsumer); + verify(recordMetadataFuture, commandProducer, commandConsumer); } @Test - public void shouldRegisterBeforeDistributeAndReturnStatusOnGetNewCommands() - throws ExecutionException, InterruptedException { - final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); + public void shouldRegisterBeforeDistributeAndReturnStatusOnGetNewCommands() { + givenCommandStoreThatAssignsSameId(commandId); // Given: setupConsumerToReturnCommand(commandId, command); @@ -204,18 +219,17 @@ public void shouldRegisterBeforeDistributeAndReturnStatusOnGetNewCommands() assertThat( queuedCommand.getStatus().get().getStatus().getStatus(), equalTo(CommandStatus.Status.QUEUED)); - return future; + return recordMetadataFuture; } ); - expect(future.get()).andReturn(recordMetadata); - replay(future, commandProducer); + replay(commandProducer); // When: commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); // Then: // verifying the commandProducer also verifies the assertions in its IAnswer were run - verify(future, commandProducer, commandConsumer); + verify(recordMetadataFuture, commandProducer, commandConsumer); } @Test @@ -227,12 +241,11 @@ public void shouldFilterNullCommands() { final ConsumerRecords records = buildRecords( id, null, id, command); - expectConsumerToReturnPartitionInfo(); expect(commandConsumer.poll(anyObject())).andReturn(records); replay(commandConsumer); // When: - final List commands = createCommandStore().getNewCommands(); + final List commands = commandStore.getNewCommands(); // Then: assertThat(commands, hasSize(1)); @@ -249,13 +262,16 @@ public void shouldFilterNullPriorCommand() { final ConsumerRecords records = buildRecords( id, null, id, command); - expectConsumerToReturnPartitionInfo(); + expect(commandConsumer.partitionsFor(COMMAND_TOPIC)) + .andReturn(ImmutableList.of( + new PartitionInfo(COMMAND_TOPIC, 0, node, new Node[]{node}, new Node[]{node}) + )); expect(commandConsumer.poll(anyObject())).andReturn(records); expect(commandConsumer.poll(anyObject())).andReturn(ConsumerRecords.empty()); replay(commandConsumer); // When: - final List commands = createCommandStore().getRestoreCommands(); + final List commands = commandStore.getRestoreCommands(); // Then: assertThat(commands, hasSize(1)); @@ -264,7 +280,7 @@ public void shouldFilterNullPriorCommand() { } @Test - public void shouldDistributeCommand() throws ExecutionException, InterruptedException { + public void shouldDistributeCommand() { final KsqlConfig ksqlConfig = new KsqlConfig( Collections.singletonMap(KsqlConfig.KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG, "foo")); final Map overrideProperties = Collections.singletonMap( @@ -274,14 +290,13 @@ public void shouldDistributeCommand() throws ExecutionException, InterruptedExce final Statement statement = mock(Statement.class); final Capture> recordCapture = Capture.newInstance(); - expect(commandProducer.send(capture(recordCapture))).andReturn(future); - expect(future.get()).andReturn(recordMetadata); - replay(commandProducer, future); + expect(commandProducer.send(capture(recordCapture))).andReturn(recordMetadataFuture); + replay(commandProducer); - final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); + givenCommandStoreThatAssignsSameId(commandId); commandStore.enqueueCommand(statementText, statement, ksqlConfig, overrideProperties); - verify(commandProducer, future); + verify(commandProducer, recordMetadataFuture); final ProducerRecord record = recordCapture.getValue(); assertThat(record.key(), equalTo(commandId)); @@ -291,14 +306,12 @@ public void shouldDistributeCommand() throws ExecutionException, InterruptedExce } @Test - public void shouldIncludeTopicOffsetInSuccessfulQueuedCommandStatus() - throws InterruptedException, ExecutionException { + public void shouldIncludeTopicOffsetInSuccessfulQueuedCommandStatus() { // Given: - final CommandStore commandStore = createCommandStoreThatAssignsSameId(commandId); + givenCommandStoreThatAssignsSameId(commandId); - expect(commandProducer.send(anyObject(ProducerRecord.class))).andReturn(future); - expect(future.get()).andReturn(recordMetadata); - replay(commandProducer, future); + expect(commandProducer.send(anyObject(ProducerRecord.class))).andReturn(recordMetadataFuture); + replay(commandProducer); // When: final QueuedCommandStatus commandStatus = @@ -307,71 +320,75 @@ public void shouldIncludeTopicOffsetInSuccessfulQueuedCommandStatus() // Then: assertThat(commandStatus.getCommandOffset(), equalTo(recordMetadata.offset())); - verify(commandProducer, future); + verify(commandProducer, recordMetadataFuture); } @Test - public void shouldReturnCompletedFutureIfOffsetReached() { + public void shouldNotWaitIfOffsetReached() throws Exception { // Given: - givenCmdConsumerAtPosition(1); - - final CommandStore commandStore = createCommandStore(); + givenCmdStoreUpThroughPosition(1); + expect(offsetFutureStore.getFutureForOffset(EasyMock.anyLong())) + .andThrow(new AssertionError()).anyTimes(); + replay(offsetFutureStore); // When: - final CompletableFuture future = commandStore.getConsumerPositionFuture(0); + commandStore.ensureConsumedUpThrough(0, TIMEOUT); // Then: - assertFutureIsCompleted(future); - verify(commandConsumer); + verify(commandConsumer, offsetFutureStore); } @Test - public void shouldReturnUncompletedFutureIfOffsetNotReached() { + public void shouldWaitIfOffsetNotReached() throws Exception { // Given: - givenCmdConsumerAtPosition(2); - - final CommandStore commandStore = createCommandStore(); + givenCmdStoreUpThroughPosition(2); + expect(offsetFutureStore.getFutureForOffset(EasyMock.anyLong())).andReturn(future); + expect(future.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))).andReturn(null); + replay(future, offsetFutureStore); // When: - final CompletableFuture future = commandStore.getConsumerPositionFuture(2); + commandStore.ensureConsumedUpThrough(2, TIMEOUT); // Then: - assertFutureIsNotCompleted(future); - verify(commandConsumer); + verify(commandConsumer, offsetFutureStore, future); } @Test - public void shouldCompleteFutureWhenOffsetIsReached() { + public void shouldThrowExceptionOnTimeout() throws Exception { // Given: - final CommandStore commandStore = createCommandStore(); - - givenCmdConsumerAtPosition(0); - final CompletableFuture future = commandStore.getConsumerPositionFuture(2); - givenCmdConsumerAtPosition(3, true); + givenCmdStoreUpThroughPosition(0); + expect(offsetFutureStore.getFutureForOffset(EasyMock.anyLong())).andReturn(future); + expect(future.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))) + .andThrow(new TimeoutException()); + replay(future, offsetFutureStore); - // When: - commandStore.getNewCommands(); + try { + // When: + commandStore.ensureConsumedUpThrough(2, TIMEOUT); - // Then: - assertFutureIsCompleted(future); - verify(commandConsumer); + // Then: + fail("TimeoutException should be propagated."); + } catch (final TimeoutException e) { + assertThat(e.getMessage(), + is(String.format( + "Timeout reached while waiting for command offset of 2. (Timeout: %d ms)", TIMEOUT))); + } + verify(commandConsumer, future, offsetFutureStore); } @Test - public void shouldNotCompleteFutureWhenOffsetIsNotReached() { + public void shouldCompleteFuturesWhenGettingNewCommands() { // Given: - final CommandStore commandStore = createCommandStore(); - - givenCmdConsumerAtPosition(0); - final CompletableFuture future = commandStore.getConsumerPositionFuture(2); - givenCmdConsumerAtPosition(2, true); + offsetFutureStore.completeFuturesUpToOffset(EasyMock.anyLong()); + expectLastCall(); + expect(commandConsumer.poll(anyObject(Duration.class))).andReturn(buildRecords()); + replay(offsetFutureStore, commandConsumer); // When: commandStore.getNewCommands(); // Then: - assertFutureIsNotCompleted(future); - verify(commandConsumer); + verify(offsetFutureStore); } private void setupConsumerToReturnCommand(final CommandId commandId, final Command command) { @@ -379,54 +396,35 @@ private void setupConsumerToReturnCommand(final CommandId commandId, final Comma expect(commandConsumer.poll(anyObject(Duration.class))).andReturn( buildRecords(commandId, command) ).times(1); - expectConsumerToReturnPartitionInfo(); replay(commandConsumer); } - private void givenCmdConsumerAtPosition(long position) { - givenCmdConsumerAtPosition(position, false); - } - - private void givenCmdConsumerAtPosition(long position, boolean poll) { + private void givenCmdStoreUpThroughPosition(long position) { reset(commandConsumer); - expectConsumerToReturnPartitionInfo(); expect(commandConsumer.position(anyObject(TopicPartition.class))).andReturn(position); - if (poll) { - expect(commandConsumer.poll(anyObject())) - .andReturn(new ConsumerRecords<>(Collections.emptyMap())); - } replay(commandConsumer); } - private void expectConsumerToReturnPartitionInfo() { - expect(commandConsumer.partitionsFor(COMMAND_TOPIC)) - .andReturn(ImmutableList.of( - new PartitionInfo(COMMAND_TOPIC, 0, node, new Node[]{node}, new Node[]{node}) - )); - } - - private CommandStore createCommandStoreThatAssignsSameId(final CommandId commandId) { - final CommandIdAssigner commandIdAssigner = mock(CommandIdAssigner.class); + private void givenCommandStoreThatAssignsSameId(final CommandId commandId) { + commandIdAssigner = mock(CommandIdAssigner.class); expect(commandIdAssigner.getCommandId(anyObject())).andStubAnswer( () -> new CommandId(commandId.getType(), commandId.getEntity(), commandId.getAction()) ); replay(commandIdAssigner); - return createCommandStore(commandIdAssigner); - } - - private CommandStore createCommandStore() { - return createCommandStore(new CommandIdAssigner(new MetaStoreImpl(new InternalFunctionRegistry()))); + setUpCommandStore(); } - private CommandStore createCommandStore(final CommandIdAssigner commandIdAssigner) { - return new CommandStore( + private void setUpCommandStore() { + commandStore = new CommandStore( COMMAND_TOPIC, commandConsumer, commandProducer, - commandIdAssigner); + commandIdAssigner, + offsetFutureStore + ); } - private List> getPriorCommands(final CommandStore commandStore) { + private static List> getPriorCommands(final CommandStore commandStore) { return commandStore.getRestoreCommands().stream() .map( queuedCommand -> new Pair<>( @@ -434,7 +432,7 @@ private List> getPriorCommands(final CommandStore comma .collect(Collectors.toList()); } - private ConsumerRecords buildRecords(final Object ...args) { + private static ConsumerRecords buildRecords(final Object ...args) { assertThat(args.length % 2, equalTo(0)); final List> records = new ArrayList<>(); for (int i = 0; i < args.length; i += 2) { @@ -450,14 +448,4 @@ private ConsumerRecords buildRecords(final Object ...args) { ) ); } - - private void assertFutureIsCompleted(CompletableFuture future) { - assertThat(future.isDone(), is(true)); - assertThat(future.isCancelled(), is(false)); - assertThat(future.isCompletedExceptionally(), is(false)); - } - - private void assertFutureIsNotCompleted(CompletableFuture future) { - assertThat(future.isDone(), is(false)); - } } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/OffsetFutureStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/OffsetFutureStoreTest.java new file mode 100644 index 000000000000..340c89e4d371 --- /dev/null +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/OffsetFutureStoreTest.java @@ -0,0 +1,101 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.computation; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; + +import java.util.concurrent.CompletableFuture; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class OffsetFutureStoreTest { + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + private OffsetFutureStore offsetFutureStore; + + @Before + public void setUp() { + offsetFutureStore = new OffsetFutureStore(); + } + + @Test + public void shouldReturnFutureForNewOffset() { + // When: + final CompletableFuture future = offsetFutureStore.getFutureForOffset(2); + + // Then: + assertFutureIsNotCompleted(future); + } + + @Test + public void shouldReturnFutureForExistingOffset() { + // Given: + final CompletableFuture existingFuture = offsetFutureStore.getFutureForOffset(2); + + // When: + final CompletableFuture newFuture = offsetFutureStore.getFutureForOffset(2); + + // Then: + assertThat(newFuture, is(sameInstance(existingFuture))); + } + + @Test + public void shouldReturnFutureForCompletedOffset() { + // Given: + final CompletableFuture firstFuture = offsetFutureStore.getFutureForOffset(2); + offsetFutureStore.completeFuturesUpToOffset(3); + assertFutureIsCompleted(firstFuture); + + // When: + final CompletableFuture secondFuture = offsetFutureStore.getFutureForOffset(2); + + // Then: + assertFutureIsNotCompleted(secondFuture); + assertThat(secondFuture, is(not(sameInstance(firstFuture)))); + } + + @Test + public void shouldCompleteFutures() { + // Given: + final CompletableFuture firstFuture = offsetFutureStore.getFutureForOffset(2); + final CompletableFuture secondFuture = offsetFutureStore.getFutureForOffset(3); + + // When: + offsetFutureStore.completeFuturesUpToOffset(3); + + // Then: + assertFutureIsCompleted(firstFuture); + assertFutureIsNotCompleted(secondFuture); + } + + private static void assertFutureIsCompleted(CompletableFuture future) { + assertThat(future.isDone(), is(true)); + assertThat(future.isCancelled(), is(false)); + assertThat(future.isCompletedExceptionally(), is(false)); + } + + private static void assertFutureIsNotCompleted(CompletableFuture future) { + assertThat(future.isDone(), is(false)); + } +} \ No newline at end of file diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index 89d9afde5e41..8a7f995a51bd 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -121,6 +121,7 @@ public QueuedCommandStatus enqueueCommand( final KsqlConfig ksqlConfig, final Map overwriteProperties) { final CommandId commandId = commandIdAssigner.getCommandId(statement); + final long commandSequenceNumber = commandLog.size(); commandLog.add( new QueuedCommand( commandId, @@ -129,7 +130,7 @@ public QueuedCommandStatus enqueueCommand( Collections.emptyMap(), ksqlConfig.getAllConfigPropsWithSecretsObfuscated()), Optional.empty())); - return new QueuedCommandStatus(commandId); + return new QueuedCommandStatus(commandSequenceNumber, new CommandStatusFuture(commandId)); } @Override @@ -147,8 +148,7 @@ public List getRestoreCommands() { } @Override - public CompletableFuture getConsumerPositionFuture(final long offset) { - return null; + public void ensureConsumedUpThrough(final long offset, final long timeout) { } @Override diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/StatementExecutorTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/StatementExecutorTest.java index d3f5824dad33..56d136984718 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/StatementExecutorTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/StatementExecutorTest.java @@ -136,7 +136,7 @@ public void tearDown() { private void handleStatement( final Command command, final CommandId commandId, - final Optional commandStatus) { + final Optional commandStatus) { handleStatement(statementExecutor, command, commandId, commandStatus); } @@ -144,7 +144,7 @@ private void handleStatement( final StatementExecutor statementExecutor, final Command command, final CommandId commandId, - final Optional commandStatus) { + final Optional commandStatus) { statementExecutor.handleStatement(new QueuedCommand(commandId, command, commandStatus)); } @@ -373,7 +373,7 @@ public void shouldCompleteFutureOnSuccess() { final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); - final QueuedCommandStatus status = mock(QueuedCommandStatus.class); + final CommandStatusFuture status = mock(CommandStatusFuture.class); status.setStatus(sameStatus(CommandStatus.Status.PARSING)); expectLastCall(); status.setStatus(sameStatus(CommandStatus.Status.EXECUTING)); @@ -402,7 +402,7 @@ public void shouldCompleteFutureOnFailure() { final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); - final QueuedCommandStatus status = mock(QueuedCommandStatus.class); + final CommandStatusFuture status = mock(CommandStatusFuture.class); status.setStatus(sameStatus(CommandStatus.Status.PARSING)); expectLastCall(); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 3cc46fceb9af..15e0289fb11d 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -68,9 +68,9 @@ import io.confluent.ksql.rest.entity.TablesList; import io.confluent.ksql.rest.server.KsqlRestConfig; import io.confluent.ksql.rest.server.computation.CommandId; +import io.confluent.ksql.rest.server.computation.CommandStatusFuture; import io.confluent.ksql.rest.server.computation.CommandStore; import io.confluent.ksql.rest.server.computation.QueuedCommandStatus; -import io.confluent.ksql.rest.server.utils.TestUtils; import io.confluent.ksql.rest.util.EntityUtil; import io.confluent.ksql.serde.DataSource; import io.confluent.ksql.serde.json.KsqlJsonTopicSerDe; @@ -89,9 +89,6 @@ import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; import java.util.function.Consumer; import java.util.stream.Collectors; import javax.ws.rs.core.Response; @@ -111,7 +108,6 @@ import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.junit.runners.model.TestTimedOutException; @SuppressWarnings("unchecked") @RunWith(EasyMockRunner.class) @@ -163,8 +159,7 @@ public void tearDown() { public void shouldInstantRegisterTopic() { // Given: final QueuedCommandStatus cmdStatus - = new QueuedCommandStatus(new CommandId("TABLE", "orders", "CREATE")); - cmdStatus.setCommandOffset(0); + = givenQueuedCommandStatus(new CommandId("TABLE", "orders", "CREATE")); givenCommandStore(mockCommandStore -> EasyMock.expect(mockCommandStore.enqueueCommand( @@ -402,9 +397,8 @@ public void shouldFailForIncorrectDropTableStatement() { public void shouldDistributeStatementWithConfigAndColumnInference() { // Given: givenCommandStore(mockCommandStore -> { - final QueuedCommandStatus queuedCommandStatus - = new QueuedCommandStatus(new CommandId("TABLE", "orders", "CREATE")); - queuedCommandStatus.setFinalStatus( + final QueuedCommandStatus queuedCommandStatus = givenQueuedCommandStatusWithStatus( + new CommandId("TABLE", "orders", "CREATE"), new CommandStatus(CommandStatus.Status.SUCCESS, "success")); final String ksqlStringWithSchema = @@ -696,13 +690,13 @@ public void shouldNotIncludeSslPropertiesInListPropertiesOutput() { } @Test - public void shouldNotWaitIfNoCommandTopicOffsetSpecified() { + public void shouldNotWaitIfNoCommandTopicOffsetSpecified() throws Exception { // Given: final String sql = "LIST REGISTERED TOPICS;"; final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), null); - EasyMock.expect(commandStore.getConsumerPositionFuture(EasyMock.anyLong())) - .andThrow(new AssertionFailedError()).anyTimes(); + commandStore.ensureConsumedUpThrough(EasyMock.anyLong(), EasyMock.anyLong()); + EasyMock.expectLastCall().andThrow(new AssertionFailedError()).anyTimes(); EasyMock.replay(commandStore); // When: @@ -713,47 +707,40 @@ public void shouldNotWaitIfNoCommandTopicOffsetSpecified() { } @Test - public void shouldNotWaitIfCommandTopicOffsetReached() - throws InterruptedException, ExecutionException, TimeoutException { + public void shouldWaitIfCommandTopicOffsetSpecified() throws Exception { // Given: final String sql = "LIST REGISTERED TOPICS;"; final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), 2L); - final CompletableFuture future = EasyMock.mock(CompletableFuture.class); - EasyMock.expect(future.get(EasyMock.anyLong(), EasyMock.anyObject())).andReturn(null); - EasyMock.expect(commandStore.getConsumerPositionFuture(EasyMock.anyLong())) - .andReturn(future); - EasyMock.replay(future, commandStore); + commandStore.ensureConsumedUpThrough(EasyMock.anyLong(), EasyMock.anyLong()); + EasyMock.expectLastCall(); + EasyMock.replay(commandStore); // When: makeSingleRequest(request, KsqlTopicsList.class); // Then: - EasyMock.verify(future, commandStore); + EasyMock.verify(commandStore); } @Test - public void shouldReturn5xxIfTimeoutWhileWaitingForCommandTopicOffset() - throws InterruptedException, ExecutionException, TimeoutException { + public void shouldReturn503IfTimeoutWhileWaitingForCommandTopicOffset() throws Exception { // Given: final String sql = "LIST REGISTERED TOPICS;"; final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), 2L); - final CompletableFuture future = EasyMock.mock(CompletableFuture.class); - EasyMock.expect(future.get(EasyMock.anyLong(), EasyMock.anyObject())) - .andThrow(new TimeoutException()); - EasyMock.expect(commandStore.getConsumerPositionFuture(EasyMock.anyLong())) - .andReturn(future); - EasyMock.replay(future, commandStore); + commandStore.ensureConsumedUpThrough(EasyMock.anyLong(), EasyMock.anyLong()); + EasyMock.expectLastCall() + .andThrow(new KsqlRestException(Errors.commandQueueCatchUpTimeout("timed out!"))); + EasyMock.replay(commandStore); // When: - final KsqlErrorMessage result = makeFailingRequest(request, Code.INTERNAL_SERVER_ERROR); + final KsqlErrorMessage result = makeFailingRequest(request, Code.SERVICE_UNAVAILABLE); // Then: - assertThat(result.getErrorCode(), is(Errors.ERROR_CODE_SERVER_ERROR)); - assertThat(result.getMessage(), - containsString("Timeout reached while waiting for command offset")); - EasyMock.verify(future, commandStore); + assertThat(result.getErrorCode(), is(Errors.ERROR_CODE_COMMAND_QUEUE_CATCHUP_TIMEOUT)); + assertThat(result.getMessage(), is("timed out!")); + EasyMock.verify(commandStore); } @SuppressWarnings("SameParameterValue") @@ -963,4 +950,15 @@ private static void registerSchema(final SchemaRegistryClient schemaRegistryClie schemaRegistryClient.register("orders-topic" + KsqlConstants.SCHEMA_REGISTRY_VALUE_SUFFIX, avroSchema); } + + private static QueuedCommandStatus givenQueuedCommandStatus(final CommandId commandId) { + return new QueuedCommandStatus(0, new CommandStatusFuture(commandId)); + } + + private static QueuedCommandStatus givenQueuedCommandStatusWithStatus( + final CommandId commandId, final CommandStatus status) { + final CommandStatusFuture commandStatusFuture = new CommandStatusFuture(commandId); + commandStatusFuture.setFinalStatus(status); + return new QueuedCommandStatus(0, commandStatusFuture); + } } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java index 6d66494f1dd4..c429a5e9dc6f 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java @@ -41,6 +41,7 @@ import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.entity.StreamedRow; import io.confluent.ksql.rest.server.StatementParser; +import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; import io.confluent.ksql.rest.server.resources.streaming.StreamedQueryResource; import io.confluent.ksql.rest.util.JsonMapper; import io.confluent.ksql.serde.DataSource; @@ -66,29 +67,37 @@ import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KafkaStreams.State; import org.apache.kafka.streams.KeyValue; +import org.junit.Before; import org.junit.Test; public class StreamedQueryResourceTest { private static final Duration DISCONNECT_CHECK_INTERVAL = Duration.ofMillis(1000); + private final KsqlConfig ksqlConfig = mock(KsqlConfig.class); + private final KsqlEngine ksqlEngine = mock(KsqlEngine.class); + private final StatementParser statementParser = mock(StatementParser.class); + private final KafkaTopicClient kafkaTopicClient = mock(KafkaTopicClientImpl.class); + private final ReplayableCommandQueue replayableCommandQueue = + niceMock(ReplayableCommandQueue.class); + private StreamedQueryResource testResource; + + @Before + public void setUp() { + testResource = new StreamedQueryResource( + ksqlConfig, ksqlEngine, statementParser, replayableCommandQueue, DISCONNECT_CHECK_INTERVAL); + } + @Test public void shouldReturn400OnBadStatement() throws Exception { final String queryString = "SELECT * FROM test_stream;"; - final KsqlConfig ksqlConfig = mock(KsqlConfig.class); - final KsqlEngine mockKsqlEngine = mock(KsqlEngine.class); - final KafkaTopicClient mockKafkaTopicClient = mock(KafkaTopicClientImpl.class); - expect(mockKsqlEngine.getTopicClient()).andReturn(mockKafkaTopicClient); + expect(ksqlEngine.getTopicClient()).andReturn(kafkaTopicClient); - final StatementParser mockStatementParser = mock(StatementParser.class); - expect(mockStatementParser.parseSingleStatement(queryString)) + expect(statementParser.parseSingleStatement(queryString)) .andThrow(new IllegalArgumentException("some msg only the parser would use")); - replay(mockKsqlEngine, mockKafkaTopicClient, mockStatementParser); - - final StreamedQueryResource testResource = new StreamedQueryResource( - ksqlConfig, mockKsqlEngine, mockStatementParser, DISCONNECT_CHECK_INTERVAL); + replay(ksqlEngine, kafkaTopicClient, statementParser); final Response response = testResource.streamQuery(new KsqlRequest(queryString, Collections.emptyMap(), null)); @@ -104,22 +113,15 @@ public void shouldReturn400OnBadStatement() throws Exception { public void shouldReturn400OnBuildMultipleQueriesError() throws Exception { final String queryString = "SELECT * FROM test_stream;"; - final KsqlConfig ksqlConfig = mock(KsqlConfig.class); - final KsqlEngine mockKsqlEngine = mock(KsqlEngine.class); - final KafkaTopicClient mockKafkaTopicClient = mock(KafkaTopicClientImpl.class); - expect(mockKsqlEngine.getTopicClient()).andReturn(mockKafkaTopicClient); + expect(ksqlEngine.getTopicClient()).andReturn(kafkaTopicClient); - final StatementParser mockStatementParser = mock(StatementParser.class); - expect(mockStatementParser.parseSingleStatement(queryString)) + expect(statementParser.parseSingleStatement(queryString)) .andReturn(mock(Query.class)); - expect(mockKsqlEngine.buildMultipleQueries(queryString, ksqlConfig, Collections.emptyMap())) + expect(ksqlEngine.buildMultipleQueries(queryString, ksqlConfig, Collections.emptyMap())) .andThrow(new KsqlException("some msg only the engine would use")); - replay(mockKsqlEngine, mockKafkaTopicClient, mockStatementParser); - - final StreamedQueryResource testResource = new StreamedQueryResource( - ksqlConfig, mockKsqlEngine, mockStatementParser, DISCONNECT_CHECK_INTERVAL); + replay(ksqlEngine, kafkaTopicClient, statementParser); final Response response = testResource.streamQuery(new KsqlRequest(queryString, Collections.emptyMap(), null)); @@ -182,32 +184,25 @@ public void run() { final Map requestStreamsProperties = Collections.emptyMap(); - final KsqlConfig mockKsqlConfig = mock(KsqlConfig.class); - final KsqlEngine mockKsqlEngine = mock(KsqlEngine.class); - final KafkaTopicClient mockKafkaTopicClient = mock(KafkaTopicClientImpl.class); - expect(mockKsqlEngine.getTopicClient()).andReturn(mockKafkaTopicClient); - expect(mockKsqlEngine.getSchemaRegistryClient()).andReturn(new MockSchemaRegistryClient()); + expect(ksqlEngine.getTopicClient()).andReturn(kafkaTopicClient); + expect(ksqlEngine.getSchemaRegistryClient()).andReturn(new MockSchemaRegistryClient()); replay(mockOutputNode, mockKafkaStreams); final QueuedQueryMetadata queuedQueryMetadata = new QueuedQueryMetadata(queryString, mockKafkaStreams, mockOutputNode, "", rowQueue, DataSource.DataSourceType.KSTREAM, "", - mockKafkaTopicClient, null, Collections.emptyMap()); + kafkaTopicClient, null, Collections.emptyMap()); reset(mockOutputNode); expect(mockOutputNode.getSchema()) .andReturn(SchemaBuilder.struct().field("f1", SchemaBuilder.OPTIONAL_INT32_SCHEMA)); - expect(mockKsqlEngine.buildMultipleQueries(queryString, mockKsqlConfig, requestStreamsProperties)) + expect(ksqlEngine.buildMultipleQueries(queryString, ksqlConfig, requestStreamsProperties)) .andReturn(Collections.singletonList(queuedQueryMetadata)); - mockKsqlEngine.removeTemporaryQuery(queuedQueryMetadata); + ksqlEngine.removeTemporaryQuery(queuedQueryMetadata); expectLastCall(); - final StatementParser mockStatementParser = mock(StatementParser.class); - expect(mockStatementParser.parseSingleStatement(queryString)).andReturn(mock(Query.class)); - - replay(mockKsqlEngine, mockStatementParser, mockOutputNode); + expect(statementParser.parseSingleStatement(queryString)).andReturn(mock(Query.class)); - final StreamedQueryResource testResource = new StreamedQueryResource( - mockKsqlConfig, mockKsqlEngine, mockStatementParser, DISCONNECT_CHECK_INTERVAL); + replay(ksqlEngine, statementParser, mockOutputNode); final Response response = testResource.streamQuery(new KsqlRequest(queryString, requestStreamsProperties, null)); 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 05923f5d3773..52a9f650a631 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 @@ -3,8 +3,10 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -26,6 +28,7 @@ import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.entity.Versions; import io.confluent.ksql.rest.server.StatementParser; +import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; import io.confluent.ksql.rest.server.resources.streaming.WSQueryEndpoint.PrintTopicPublisher; import io.confluent.ksql.rest.server.resources.streaming.WSQueryEndpoint.QueryPublisher; import io.confluent.ksql.util.KafkaTopicClient; @@ -35,6 +38,7 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import javax.websocket.CloseReason; import javax.websocket.CloseReason.CloseCodes; @@ -60,9 +64,15 @@ public class WSQueryEndpointTest { private static final KsqlRequest ANOTHER_REQUEST = new KsqlRequest("other-sql", ImmutableMap.of(), null); + private static final long OFFSET = 2L; + private static final KsqlRequest REQUEST_WITHOUT_OFFSET = VALID_REQUEST; + private static final KsqlRequest REQUEST_WITH_OFFSET = new KsqlRequest("test-sql", + ImmutableMap.of(KsqlConfig.KSQL_SERVICE_ID_CONFIG, "test-id"), OFFSET); + private static final String VALID_VERSION = Versions.KSQL_V1_WS; private static final String[] NO_VERSION_PROPERTY = null; private static final KsqlRequest[] NO_REQUEST_PROPERTY = (KsqlRequest[]) null; + private static final long COMMAND_QUEUE_CATCHUP_TIMEOUT = 5000L; @Mock private KsqlConfig ksqlConfig; @@ -81,6 +91,8 @@ public class WSQueryEndpointTest { @Mock private QueryBody queryBody; @Mock + private ReplayableCommandQueue replayableCommandQueue; + @Mock private QueryPublisher queryPublisher; @Mock private PrintTopicPublisher topicPublisher; @@ -106,8 +118,8 @@ public void setUp() { givenRequest(VALID_REQUEST); wsQueryEndpoint = new WSQueryEndpoint( - ksqlConfig, OBJECT_MAPPER, statementParser, ksqlEngine, exec, - queryPublisher, topicPublisher); + ksqlConfig, OBJECT_MAPPER, statementParser, ksqlEngine, replayableCommandQueue, exec, + queryPublisher, topicPublisher, COMMAND_QUEUE_CATCHUP_TIMEOUT); } @Test @@ -119,7 +131,7 @@ public void shouldReturnErrorOnBadVersion() throws Exception { wsQueryEndpoint.onOpen(session, null); // Then: - verifyClosedWithReason("Received invalid api version: [bad-version]"); + verifyClosedWithReason("Received invalid api version: [bad-version]", CloseCodes.CANNOT_ACCEPT); } @Test @@ -131,7 +143,7 @@ public void shouldReturnErrorOnMultipleVersions() throws Exception { wsQueryEndpoint.onOpen(session, null); // Then: - verifyClosedWithReason("Received multiple api versions: [1, 2]"); + verifyClosedWithReason("Received multiple api versions: [1, 2]", CloseCodes.CANNOT_ACCEPT); } @Test @@ -179,7 +191,8 @@ public void shouldReturnErrorOnNoRequest() throws Exception { wsQueryEndpoint.onOpen(session, null); // Then: - verifyClosedWithReason("Error parsing request: missing request parameter"); + verifyClosedWithReason( + "Error parsing request: missing request parameter", CloseCodes.CANNOT_ACCEPT); } @Test @@ -191,7 +204,8 @@ public void shouldReturnErrorOnEmptyRequests() throws Exception { wsQueryEndpoint.onOpen(session, null); // Then: - verifyClosedWithReason("Error parsing request: missing request parameter"); + verifyClosedWithReason( + "Error parsing request: missing request parameter", CloseCodes.CANNOT_ACCEPT); } @Test @@ -228,7 +242,7 @@ public void shouldReturnErrorOnBadStatement() throws Exception { wsQueryEndpoint.onOpen(session, null); // Then: - verifyClosedWithReason("Error parsing query: Boom!"); + verifyClosedWithReason("Error parsing query: Boom!", CloseCodes.CANNOT_ACCEPT); } @Test @@ -246,7 +260,9 @@ public void shouldReturnErrorOnInvalidStreamProperty() throws Exception { wsQueryEndpoint.onOpen(session, null); // Then: - verifyClosedWithReason("Error parsing request: Failed to set 'unknown-property' to 'true'"); + verifyClosedWithReason( + "Error parsing request: Failed to set 'unknown-property' to 'true'", + CloseCodes.CANNOT_ACCEPT); } @Test @@ -297,7 +313,45 @@ public void shouldReturnErrorIfTopicDoesNotExist() throws Exception { wsQueryEndpoint.onOpen(session, null); // Then: - verifyClosedWithReason("topic does not exist: bob"); + verifyClosedWithReason("topic does not exist: bob", CloseCodes.CANNOT_ACCEPT); + } + + @Test + public void shouldNotWaitIfNoOffsetSpecified() throws Exception { + // Given: + givenRequest(REQUEST_WITHOUT_OFFSET); + + // When: + wsQueryEndpoint.onOpen(session, null); + + // Then: + verify(replayableCommandQueue, never()).ensureConsumedUpThrough(anyLong(), anyLong()); + } + + @Test + public void shouldWaitIfOffsetSpecified() throws Exception { + // Given: + givenRequest(REQUEST_WITH_OFFSET); + + // When: + wsQueryEndpoint.onOpen(session, null); + + // Then: + verify(replayableCommandQueue).ensureConsumedUpThrough(eq(OFFSET), anyLong()); + } + + @Test + public void shouldReturnErrorIfCommandQueueCatchupTimeout() throws Exception { + // Given: + givenRequest(REQUEST_WITH_OFFSET); + doThrow(new TimeoutException("yikes")) + .when(replayableCommandQueue).ensureConsumedUpThrough(eq(OFFSET), anyLong()); + + // When: + wsQueryEndpoint.onOpen(session, null); + + // Then: + verifyClosedWithReason("yikes", CloseCodes.TRY_AGAIN_LATER); } private PrintTopic printTopic(final String name, final boolean fromBeginning) { @@ -365,10 +419,10 @@ private static String serialize(final KsqlRequest request) { } } - private void verifyClosedWithReason(final String reason) throws Exception { + private void verifyClosedWithReason(final String reason, final CloseCodes code) throws Exception { verify(session).close(closeReasonCaptor.capture()); final CloseReason closeReason = closeReasonCaptor.getValue(); assertThat(closeReason.getReasonPhrase(), is(reason)); - assertThat(closeReason.getCloseCode(), is(CloseCodes.CANNOT_ACCEPT)); + assertThat(closeReason.getCloseCode(), is(code)); } } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java new file mode 100644 index 000000000000..1aea4ed37838 --- /dev/null +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java @@ -0,0 +1,83 @@ +package io.confluent.ksql.rest.util; + + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import io.confluent.ksql.rest.entity.KsqlErrorMessage; +import io.confluent.ksql.rest.entity.KsqlRequest; +import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; +import io.confluent.ksql.rest.server.resources.KsqlRestException; +import java.util.Optional; +import java.util.concurrent.TimeoutException; +import javax.ws.rs.core.Response; +import org.eclipse.jetty.http.HttpStatus.Code; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class CommandStoreUtilTest { + + private static final long TIMEOUT = 5000L; + private static final long OFFSET = 2; + + @Mock + private ReplayableCommandQueue replayableCommandQueue; + @Mock + private KsqlRequest request; + + @Test + public void shouldNotWaitIfNoOffsetSpecified() throws Exception { + // Given: + when(request.getCommandOffset()).thenReturn(Optional.empty()); + + // When: + CommandStoreUtil.waitForCommandOffset(replayableCommandQueue, request, TIMEOUT); + + // Then: + verify(replayableCommandQueue, never()).ensureConsumedUpThrough(anyLong(), anyLong()); + } + + @Test + public void shouldWaitIfOffsetSpecified() throws Exception { + // Given: + when(request.getCommandOffset()).thenReturn(Optional.of(OFFSET)); + + // When: + CommandStoreUtil.waitForCommandOffset(replayableCommandQueue, request, TIMEOUT); + + // Then: + verify(replayableCommandQueue).ensureConsumedUpThrough(OFFSET, TIMEOUT); + } + + @Test + public void shouldThrowKsqlRestExceptionOnTimeout() throws Exception { + // Given: + when(request.getCommandOffset()).thenReturn(Optional.of(OFFSET)); + doThrow(new TimeoutException("uh oh")) + .when(replayableCommandQueue).ensureConsumedUpThrough(OFFSET, TIMEOUT); + + try { + // When: + CommandStoreUtil.httpWaitForCommandOffset(replayableCommandQueue, request, TIMEOUT); + + // Then: + fail("Should propagate error."); + } catch (final KsqlRestException e) { + final Response response = e.getResponse(); + assertThat(response.getStatus(), is(Code.SERVICE_UNAVAILABLE.getCode())); + assertThat(response.getEntity(), is(instanceOf(KsqlErrorMessage.class))); + final KsqlErrorMessage message = (KsqlErrorMessage) (response.getEntity()); + assertThat(message.getMessage(), is("uh oh")); + } + } +} \ No newline at end of file From 8890ad95fa761c274fe5e4c728b14c21b2b7fb8f Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Tue, 27 Nov 2018 23:43:55 -0800 Subject: [PATCH 04/13] rename 'offset' to 'sequence number'. --- .../ksql/rest/entity/CommandStatusEntity.java | 20 ++++---- .../ksql/rest/entity/KsqlRequest.java | 14 +++--- .../rest/server/computation/CommandStore.java | 39 ++++++++------- .../computation/QueuedCommandStatus.java | 10 ++-- .../computation/ReplayableCommandQueue.java | 2 +- ...re.java => SequenceNumberFutureStore.java} | 22 ++++----- .../rest/server/resources/KsqlResource.java | 4 +- .../streaming/StreamedQueryResource.java | 2 +- .../resources/streaming/WSQueryEndpoint.java | 2 +- .../ksql/rest/util/CommandStoreUtil.java | 14 +++--- .../rest/entity/CommandStatusEntityTest.java | 2 +- .../ksql/rest/entity/KsqlRequestTest.java | 48 +++++++++---------- .../server/computation/CommandStoreTest.java | 39 +++++++-------- .../rest/server/computation/RecoveryTest.java | 2 +- ...ava => SequenceNumberFutureStoreTest.java} | 30 ++++++------ .../server/resources/KsqlResourceTest.java | 6 +-- .../streaming/WSQueryEndpointTest.java | 22 ++++----- .../ksql/rest/util/CommandStoreUtilTest.java | 22 ++++----- 18 files changed, 153 insertions(+), 147 deletions(-) rename ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/{OffsetFutureStore.java => SequenceNumberFutureStore.java} (59%) rename ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/{OffsetFutureStoreTest.java => SequenceNumberFutureStoreTest.java} (63%) diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java index 0de83ff8d1bd..9d65117ea6f2 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java @@ -28,18 +28,18 @@ public class CommandStatusEntity extends KsqlEntity { private final CommandId commandId; private final CommandStatus commandStatus; - private final long commandOffset; + private final long commandSequenceNumber; public CommandStatusEntity( final String statementText, final CommandId commandId, final CommandStatus commandStatus, - final long commandOffset + final long commandSequenceNumber ) { super(statementText); this.commandId = commandId; this.commandStatus = commandStatus; - this.commandOffset = commandOffset; + this.commandSequenceNumber = commandSequenceNumber; } @JsonCreator @@ -47,7 +47,7 @@ public CommandStatusEntity( @JsonProperty("statementText") final String statementText, @JsonProperty("commandId") final String commandId, @JsonProperty("commandStatus") final Map commandStatus, - @JsonProperty("commandOffset") final Long commandOffset + @JsonProperty("commandSequenceNumber") final Long commandSequenceNumber ) { this( statementText, @@ -55,7 +55,7 @@ public CommandStatusEntity( new CommandStatus( CommandStatus.Status.valueOf((String) commandStatus.get("status")), (String) commandStatus.get("message")), - commandOffset == null ? -1 : commandOffset + commandSequenceNumber == null ? -1 : commandSequenceNumber ); } @@ -68,8 +68,8 @@ public CommandStatus getCommandStatus() { return commandStatus; } - public long getCommandOffset() { - return commandOffset; + public long getCommandSequenceNumber() { + return commandSequenceNumber; } @Override @@ -83,12 +83,12 @@ public boolean equals(final Object o) { final CommandStatusEntity that = (CommandStatusEntity) o; return Objects.equals(commandId, that.commandId) && Objects.equals(commandStatus, that.commandStatus) - && (commandOffset == that.commandOffset); + && (commandSequenceNumber == that.commandSequenceNumber); } @Override public int hashCode() { - return Objects.hash(getCommandId(), getCommandStatus(), getCommandOffset()); + return Objects.hash(getCommandId(), getCommandStatus(), getCommandSequenceNumber()); } @Override @@ -96,7 +96,7 @@ public String toString() { return "CommandStatusEntity{" + "commandId=" + commandId + ", commandStatus=" + commandStatus - + ", commandOffset=" + commandOffset + + ", commandSequenceNumber=" + commandSequenceNumber + '}'; } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java index 78acdba0fcb5..e9f5624d93bd 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java @@ -36,19 +36,19 @@ public class KsqlRequest { private final String ksql; private final Map streamsProperties; - private final Optional commandOffset; + private final Optional commandSequenceNumber; @JsonCreator public KsqlRequest( @JsonProperty("ksql") final String ksql, @JsonProperty("streamsProperties") final Map streamsProperties, - @JsonProperty("commandOffset") final Long commandOffset + @JsonProperty("commandSequenceNumber") final Long commandSequenceNumber ) { this.ksql = ksql == null ? "" : ksql; this.streamsProperties = streamsProperties == null ? Collections.emptyMap() : Collections.unmodifiableMap(new HashMap<>(streamsProperties)); - this.commandOffset = Optional.ofNullable(commandOffset); + this.commandSequenceNumber = Optional.ofNullable(commandSequenceNumber); } public String getKsql() { @@ -59,8 +59,8 @@ public Map getStreamsProperties() { return coerceTypes(streamsProperties); } - public Optional getCommandOffset() { - return commandOffset; + public Optional getCommandSequenceNumber() { + return commandSequenceNumber; } @Override @@ -76,12 +76,12 @@ public boolean equals(final Object o) { final KsqlRequest that = (KsqlRequest) o; return Objects.equals(getKsql(), that.getKsql()) && Objects.equals(getStreamsProperties(), that.getStreamsProperties()) - && Objects.equals(getCommandOffset(), that.getCommandOffset()); + && Objects.equals(getCommandSequenceNumber(), that.getCommandSequenceNumber()); } @Override public int hashCode() { - return Objects.hash(getKsql(), getStreamsProperties(), getCommandOffset()); + return Objects.hash(getKsql(), getStreamsProperties(), getCommandSequenceNumber()); } private static Map coerceTypes(final Map streamsProperties) { diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 24c3d5057b90..60e958c12214 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -63,7 +63,7 @@ public class CommandStore implements ReplayableCommandQueue, Closeable { private final Producer commandProducer; private final CommandIdAssigner commandIdAssigner; private final Map commandStatusMap; - private final OffsetFutureStore offsetFutureStore; + private final SequenceNumberFutureStore sequenceNumberFutureStore; public CommandStore( final String commandTopic, @@ -72,7 +72,11 @@ public CommandStore( final CommandIdAssigner commandIdAssigner ) { this( - commandTopic,commandConsumer, commandProducer, commandIdAssigner, new OffsetFutureStore()); + commandTopic, + commandConsumer, + commandProducer, + commandIdAssigner, + new SequenceNumberFutureStore()); } CommandStore( @@ -80,7 +84,7 @@ public CommandStore( final Consumer commandConsumer, final Producer commandProducer, final CommandIdAssigner commandIdAssigner, - final OffsetFutureStore offsetFutureStore + final SequenceNumberFutureStore sequenceNumberFutureStore ) { this.commandTopic = commandTopic; this.topicPartition = new TopicPartition(commandTopic, 0); @@ -88,7 +92,7 @@ public CommandStore( this.commandProducer = commandProducer; this.commandIdAssigner = commandIdAssigner; this.commandStatusMap = Maps.newConcurrentMap(); - this.offsetFutureStore = offsetFutureStore; + this.sequenceNumberFutureStore = sequenceNumberFutureStore; commandConsumer.assign(Collections.singleton(topicPartition)); } @@ -161,7 +165,7 @@ public QueuedCommandStatus enqueueCommand( * @return The commands that have been polled from the command topic */ public List getNewCommands() { - completeSatisfiedOffsetFutures(); + completeSatisfiedSequenceNumberFutures(); final List queuedCommands = Lists.newArrayList(); commandConsumer.poll(Duration.ofMillis(Long.MAX_VALUE)).forEach( @@ -211,32 +215,33 @@ public List getRestoreCommands() { } @Override - public void ensureConsumedUpThrough(final long offset, final long timeout) + public void ensureConsumedUpThrough(final long seqNum, final long timeout) throws TimeoutException { - final long consumerPosition = getNextConsumerOffset(); - if (consumerPosition > offset) { + final long consumerPosition = getNextConsumerSequenceNumber(); + if (consumerPosition > seqNum) { return; } - final CompletableFuture future = offsetFutureStore.getFutureForOffset(offset); + final CompletableFuture future = + sequenceNumberFutureStore.getFutureForSequenceNumber(seqNum); try { future.get(timeout, TimeUnit.MILLISECONDS); } catch (final ExecutionException e) { throw new RuntimeException( - "Error waiting for command offset of " + offset, e.getCause()); + "Error waiting for command sequence number of " + seqNum, e.getCause()); } catch (final InterruptedException e) { throw new RuntimeException( - "Interrupted while waiting for command offset of " + offset, e); + "Interrupted while waiting for command sequence number of " + seqNum, e); } catch (final TimeoutException e) { throw new TimeoutException( String.format( - "Timeout reached while waiting for command offset of %d. (Timeout: %d ms)", - offset, + "Timeout reached while waiting for command sequence number of %d. (Timeout: %d ms)", + seqNum, timeout)); } } - private long getNextConsumerOffset() { + private long getNextConsumerSequenceNumber() { return commandConsumer.position(topicPartition); } @@ -250,8 +255,8 @@ private Collection getTopicPartitionsForTopic(final String topic return result; } - private void completeSatisfiedOffsetFutures() { - final long consumerPosition = getNextConsumerOffset(); - offsetFutureStore.completeFuturesUpToOffset(consumerPosition); + private void completeSatisfiedSequenceNumberFutures() { + final long consumerPosition = getNextConsumerSequenceNumber(); + sequenceNumberFutureStore.completeFuturesUpToSequenceNumber(consumerPosition); } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java index 88cd8b86e72b..799a04ce06e8 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java @@ -22,11 +22,11 @@ public class QueuedCommandStatus { private final CommandStatusFuture commandStatusFuture; - private final long commandOffset; + private final long commandSequenceNumber; public QueuedCommandStatus( - final long commandOffset, final CommandStatusFuture commandStatusFuture) { - this.commandOffset = commandOffset; + final long commandSequenceNumber, final CommandStatusFuture commandStatusFuture) { + this.commandSequenceNumber = commandSequenceNumber; this.commandStatusFuture = commandStatusFuture; } @@ -38,8 +38,8 @@ public CommandId getCommandId() { return commandStatusFuture.getCommandId(); } - public long getCommandOffset() { - return commandOffset; + public long getCommandSequenceNumber() { + return commandSequenceNumber; } public CommandStatus tryWaitForFinalStatus(final Duration timeout) throws InterruptedException { diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java index 2f213afa5e2f..647d4c78da61 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java @@ -36,5 +36,5 @@ QueuedCommandStatus enqueueCommand( List getRestoreCommands(); - void ensureConsumedUpThrough(long offset, long timeout) throws TimeoutException; + void ensureConsumedUpThrough(long seqNum, long timeout) throws TimeoutException; } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/OffsetFutureStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java similarity index 59% rename from ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/OffsetFutureStore.java rename to ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java index 8bc6f79d109b..4c59f8e4d850 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/OffsetFutureStore.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java @@ -19,23 +19,23 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -public class OffsetFutureStore { - private final ConcurrentHashMap> offsetFutures; +public class SequenceNumberFutureStore { + private final ConcurrentHashMap> sequenceNumberFutures; - public OffsetFutureStore() { - offsetFutures = new ConcurrentHashMap<>(8, 0.9f, 1); + public SequenceNumberFutureStore() { + sequenceNumberFutures = new ConcurrentHashMap<>(8, 0.9f, 1); } - public CompletableFuture getFutureForOffset(final long offset) { - return offsetFutures.computeIfAbsent(offset, k -> new CompletableFuture<>()); + public CompletableFuture getFutureForSequenceNumber(final long seqNum) { + return sequenceNumberFutures.computeIfAbsent(seqNum, k -> new CompletableFuture<>()); } - public void completeFuturesUpToOffset(final long offset) { - offsetFutures.keySet().stream() - .filter(k -> k < offset) + public void completeFuturesUpToSequenceNumber(final long seqNum) { + sequenceNumberFutures.keySet().stream() + .filter(k -> k < seqNum) .forEach(k -> { - offsetFutures.get(k).complete(null); - offsetFutures.remove(k); + sequenceNumberFutures.get(k).complete(null); + sequenceNumberFutures.remove(k); }); } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index ba1b653fe626..16186c60fd76 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -163,7 +163,7 @@ public Response handleKsqlStatements(final KsqlRequest request) { final KsqlEntityList result = new KsqlEntityList(); try { - CommandStoreUtil.httpWaitForCommandOffset( + CommandStoreUtil.httpWaitForCommandSequenceNumber( replayableCommandQueue, request, distributedCommandResponseTimeout); } catch (final KsqlRestException e) { throw e; @@ -352,7 +352,7 @@ private CommandStatusEntity distributeStatement( queuedCommandStatus.getCommandId(), queuedCommandStatus.tryWaitForFinalStatus( Duration.ofMillis(distributedCommandResponseTimeout)), - queuedCommandStatus.getCommandOffset() + queuedCommandStatus.getCommandSequenceNumber() ); } catch (final Exception e) { throw new RuntimeException(e); 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 ed76d6e6436b..4b7a23a9a3ad 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 @@ -82,7 +82,7 @@ public Response streamQuery(final KsqlRequest request) throws Exception { } try { - CommandStoreUtil.httpWaitForCommandOffset( + CommandStoreUtil.httpWaitForCommandSequenceNumber( replayableCommandQueue, request, disconnectCheckInterval.toMillis()); statement = statementParser.parseSingleStatement(ksql); } catch (final KsqlRestException e) { 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 1a6c540d72dc..1d67a3f5e2d1 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 @@ -120,7 +120,7 @@ public void onOpen(final Session session, final EndpointConfig unused) { validateVersion(session); final KsqlRequest request = parseRequest(session); - CommandStoreUtil.waitForCommandOffset(replayableCommandQueue, request, timeout); + CommandStoreUtil.waitForCommandSequenceNumber(replayableCommandQueue, request, timeout); final Statement statement = parseStatement(request); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java index 3bd89646d6e3..2925c0e9f501 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java @@ -27,25 +27,25 @@ public final class CommandStoreUtil { private CommandStoreUtil() { } - public static void httpWaitForCommandOffset( + public static void httpWaitForCommandSequenceNumber( final ReplayableCommandQueue replayableCommandQueue, final KsqlRequest request, final long timeout) { try { - waitForCommandOffset(replayableCommandQueue, request, timeout); + waitForCommandSequenceNumber(replayableCommandQueue, request, timeout); } catch (final TimeoutException e) { throw new KsqlRestException(Errors.commandQueueCatchUpTimeout(e.getMessage())); } } - public static void waitForCommandOffset( + public static void waitForCommandSequenceNumber( final ReplayableCommandQueue replayableCommandQueue, final KsqlRequest request, final long timeout) throws TimeoutException { - final Optional commandOffset = request.getCommandOffset(); - if (commandOffset.isPresent()) { - final long offset = commandOffset.get(); - replayableCommandQueue.ensureConsumedUpThrough(offset, timeout); + final Optional commandSequenceNumber = request.getCommandSequenceNumber(); + if (commandSequenceNumber.isPresent()) { + final long seqNum = commandSequenceNumber.get(); + replayableCommandQueue.ensureConsumedUpThrough(seqNum, timeout); } } } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java index 9368f64fbab9..69913539f5af 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java @@ -34,7 +34,7 @@ public class CommandStatusEntityTest { + "\"status\":\"SUCCESS\"," + "\"message\":\"some success message\"" + "}," - + "\"commandOffset\":0" + + "\"commandSequenceNumber\":0" + "}"; private static final CommandStatusEntity ENTITY = new CommandStatusEntity( diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/KsqlRequestTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/KsqlRequestTest.java index 7ccd06a9515c..9bf72a42c7cc 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/KsqlRequestTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/KsqlRequestTest.java @@ -48,27 +48,27 @@ public class KsqlRequestTest { + "\"streamsProperties\":{" + "\"" + KsqlConfig.KSQL_SERVICE_ID_CONFIG + "\":\"some-service-id\"" + "}}"; - private static final String A_JSON_REQUEST_WITH_OFFSET = "{" + private static final String A_JSON_REQUEST_WITH_COMMAND_NUMBER = "{" + "\"ksql\":\"sql\"," + "\"streamsProperties\":{" + "\"" + KsqlConfig.KSQL_SERVICE_ID_CONFIG + "\":\"some-service-id\"" + "}," - + "\"commandOffset\":2}"; - private static final String A_JSON_REQUEST_WITH_NULL_OFFSET = "{" + + "\"commandSequenceNumber\":2}"; + private static final String A_JSON_REQUEST_WITH_NULL_COMMAND_NUMBER = "{" + "\"ksql\":\"sql\"," + "\"streamsProperties\":{" + "\"" + KsqlConfig.KSQL_SERVICE_ID_CONFIG + "\":\"some-service-id\"" + "}," - + "\"commandOffset\":null}"; + + "\"commandSequenceNumber\":null}"; private static final ImmutableMap SOME_PROPS = ImmutableMap.of( KsqlConfig.KSQL_SERVICE_ID_CONFIG, "some-service-id" ); - private static final long SOME_OFFSET = 2L; + private static final long SOME_COMMAND_NUMBER = 2L; private static final KsqlRequest A_REQUEST = new KsqlRequest("sql", SOME_PROPS, null); - private static final KsqlRequest A_REQUEST_WITH_OFFSET = - new KsqlRequest("sql", SOME_PROPS, SOME_OFFSET); + private static final KsqlRequest A_REQUEST_WITH_COMMAND_NUMBER = + new KsqlRequest("sql", SOME_PROPS, SOME_COMMAND_NUMBER); @BeforeClass public static void setUpClass() { @@ -80,18 +80,18 @@ public static void setUpClass() { @Test public void shouldHandleNullStatement() { - assertThat(new KsqlRequest(null, SOME_PROPS, SOME_OFFSET).getKsql(), is("")); + assertThat(new KsqlRequest(null, SOME_PROPS, SOME_COMMAND_NUMBER).getKsql(), is("")); } @Test public void shouldHandleNullProps() { - assertThat(new KsqlRequest("sql", null, SOME_OFFSET).getStreamsProperties(), + assertThat(new KsqlRequest("sql", null, SOME_COMMAND_NUMBER).getStreamsProperties(), is(Collections.emptyMap())); } @Test - public void shouldHandleNullOffset() { - assertThat(new KsqlRequest("sql", SOME_PROPS, null).getCommandOffset(), is(Optional.empty())); + public void shouldHandleNullCommandNumber() { + assertThat(new KsqlRequest("sql", SOME_PROPS, null).getCommandSequenceNumber(), is(Optional.empty())); } @Test @@ -104,18 +104,18 @@ public void shouldDeserializeFromJson() { } @Test - public void shouldDeserializeFromJsonWithOffset() { + public void shouldDeserializeFromJsonWithCommandNumber() { // When: - final KsqlRequest request = deserialize(A_JSON_REQUEST_WITH_OFFSET); + final KsqlRequest request = deserialize(A_JSON_REQUEST_WITH_COMMAND_NUMBER); // Then: - assertThat(request, is(A_REQUEST_WITH_OFFSET)); + assertThat(request, is(A_REQUEST_WITH_COMMAND_NUMBER)); } @Test - public void shouldDeserializeFromJsonWithNullOffset() { + public void shouldDeserializeFromJsonWithNullCommandNumber() { // When: - final KsqlRequest request = deserialize(A_JSON_REQUEST_WITH_NULL_OFFSET); + final KsqlRequest request = deserialize(A_JSON_REQUEST_WITH_NULL_COMMAND_NUMBER); // Then: assertThat(request, is(A_REQUEST)); @@ -127,25 +127,25 @@ public void shouldSerializeToJson() { final String jsonRequest = serialize(A_REQUEST); // Then: - assertThat(jsonRequest, is(A_JSON_REQUEST_WITH_NULL_OFFSET)); + assertThat(jsonRequest, is(A_JSON_REQUEST_WITH_NULL_COMMAND_NUMBER)); } @Test - public void shouldSerializeToJsonWithOffset() { + public void shouldSerializeToJsonWithCommandNumber() { // When: - final String jsonRequest = serialize(A_REQUEST_WITH_OFFSET); + final String jsonRequest = serialize(A_REQUEST_WITH_COMMAND_NUMBER); // Then: - assertThat(jsonRequest, is(A_JSON_REQUEST_WITH_OFFSET)); + assertThat(jsonRequest, is(A_JSON_REQUEST_WITH_COMMAND_NUMBER)); } @Test public void shouldImplementHashCodeAndEqualsCorrectly() { new EqualsTester() - .addEqualityGroup(new KsqlRequest("sql", SOME_PROPS, SOME_OFFSET), - new KsqlRequest("sql", SOME_PROPS, SOME_OFFSET)) - .addEqualityGroup(new KsqlRequest("different-sql", SOME_PROPS, SOME_OFFSET)) - .addEqualityGroup(new KsqlRequest("sql", ImmutableMap.of(), SOME_OFFSET)) + .addEqualityGroup(new KsqlRequest("sql", SOME_PROPS, SOME_COMMAND_NUMBER), + new KsqlRequest("sql", SOME_PROPS, SOME_COMMAND_NUMBER)) + .addEqualityGroup(new KsqlRequest("different-sql", SOME_PROPS, SOME_COMMAND_NUMBER)) + .addEqualityGroup(new KsqlRequest("sql", ImmutableMap.of(), SOME_COMMAND_NUMBER)) .addEqualityGroup(new KsqlRequest("sql", SOME_PROPS, null)) .testEquals(); } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index 6a4989433876..cd3deffa00fa 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -86,7 +86,8 @@ public class CommandStoreTest { private final Producer commandProducer = mock(Producer.class); private CommandIdAssigner commandIdAssigner = new CommandIdAssigner(new MetaStoreImpl(new InternalFunctionRegistry())); - private final OffsetFutureStore offsetFutureStore = mock(OffsetFutureStore.class); + private final SequenceNumberFutureStore sequenceNumberFutureStore = mock( + SequenceNumberFutureStore.class); private final CompletableFuture future = niceMock(CompletableFuture.class); private final String statementText = "test-statement"; private final CommandId commandId = @@ -306,7 +307,7 @@ public void shouldDistributeCommand() { } @Test - public void shouldIncludeTopicOffsetInSuccessfulQueuedCommandStatus() { + public void shouldIncludeCommandSequenceNumberInSuccessfulQueuedCommandStatus() { // Given: givenCommandStoreThatAssignsSameId(commandId); @@ -318,49 +319,49 @@ public void shouldIncludeTopicOffsetInSuccessfulQueuedCommandStatus() { commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); // Then: - assertThat(commandStatus.getCommandOffset(), equalTo(recordMetadata.offset())); + assertThat(commandStatus.getCommandSequenceNumber(), equalTo(recordMetadata.offset())); verify(commandProducer, recordMetadataFuture); } @Test - public void shouldNotWaitIfOffsetReached() throws Exception { + public void shouldNotWaitIfSequenceNumberReached() throws Exception { // Given: givenCmdStoreUpThroughPosition(1); - expect(offsetFutureStore.getFutureForOffset(EasyMock.anyLong())) + expect(sequenceNumberFutureStore.getFutureForSequenceNumber(EasyMock.anyLong())) .andThrow(new AssertionError()).anyTimes(); - replay(offsetFutureStore); + replay(sequenceNumberFutureStore); // When: commandStore.ensureConsumedUpThrough(0, TIMEOUT); // Then: - verify(commandConsumer, offsetFutureStore); + verify(commandConsumer, sequenceNumberFutureStore); } @Test - public void shouldWaitIfOffsetNotReached() throws Exception { + public void shouldWaitIfSequenceNumberNotReached() throws Exception { // Given: givenCmdStoreUpThroughPosition(2); - expect(offsetFutureStore.getFutureForOffset(EasyMock.anyLong())).andReturn(future); + expect(sequenceNumberFutureStore.getFutureForSequenceNumber(EasyMock.anyLong())).andReturn(future); expect(future.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))).andReturn(null); - replay(future, offsetFutureStore); + replay(future, sequenceNumberFutureStore); // When: commandStore.ensureConsumedUpThrough(2, TIMEOUT); // Then: - verify(commandConsumer, offsetFutureStore, future); + verify(commandConsumer, sequenceNumberFutureStore, future); } @Test public void shouldThrowExceptionOnTimeout() throws Exception { // Given: givenCmdStoreUpThroughPosition(0); - expect(offsetFutureStore.getFutureForOffset(EasyMock.anyLong())).andReturn(future); + expect(sequenceNumberFutureStore.getFutureForSequenceNumber(EasyMock.anyLong())).andReturn(future); expect(future.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))) .andThrow(new TimeoutException()); - replay(future, offsetFutureStore); + replay(future, sequenceNumberFutureStore); try { // When: @@ -371,24 +372,24 @@ public void shouldThrowExceptionOnTimeout() throws Exception { } catch (final TimeoutException e) { assertThat(e.getMessage(), is(String.format( - "Timeout reached while waiting for command offset of 2. (Timeout: %d ms)", TIMEOUT))); + "Timeout reached while waiting for command sequence number of 2. (Timeout: %d ms)", TIMEOUT))); } - verify(commandConsumer, future, offsetFutureStore); + verify(commandConsumer, future, sequenceNumberFutureStore); } @Test public void shouldCompleteFuturesWhenGettingNewCommands() { // Given: - offsetFutureStore.completeFuturesUpToOffset(EasyMock.anyLong()); + sequenceNumberFutureStore.completeFuturesUpToSequenceNumber(EasyMock.anyLong()); expectLastCall(); expect(commandConsumer.poll(anyObject(Duration.class))).andReturn(buildRecords()); - replay(offsetFutureStore, commandConsumer); + replay(sequenceNumberFutureStore, commandConsumer); // When: commandStore.getNewCommands(); // Then: - verify(offsetFutureStore); + verify(sequenceNumberFutureStore); } private void setupConsumerToReturnCommand(final CommandId commandId, final Command command) { @@ -420,7 +421,7 @@ private void setUpCommandStore() { commandConsumer, commandProducer, commandIdAssigner, - offsetFutureStore + sequenceNumberFutureStore ); } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index 8a7f995a51bd..5b9ee79e1308 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -148,7 +148,7 @@ public List getRestoreCommands() { } @Override - public void ensureConsumedUpThrough(final long offset, final long timeout) { + public void ensureConsumedUpThrough(final long seqNum, final long timeout) { } @Override diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/OffsetFutureStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java similarity index 63% rename from ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/OffsetFutureStoreTest.java rename to ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java index 340c89e4d371..fbff1755e3d0 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/OffsetFutureStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java @@ -27,48 +27,48 @@ import org.junit.Test; import org.junit.rules.ExpectedException; -public class OffsetFutureStoreTest { +public class SequenceNumberFutureStoreTest { @Rule public final ExpectedException expectedException = ExpectedException.none(); - private OffsetFutureStore offsetFutureStore; + private SequenceNumberFutureStore sequenceNumberFutureStore; @Before public void setUp() { - offsetFutureStore = new OffsetFutureStore(); + sequenceNumberFutureStore = new SequenceNumberFutureStore(); } @Test - public void shouldReturnFutureForNewOffset() { + public void shouldReturnFutureForNewSequenceNumber() { // When: - final CompletableFuture future = offsetFutureStore.getFutureForOffset(2); + final CompletableFuture future = sequenceNumberFutureStore.getFutureForSequenceNumber(2); // Then: assertFutureIsNotCompleted(future); } @Test - public void shouldReturnFutureForExistingOffset() { + public void shouldReturnFutureForExistingSequenceNumber() { // Given: - final CompletableFuture existingFuture = offsetFutureStore.getFutureForOffset(2); + final CompletableFuture existingFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); // When: - final CompletableFuture newFuture = offsetFutureStore.getFutureForOffset(2); + final CompletableFuture newFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); // Then: assertThat(newFuture, is(sameInstance(existingFuture))); } @Test - public void shouldReturnFutureForCompletedOffset() { + public void shouldReturnFutureForCompletedSequenceNumber() { // Given: - final CompletableFuture firstFuture = offsetFutureStore.getFutureForOffset(2); - offsetFutureStore.completeFuturesUpToOffset(3); + final CompletableFuture firstFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); + sequenceNumberFutureStore.completeFuturesUpToSequenceNumber(3); assertFutureIsCompleted(firstFuture); // When: - final CompletableFuture secondFuture = offsetFutureStore.getFutureForOffset(2); + final CompletableFuture secondFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); // Then: assertFutureIsNotCompleted(secondFuture); @@ -78,11 +78,11 @@ public void shouldReturnFutureForCompletedOffset() { @Test public void shouldCompleteFutures() { // Given: - final CompletableFuture firstFuture = offsetFutureStore.getFutureForOffset(2); - final CompletableFuture secondFuture = offsetFutureStore.getFutureForOffset(3); + final CompletableFuture firstFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); + final CompletableFuture secondFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(3); // When: - offsetFutureStore.completeFuturesUpToOffset(3); + sequenceNumberFutureStore.completeFuturesUpToSequenceNumber(3); // Then: assertFutureIsCompleted(firstFuture); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 15e0289fb11d..7d8358f7f90b 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -690,7 +690,7 @@ public void shouldNotIncludeSslPropertiesInListPropertiesOutput() { } @Test - public void shouldNotWaitIfNoCommandTopicOffsetSpecified() throws Exception { + public void shouldNotWaitIfNoCommandSequenceNumberSpecified() throws Exception { // Given: final String sql = "LIST REGISTERED TOPICS;"; final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), null); @@ -707,7 +707,7 @@ public void shouldNotWaitIfNoCommandTopicOffsetSpecified() throws Exception { } @Test - public void shouldWaitIfCommandTopicOffsetSpecified() throws Exception { + public void shouldWaitIfCommandSequenceNumberSpecified() throws Exception { // Given: final String sql = "LIST REGISTERED TOPICS;"; final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), 2L); @@ -724,7 +724,7 @@ public void shouldWaitIfCommandTopicOffsetSpecified() throws Exception { } @Test - public void shouldReturn503IfTimeoutWhileWaitingForCommandTopicOffset() throws Exception { + public void shouldReturn503IfTimeoutWhileWaitingForCommandSequenceNumber() throws Exception { // Given: final String sql = "LIST REGISTERED TOPICS;"; final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), 2L); 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 52a9f650a631..badea4d37030 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 @@ -64,10 +64,10 @@ public class WSQueryEndpointTest { private static final KsqlRequest ANOTHER_REQUEST = new KsqlRequest("other-sql", ImmutableMap.of(), null); - private static final long OFFSET = 2L; - private static final KsqlRequest REQUEST_WITHOUT_OFFSET = VALID_REQUEST; - private static final KsqlRequest REQUEST_WITH_OFFSET = new KsqlRequest("test-sql", - ImmutableMap.of(KsqlConfig.KSQL_SERVICE_ID_CONFIG, "test-id"), OFFSET); + private static final long SEQUENCE_NUMBER = 2L; + private static final KsqlRequest REQUEST_WITHOUT_SEQUENCE_NUMBER = VALID_REQUEST; + private static final KsqlRequest REQUEST_WITH_SEQUENCE_NUMBER = new KsqlRequest("test-sql", + ImmutableMap.of(KsqlConfig.KSQL_SERVICE_ID_CONFIG, "test-id"), SEQUENCE_NUMBER); private static final String VALID_VERSION = Versions.KSQL_V1_WS; private static final String[] NO_VERSION_PROPERTY = null; @@ -317,9 +317,9 @@ public void shouldReturnErrorIfTopicDoesNotExist() throws Exception { } @Test - public void shouldNotWaitIfNoOffsetSpecified() throws Exception { + public void shouldNotWaitIfNoSequenceNumberSpecified() throws Exception { // Given: - givenRequest(REQUEST_WITHOUT_OFFSET); + givenRequest(REQUEST_WITHOUT_SEQUENCE_NUMBER); // When: wsQueryEndpoint.onOpen(session, null); @@ -329,23 +329,23 @@ public void shouldNotWaitIfNoOffsetSpecified() throws Exception { } @Test - public void shouldWaitIfOffsetSpecified() throws Exception { + public void shouldWaitIfSequenceNumberSpecified() throws Exception { // Given: - givenRequest(REQUEST_WITH_OFFSET); + givenRequest(REQUEST_WITH_SEQUENCE_NUMBER); // When: wsQueryEndpoint.onOpen(session, null); // Then: - verify(replayableCommandQueue).ensureConsumedUpThrough(eq(OFFSET), anyLong()); + verify(replayableCommandQueue).ensureConsumedUpThrough(eq(SEQUENCE_NUMBER), anyLong()); } @Test public void shouldReturnErrorIfCommandQueueCatchupTimeout() throws Exception { // Given: - givenRequest(REQUEST_WITH_OFFSET); + givenRequest(REQUEST_WITH_SEQUENCE_NUMBER); doThrow(new TimeoutException("yikes")) - .when(replayableCommandQueue).ensureConsumedUpThrough(eq(OFFSET), anyLong()); + .when(replayableCommandQueue).ensureConsumedUpThrough(eq(SEQUENCE_NUMBER), anyLong()); // When: wsQueryEndpoint.onOpen(session, null); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java index 1aea4ed37838..aa821a83d8fc 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java @@ -28,7 +28,7 @@ public class CommandStoreUtilTest { private static final long TIMEOUT = 5000L; - private static final long OFFSET = 2; + private static final long SEQUENCE_NUMBER = 2; @Mock private ReplayableCommandQueue replayableCommandQueue; @@ -36,39 +36,39 @@ public class CommandStoreUtilTest { private KsqlRequest request; @Test - public void shouldNotWaitIfNoOffsetSpecified() throws Exception { + public void shouldNotWaitIfNoSequenceNumberSpecified() throws Exception { // Given: - when(request.getCommandOffset()).thenReturn(Optional.empty()); + when(request.getCommandSequenceNumber()).thenReturn(Optional.empty()); // When: - CommandStoreUtil.waitForCommandOffset(replayableCommandQueue, request, TIMEOUT); + CommandStoreUtil.waitForCommandSequenceNumber(replayableCommandQueue, request, TIMEOUT); // Then: verify(replayableCommandQueue, never()).ensureConsumedUpThrough(anyLong(), anyLong()); } @Test - public void shouldWaitIfOffsetSpecified() throws Exception { + public void shouldWaitIfSequenceNumberSpecified() throws Exception { // Given: - when(request.getCommandOffset()).thenReturn(Optional.of(OFFSET)); + when(request.getCommandSequenceNumber()).thenReturn(Optional.of(SEQUENCE_NUMBER)); // When: - CommandStoreUtil.waitForCommandOffset(replayableCommandQueue, request, TIMEOUT); + CommandStoreUtil.waitForCommandSequenceNumber(replayableCommandQueue, request, TIMEOUT); // Then: - verify(replayableCommandQueue).ensureConsumedUpThrough(OFFSET, TIMEOUT); + verify(replayableCommandQueue).ensureConsumedUpThrough(SEQUENCE_NUMBER, TIMEOUT); } @Test public void shouldThrowKsqlRestExceptionOnTimeout() throws Exception { // Given: - when(request.getCommandOffset()).thenReturn(Optional.of(OFFSET)); + when(request.getCommandSequenceNumber()).thenReturn(Optional.of(SEQUENCE_NUMBER)); doThrow(new TimeoutException("uh oh")) - .when(replayableCommandQueue).ensureConsumedUpThrough(OFFSET, TIMEOUT); + .when(replayableCommandQueue).ensureConsumedUpThrough(SEQUENCE_NUMBER, TIMEOUT); try { // When: - CommandStoreUtil.httpWaitForCommandOffset(replayableCommandQueue, request, TIMEOUT); + CommandStoreUtil.httpWaitForCommandSequenceNumber(replayableCommandQueue, request, TIMEOUT); // Then: fail("Should propagate error."); From b3431304d200d2df8334373387dcd1acaabd76a7 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Thu, 29 Nov 2018 13:20:52 -0800 Subject: [PATCH 05/13] add StreamQueryResource tests for new behavior. --- .../streaming/StreamedQueryResource.java | 2 +- .../resources/StreamedQueryResourceTest.java | 56 ++++++++++++++++++- 2 files changed, 56 insertions(+), 2 deletions(-) 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 d1a87579bd41..61d43f51ffa2 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 @@ -91,7 +91,7 @@ public Response streamQuery(final KsqlRequest request) throws Exception { replayableCommandQueue, request, disconnectCheckInterval.toMillis()); statement = statementParser.parseSingleStatement(ksql); } catch (final KsqlRestException e) { - throw e; + return e.getResponse(); } catch (IllegalArgumentException | KsqlException e) { return Errors.badRequest(e); } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java index e2edb33e3fe3..73331f4ec617 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java @@ -16,6 +16,7 @@ package io.confluent.ksql.rest.server.resources; +import static org.easymock.EasyMock.anyLong; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.anyString; import static org.easymock.EasyMock.expect; @@ -61,9 +62,11 @@ import java.util.Map; import java.util.Scanner; import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; +import junit.framework.AssertionFailedError; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KafkaStreams.State; @@ -89,7 +92,7 @@ public class StreamedQueryResourceTest { private KafkaTopicClient mockKafkaTopicClient; @Mock(MockType.NICE) private StatementParser mockStatementParser; - @Mock(MockType.NICE) + @Mock private ReplayableCommandQueue replayableCommandQueue; @Mock(MockType.NICE) private ActivenessRegistrar activenessRegistrar; @@ -136,6 +139,57 @@ public void shouldReturn400OnBadStatement() throws Exception { assertThat(errorMessage.getMessage(), containsString("some error message")); } + @Test + public void shouldNotWaitIfCommandSequenceNumberSpecified() throws Exception { + // Given: + replayableCommandQueue.ensureConsumedUpThrough(anyLong(), anyLong()); + expectLastCall().andThrow(new AssertionFailedError()).anyTimes(); + + replay(replayableCommandQueue); + + // When: + testResource.streamQuery(new KsqlRequest(queryString, Collections.emptyMap(), null)); + + // Then: + verify(replayableCommandQueue); + } + + @Test + public void shouldWaitIfCommandSequenceNumberSpecified() throws Exception { + // Given: + replayableCommandQueue.ensureConsumedUpThrough(anyLong(), anyLong()); + expectLastCall(); + + replay(replayableCommandQueue); + + // When: + testResource.streamQuery(new KsqlRequest(queryString, Collections.emptyMap(), 3L)); + + // Then: + verify(replayableCommandQueue); + } + + @Test + public void shouldReturn503IfTimeoutWhileWaitingForCommandSequenceNumber() throws Exception { + // Given: + replayableCommandQueue.ensureConsumedUpThrough(anyLong(), anyLong()); + expectLastCall().andThrow(new TimeoutException("whoops")); + + replay(replayableCommandQueue); + + // When: + final Response response = testResource + .streamQuery(new KsqlRequest(queryString, Collections.emptyMap(), 3L)); + + // Then: + assertThat(response.getStatus(), equalTo(Response.Status.SERVICE_UNAVAILABLE.getStatusCode())); + assertThat(response.getEntity(), instanceOf(KsqlErrorMessage.class)); + final KsqlErrorMessage errorMessage = (KsqlErrorMessage)response.getEntity(); + assertThat( + errorMessage.getErrorCode(), equalTo(Errors.ERROR_CODE_COMMAND_QUEUE_CATCHUP_TIMEOUT)); + assertThat(errorMessage.getMessage(), equalTo("whoops")); + } + @SuppressWarnings("unchecked") @Test public void shouldStreamRowsCorrectly() throws Throwable { From e9af7fc8bf4b9ed9aa9f896dfda2c2f263bf6c7b Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Fri, 30 Nov 2018 10:31:34 -0800 Subject: [PATCH 06/13] add documentation. --- docs/developer-guide/api.rst | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/developer-guide/api.rst b/docs/developer-guide/api.rst index 9f297ac95900..b9b810d4a981 100644 --- a/docs/developer-guide/api.rst +++ b/docs/developer-guide/api.rst @@ -103,6 +103,7 @@ The KSQL resource runs a sequence of KSQL statements. All statements, except tho :json string ksql: A semicolon-delimited sequence of KSQL statements to run. :json map streamsProperties: Property overrides to run the statements with. Refer to the :ref:`Config Reference ` for details on properties that can be set. :json string streamsProperties[``property-name``]: The value of the property named by ``property-name``. Both the value and ``property-name`` should be strings. + :json long commandSequenceNumber: Optional. If specified, the statements will not be run until all existing commands up through the specified sequence number have completed. If unspecified, the statements will be run immediately. The response JSON is an array of result objects. The result object contents depend on the statement that it is returning results for. The following sections detail the contents of the result objects by statement. @@ -112,6 +113,7 @@ The KSQL resource runs a sequence of KSQL statements. All statements, except tho :>json string commandId: A string that identifies the requested operation. You can use this ID to poll the result of the operation using the status endpoint. :>json string commandStatus.status: One of QUEUED, PARSING, EXECUTING, TERMINATED, SUCCESS, or ERROR. :>json string commandStatus.message: Detailed message regarding the status of the execution statement. + :>json long commandSequenceNumber: The sequence number of the requested operation in the command queue. **LIST STREAMS, SHOW STREAMS** From d89059ec7a9eeb2fd46811a0ccf5098636104c2c Mon Sep 17 00:00:00 2001 From: Jim Galasyn Date: Fri, 30 Nov 2018 12:48:02 -0800 Subject: [PATCH 07/13] jim's requested edit. Co-Authored-By: vcrfxia --- docs/developer-guide/api.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/developer-guide/api.rst b/docs/developer-guide/api.rst index b9b810d4a981..506609f112b3 100644 --- a/docs/developer-guide/api.rst +++ b/docs/developer-guide/api.rst @@ -103,7 +103,7 @@ The KSQL resource runs a sequence of KSQL statements. All statements, except tho :json string ksql: A semicolon-delimited sequence of KSQL statements to run. :json map streamsProperties: Property overrides to run the statements with. Refer to the :ref:`Config Reference ` for details on properties that can be set. :json string streamsProperties[``property-name``]: The value of the property named by ``property-name``. Both the value and ``property-name`` should be strings. - :json long commandSequenceNumber: Optional. If specified, the statements will not be run until all existing commands up through the specified sequence number have completed. If unspecified, the statements will be run immediately. + :json long commandSequenceNumber: Optional. If specified, the statements will not be run until all existing commands up through the specified sequence number have completed. If unspecified, the statements are run immediately. The response JSON is an array of result objects. The result object contents depend on the statement that it is returning results for. The following sections detail the contents of the result objects by statement. From 44f5b28d770a9a0fdad07de13f84eb0310c154c8 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Mon, 3 Dec 2018 13:45:05 -0800 Subject: [PATCH 08/13] rohan's requested changes. --- .../ksql/rest/entity/CommandStatusEntity.java | 27 +++------------- .../ksql/rest/server/KsqlRestApplication.java | 7 ++++- .../server/computation/CommandRunner.java | 4 --- .../rest/server/computation/CommandStore.java | 13 ++------ .../SequenceNumberFutureStore.java | 16 +++++++--- .../rest/server/resources/KsqlResource.java | 2 +- .../streaming/StreamedQueryResource.java | 2 -- .../resources/streaming/WSQueryEndpoint.java | 13 +++++--- .../server/computation/CommandStoreTest.java | 31 +++---------------- .../SequenceNumberFutureStoreTest.java | 6 ++-- .../server/resources/KsqlResourceTest.java | 14 ++++++--- .../resources/StreamedQueryResourceTest.java | 31 ++++++++++--------- 12 files changed, 67 insertions(+), 99 deletions(-) diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java index 9d65117ea6f2..033275fc0270 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.confluent.ksql.rest.server.computation.CommandId; -import java.util.Map; import java.util.Objects; @JsonIgnoreProperties(ignoreUnknown = true) @@ -30,11 +29,12 @@ public class CommandStatusEntity extends KsqlEntity { private final CommandStatus commandStatus; private final long commandSequenceNumber; + @JsonCreator public CommandStatusEntity( - final String statementText, - final CommandId commandId, - final CommandStatus commandStatus, - final long commandSequenceNumber + @JsonProperty("statementText") final String statementText, + @JsonProperty("commandId") final CommandId commandId, + @JsonProperty("commandStatus") final CommandStatus commandStatus, + @JsonProperty("commandSequenceNumber") final long commandSequenceNumber ) { super(statementText); this.commandId = commandId; @@ -42,23 +42,6 @@ public CommandStatusEntity( this.commandSequenceNumber = commandSequenceNumber; } - @JsonCreator - public CommandStatusEntity( - @JsonProperty("statementText") final String statementText, - @JsonProperty("commandId") final String commandId, - @JsonProperty("commandStatus") final Map commandStatus, - @JsonProperty("commandSequenceNumber") final Long commandSequenceNumber - ) { - this( - statementText, - CommandId.fromString(commandId), - new CommandStatus( - CommandStatus.Status.valueOf((String) commandStatus.get("status")), - (String) commandStatus.get("message")), - commandSequenceNumber == null ? -1 : commandSequenceNumber - ); - } - public CommandId getCommandId() { return commandId; } 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 d50433486953..b82ba78db031 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 @@ -44,6 +44,7 @@ import io.confluent.ksql.rest.server.computation.CommandIdAssigner; import io.confluent.ksql.rest.server.computation.CommandRunner; import io.confluent.ksql.rest.server.computation.CommandStore; +import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; import io.confluent.ksql.rest.server.computation.StatementExecutor; import io.confluent.ksql.rest.server.resources.KsqlExceptionMapper; import io.confluent.ksql.rest.server.resources.KsqlResource; @@ -109,6 +110,7 @@ public final class KsqlRestApplication extends Application imple private final KsqlConfig ksqlConfig; private final KsqlEngine ksqlEngine; private final CommandRunner commandRunner; + private final ReplayableCommandQueue replayableCommandQueue; private final RootDocument rootDocument; private final StatusResource statusResource; private final StreamedQueryResource streamedQueryResource; @@ -127,6 +129,7 @@ private KsqlRestApplication( final KsqlConfig ksqlConfig, final KsqlRestConfig config, final CommandRunner commandRunner, + final ReplayableCommandQueue replayableCommandQueue, final RootDocument rootDocument, final StatusResource statusResource, final StreamedQueryResource streamedQueryResource, @@ -137,6 +140,7 @@ private KsqlRestApplication( this.ksqlConfig = ksqlConfig; this.ksqlEngine = ksqlEngine; this.commandRunner = commandRunner; + this.replayableCommandQueue = replayableCommandQueue; this.rootDocument = rootDocument; this.statusResource = statusResource; this.streamedQueryResource = streamedQueryResource; @@ -264,7 +268,7 @@ public T getEndpointInstance(final Class endpointClass) { JsonMapper.INSTANCE.mapper, statementParser, ksqlEngine, - commandRunner.getCommandStore(), + replayableCommandQueue, exec, versionCheckerAgent::updateLastRequestTime, config.getLong(KsqlRestConfig.DISTRIBUTED_COMMAND_RESPONSE_TIMEOUT_MS_CONFIG) @@ -393,6 +397,7 @@ public static KsqlRestApplication buildApplication( ksqlConfig, restConfig, commandRunner, + commandStore, rootDocument, statusResource, streamedQueryResource, diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index caf96fe003c5..37cad01b3486 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -86,10 +86,6 @@ public void close() { } } - public ReplayableCommandQueue getCommandStore() { - return commandStore; - } - void fetchAndRunCommands() { final List commands = commandStore.getNewCommands(); log.trace("Found {} new writes to command topic", commands.size()); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 60e958c12214..80df96e26631 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -217,11 +217,6 @@ public List getRestoreCommands() { @Override public void ensureConsumedUpThrough(final long seqNum, final long timeout) throws TimeoutException { - final long consumerPosition = getNextConsumerSequenceNumber(); - if (consumerPosition > seqNum) { - return; - } - final CompletableFuture future = sequenceNumberFutureStore.getFutureForSequenceNumber(seqNum); try { @@ -241,10 +236,6 @@ public void ensureConsumedUpThrough(final long seqNum, final long timeout) } } - private long getNextConsumerSequenceNumber() { - return commandConsumer.position(topicPartition); - } - private Collection getTopicPartitionsForTopic(final String topic) { final List partitionInfoList = commandConsumer.partitionsFor(topic); @@ -256,7 +247,7 @@ private Collection getTopicPartitionsForTopic(final String topic } private void completeSatisfiedSequenceNumberFutures() { - final long consumerPosition = getNextConsumerSequenceNumber(); - sequenceNumberFutureStore.completeFuturesUpToSequenceNumber(consumerPosition); + final long consumerPosition = commandConsumer.position(topicPartition);; + sequenceNumberFutureStore.completeFuturesUpThroughSequenceNumber(consumerPosition - 1); } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java index 4c59f8e4d850..72f008508e75 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java @@ -21,18 +21,26 @@ public class SequenceNumberFutureStore { private final ConcurrentHashMap> sequenceNumberFutures; + private volatile long lastCompletedSequenceNumber; public SequenceNumberFutureStore() { - sequenceNumberFutures = new ConcurrentHashMap<>(8, 0.9f, 1); + sequenceNumberFutures = new ConcurrentHashMap<>(); + lastCompletedSequenceNumber = -1; } - public CompletableFuture getFutureForSequenceNumber(final long seqNum) { + public synchronized CompletableFuture getFutureForSequenceNumber(final long seqNum) { + if (seqNum <= lastCompletedSequenceNumber) { + return CompletableFuture.completedFuture(null); + } return sequenceNumberFutures.computeIfAbsent(seqNum, k -> new CompletableFuture<>()); } - public void completeFuturesUpToSequenceNumber(final long seqNum) { + public void completeFuturesUpThroughSequenceNumber(final long seqNum) { + synchronized (this) { + lastCompletedSequenceNumber = seqNum; + } sequenceNumberFutures.keySet().stream() - .filter(k -> k < seqNum) + .filter(k -> k <= seqNum) .forEach(k -> { sequenceNumberFutures.get(k).complete(null); sequenceNumberFutures.remove(k); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index 4d9ce1a65249..b74f99bc177a 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -198,7 +198,7 @@ public Response handleKsqlStatements(final KsqlRequest request) { return executeStatements(statements, request.getStreamsProperties()); } catch (final KsqlRestException e) { - return e.getResponse(); + throw e; } catch (final KsqlStatementException e) { return Errors.badStatement(e.getRawMessage(), e.getSqlStatement()); } catch (final KsqlException e) { 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 61d43f51ffa2..9742cb5a5822 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 @@ -90,8 +90,6 @@ public Response streamQuery(final KsqlRequest request) throws Exception { CommandStoreUtil.httpWaitForCommandSequenceNumber( replayableCommandQueue, request, disconnectCheckInterval.toMillis()); statement = statementParser.parseSingleStatement(ksql); - } catch (final KsqlRestException e) { - return e.getResponse(); } catch (IllegalArgumentException | KsqlException e) { return Errors.badRequest(e); } 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 09cb104bc18c..0c1bf2442799 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 @@ -133,17 +133,20 @@ public void onOpen(final Session session, final EndpointConfig unused) { validateVersion(session); final KsqlRequest request = parseRequest(session); - CommandStoreUtil.waitForCommandSequenceNumber(replayableCommandQueue, request, - commandQueueCatchupTimeout); + + try { + CommandStoreUtil.waitForCommandSequenceNumber(replayableCommandQueue, request, + commandQueueCatchupTimeout); + } catch (final TimeoutException e) { + log.debug("Timeout while processing request", e); + SessionUtil.closeSilently(session, CloseCodes.TRY_AGAIN_LATER, e.getMessage()); + } final Statement statement = parseStatement(request); HANDLER_MAP .getOrDefault(statement.getClass(), WSQueryEndpoint::handleUnsupportedStatement) .handle(this, new SessionAndRequest(session, request), statement); - } catch (final TimeoutException e) { - log.debug("Timeout while processing request", e); - SessionUtil.closeSilently(session, CloseCodes.TRY_AGAIN_LATER, e.getMessage()); } catch (final Exception e) { log.debug("Error processing request", e); SessionUtil.closeSilently(session, CloseCodes.CANNOT_ACCEPT, e.getMessage()); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index 12f31896533b..ec9532ff811c 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -325,24 +325,8 @@ public void shouldIncludeCommandSequenceNumberInSuccessfulQueuedCommandStatus() } @Test - public void shouldNotWaitIfSequenceNumberReached() throws Exception { + public void shouldWaitOnSequenceNumberFuture() throws Exception { // Given: - givenCmdStoreUpThroughPosition(1); - expect(sequenceNumberFutureStore.getFutureForSequenceNumber(EasyMock.anyLong())) - .andThrow(new AssertionError()).anyTimes(); - replay(sequenceNumberFutureStore); - - // When: - commandStore.ensureConsumedUpThrough(0, TIMEOUT); - - // Then: - verify(commandConsumer, sequenceNumberFutureStore); - } - - @Test - public void shouldWaitIfSequenceNumberNotReached() throws Exception { - // Given: - givenCmdStoreUpThroughPosition(2); expect(sequenceNumberFutureStore.getFutureForSequenceNumber(EasyMock.anyLong())).andReturn(future); expect(future.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))).andReturn(null); replay(future, sequenceNumberFutureStore); @@ -351,13 +335,12 @@ public void shouldWaitIfSequenceNumberNotReached() throws Exception { commandStore.ensureConsumedUpThrough(2, TIMEOUT); // Then: - verify(commandConsumer, sequenceNumberFutureStore, future); + verify(sequenceNumberFutureStore, future); } @Test public void shouldThrowExceptionOnTimeout() throws Exception { // Given: - givenCmdStoreUpThroughPosition(0); expect(sequenceNumberFutureStore.getFutureForSequenceNumber(EasyMock.anyLong())).andReturn(future); expect(future.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))) .andThrow(new TimeoutException()); @@ -374,13 +357,13 @@ public void shouldThrowExceptionOnTimeout() throws Exception { is(String.format( "Timeout reached while waiting for command sequence number of 2. (Timeout: %d ms)", TIMEOUT))); } - verify(commandConsumer, future, sequenceNumberFutureStore); + verify(future, sequenceNumberFutureStore); } @Test public void shouldCompleteFuturesWhenGettingNewCommands() { // Given: - sequenceNumberFutureStore.completeFuturesUpToSequenceNumber(EasyMock.anyLong()); + sequenceNumberFutureStore.completeFuturesUpThroughSequenceNumber(EasyMock.anyLong()); expectLastCall(); expect(commandConsumer.poll(anyObject(Duration.class))).andReturn(buildRecords()); replay(sequenceNumberFutureStore, commandConsumer); @@ -400,12 +383,6 @@ private void setupConsumerToReturnCommand(final CommandId commandId, final Comma replay(commandConsumer); } - private void givenCmdStoreUpThroughPosition(long position) { - reset(commandConsumer); - expect(commandConsumer.position(anyObject(TopicPartition.class))).andReturn(position); - replay(commandConsumer); - } - private void givenCommandStoreThatAssignsSameId(final CommandId commandId) { commandIdAssigner = mock(CommandIdAssigner.class); expect(commandIdAssigner.getCommandId(anyObject())).andStubAnswer( diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java index fbff1755e3d0..c46ffcc140b2 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java @@ -64,14 +64,14 @@ public void shouldReturnFutureForExistingSequenceNumber() { public void shouldReturnFutureForCompletedSequenceNumber() { // Given: final CompletableFuture firstFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); - sequenceNumberFutureStore.completeFuturesUpToSequenceNumber(3); + sequenceNumberFutureStore.completeFuturesUpThroughSequenceNumber(2); assertFutureIsCompleted(firstFuture); // When: final CompletableFuture secondFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); // Then: - assertFutureIsNotCompleted(secondFuture); + assertFutureIsCompleted(secondFuture); assertThat(secondFuture, is(not(sameInstance(firstFuture)))); } @@ -82,7 +82,7 @@ public void shouldCompleteFutures() { final CompletableFuture secondFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(3); // When: - sequenceNumberFutureStore.completeFuturesUpToSequenceNumber(3); + sequenceNumberFutureStore.completeFuturesUpThroughSequenceNumber(2); // Then: assertFutureIsCompleted(firstFuture); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 01aafadc105d..d4c3db3dd159 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -1009,7 +1009,7 @@ public void shouldReturn503IfTimeoutWhileWaitingForCommandSequenceNumber() throw @Test public void shouldUpdateTheLastRequestTime() { // When: - ksqlResource.handleKsqlStatements(new KsqlRequest("foo", Collections.emptyMap(), null)); + ksqlResource.handleKsqlStatements(VALID_EXECUTABLE_REQUEST); // Then: verify(activenessRegistrar).updateLastRequestTime(); @@ -1076,10 +1076,14 @@ private KsqlErrorMessage makeFailingRequestWithSequenceNumber( } private KsqlErrorMessage makeFailingRequest(final KsqlRequest ksqlRequest, final Code errorCode) { - final Response response = ksqlResource.handleKsqlStatements(ksqlRequest); - assertThat(response.getStatus(), is(errorCode.getCode())); - assertThat(response.getEntity(), instanceOf(KsqlErrorMessage.class)); - return (KsqlErrorMessage) response.getEntity(); + try { + final Response response = ksqlResource.handleKsqlStatements(ksqlRequest); + assertThat(response.getStatus(), is(errorCode.getCode())); + assertThat(response.getEntity(), instanceOf(KsqlErrorMessage.class)); + return (KsqlErrorMessage) response.getEntity(); + } catch (KsqlRestException e) { + return (KsqlErrorMessage) e.getResponse().getEntity(); + } } private T makeSingleRequest( diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java index 73331f4ec617..1503f5d581a4 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java @@ -29,8 +29,10 @@ import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; import com.fasterxml.jackson.databind.ObjectMapper; import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; @@ -142,9 +144,6 @@ public void shouldReturn400OnBadStatement() throws Exception { @Test public void shouldNotWaitIfCommandSequenceNumberSpecified() throws Exception { // Given: - replayableCommandQueue.ensureConsumedUpThrough(anyLong(), anyLong()); - expectLastCall().andThrow(new AssertionFailedError()).anyTimes(); - replay(replayableCommandQueue); // When: @@ -177,17 +176,21 @@ public void shouldReturn503IfTimeoutWhileWaitingForCommandSequenceNumber() throw replay(replayableCommandQueue); - // When: - final Response response = testResource - .streamQuery(new KsqlRequest(queryString, Collections.emptyMap(), 3L)); - - // Then: - assertThat(response.getStatus(), equalTo(Response.Status.SERVICE_UNAVAILABLE.getStatusCode())); - assertThat(response.getEntity(), instanceOf(KsqlErrorMessage.class)); - final KsqlErrorMessage errorMessage = (KsqlErrorMessage)response.getEntity(); - assertThat( - errorMessage.getErrorCode(), equalTo(Errors.ERROR_CODE_COMMAND_QUEUE_CATCHUP_TIMEOUT)); - assertThat(errorMessage.getMessage(), equalTo("whoops")); + try { + // When: + testResource.streamQuery(new KsqlRequest(queryString, Collections.emptyMap(), 3L)); + + // Then: + fail("Should throw KsqlRestException in response to timeout."); + } catch (KsqlRestException e) { + final Response response = e.getResponse(); + assertThat(response.getStatus(), is(Response.Status.SERVICE_UNAVAILABLE.getStatusCode())); + assertThat(response.getEntity(), instanceOf(KsqlErrorMessage.class)); + final KsqlErrorMessage errorMessage = (KsqlErrorMessage)response.getEntity(); + assertThat( + errorMessage.getErrorCode(), is(Errors.ERROR_CODE_COMMAND_QUEUE_CATCHUP_TIMEOUT)); + assertThat(errorMessage.getMessage(), equalTo("whoops")); + } } @SuppressWarnings("unchecked") From 3463a22cfdd5d73216c37c160e3701f6d9f2ce01 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Mon, 3 Dec 2018 20:07:08 -0800 Subject: [PATCH 09/13] refactor CommandStoreTest to Mockito. --- .../server/computation/CommandStoreTest.java | 283 +++++++----------- 1 file changed, 105 insertions(+), 178 deletions(-) diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index ec9532ff811c..2fc2b8ed65e6 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -16,15 +16,6 @@ package io.confluent.ksql.rest.server.computation; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; -import static org.easymock.EasyMock.mock; -import static org.easymock.EasyMock.niceMock; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.reset; -import static org.easymock.EasyMock.verify; import static org.hamcrest.CoreMatchers.anyOf; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.is; @@ -33,10 +24,13 @@ import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableList; -import io.confluent.ksql.function.InternalFunctionRegistry; -import io.confluent.ksql.metastore.MetaStoreImpl; import io.confluent.ksql.parser.tree.Statement; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.util.KsqlConfig; @@ -65,38 +59,52 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.RecordBatch; -import org.easymock.Capture; -import org.easymock.EasyMock; -import org.easymock.EasyMockRunner; 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.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; @SuppressWarnings("unchecked") -@RunWith(EasyMockRunner.class) +@RunWith(MockitoJUnitRunner.class) public class CommandStoreTest { private static final String COMMAND_TOPIC = "command"; - private static final KsqlConfig KSQL_CONFIG = new KsqlConfig(Collections.emptyMap()); - private static final Map OVERRIDE_PROPERTIES = Collections.emptyMap(); + private static final KsqlConfig KSQL_CONFIG = new KsqlConfig( + Collections.singletonMap(KsqlConfig.KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG, "foo")); + private static final Map OVERRIDE_PROPERTIES = + Collections.singletonMap(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); private static final long TIMEOUT = 1000; private static final String statementText = "test-statement"; - private final Consumer commandConsumer = niceMock(Consumer.class); - private final Producer commandProducer = mock(Producer.class); - private CommandIdAssigner commandIdAssigner = - new CommandIdAssigner(new MetaStoreImpl(new InternalFunctionRegistry())); - private final SequenceNumberFutureStore sequenceNumberFutureStore = mock( - SequenceNumberFutureStore.class); - private final CompletableFuture future = niceMock(CompletableFuture.class); + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Mock + private Consumer commandConsumer; + @Mock + private Producer commandProducer; + @Mock + private SequenceNumberFutureStore sequenceNumberFutureStore; + @Mock + private CompletableFuture future; + @Mock + private Statement statement; + @Mock + private Future recordMetadataFuture; + @Mock + private Node node; + @Mock + private CommandIdAssigner commandIdAssigner; + private final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); - private final Statement statement = mock(Statement.class); - private final Future recordMetadataFuture = niceMock(Future.class); private final Command command = new Command(statementText, Collections.emptyMap(), Collections.emptyMap()); - private final Node node = mock(Node.class); private final RecordMetadata recordMetadata = new RecordMetadata( new TopicPartition("topic", 0), 0, 0, RecordBatch.NO_TIMESTAMP, 0L, 0, 0); @@ -104,14 +112,31 @@ public class CommandStoreTest { @Before public void setUp() throws Exception { - expect(recordMetadataFuture.get()).andReturn(recordMetadata); - replay(recordMetadataFuture); + when(commandIdAssigner.getCommandId(any())).thenReturn(commandId); + + when(commandProducer.send(any(ProducerRecord.class))).thenReturn(recordMetadataFuture); + when(recordMetadataFuture.get()).thenReturn(recordMetadata); + + when(commandConsumer.poll(any(Duration.class))).thenReturn(buildRecords(commandId, command)); + when(commandConsumer.partitionsFor(COMMAND_TOPIC)) + .thenReturn(ImmutableList.of( + new PartitionInfo(COMMAND_TOPIC, 0, node, new Node[]{node}, new Node[]{node}) + )); - setUpCommandStore(); + when(sequenceNumberFutureStore.getFutureForSequenceNumber(anyLong())).thenReturn(future); + + commandStore = new CommandStore( + COMMAND_TOPIC, + commandConsumer, + commandProducer, + commandIdAssigner, + sequenceNumberFutureStore + ); } @Test public void shouldHaveAllCreateCommandsInOrder() { + // Given: final CommandId createId = new CommandId(CommandId.Type.TABLE, "one", CommandId.Action.CREATE); final CommandId dropId = new CommandId(CommandId.Type.TABLE, "one", CommandId.Action.DROP); final KsqlConfig ksqlConfig = new KsqlConfig(Collections.emptyMap()); @@ -123,18 +148,19 @@ public void shouldHaveAllCreateCommandsInOrder() { "a new statement", Collections.emptyMap(), ksqlConfig.getAllConfigPropsWithSecretsObfuscated()); final ConsumerRecords records = buildRecords( - createId, originalCommand, - dropId, dropCommand, - createId, latestCommand + createId, originalCommand, + dropId, dropCommand, + createId, latestCommand ); - EasyMock.expect(commandConsumer.partitionsFor(COMMAND_TOPIC)).andReturn(Collections.emptyList()); - - EasyMock.expect(commandConsumer.poll(anyObject())).andReturn(records) - .andReturn(new ConsumerRecords<>(Collections.emptyMap())); - EasyMock.replay(commandConsumer); + when(commandConsumer.poll(any())) + .thenReturn(records) + .thenReturn(new ConsumerRecords<>(Collections.emptyMap())); + // When: final List> commands = getPriorCommands(commandStore); + + // Then: assertThat(commands, equalTo(Arrays.asList(new Pair<>(createId, originalCommand), new Pair<>(dropId, dropCommand), new Pair<>(createId, latestCommand)))); @@ -142,31 +168,21 @@ public void shouldHaveAllCreateCommandsInOrder() { @Test public void shouldFailEnqueueIfCommandWithSameIdRegistered() { - givenCommandStoreThatAssignsSameId(commandId); - // Given: - expect(commandProducer.send(anyObject())).andReturn(recordMetadataFuture); - replay(commandProducer); - commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); + expectedException.expect(IllegalStateException.class); - try { - // When: - commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); + commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); - // Then: - fail("Second enqueue call should throw IllegalStateException"); - } catch (final IllegalStateException e) { - } + // When: + commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); } @Test public void shouldCleanupCommandStatusOnProduceError() { - givenCommandStoreThatAssignsSameId(commandId); - // Given: - expect(commandProducer.send(anyObject())).andThrow(new RuntimeException("oops")).times(1); - expect(commandProducer.send(anyObject())).andReturn(recordMetadataFuture).times(1); - replay(commandProducer); + when(commandProducer.send(any(ProducerRecord.class))) + .thenThrow(new RuntimeException("oops")) + .thenReturn(recordMetadataFuture); try { commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); fail("enqueueCommand should have raised an exception"); @@ -176,44 +192,26 @@ public void shouldCleanupCommandStatusOnProduceError() { // When: commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); - // Then: - // main condition being verified is that the above call doesn't throw - verify(commandProducer); + // Then: condition being verified is that the above call doesn't throw } @Test - public void shouldEnqueueNewAfterHandlingExistingCommand() throws Exception { - givenCommandStoreThatAssignsSameId(commandId); - + public void shouldEnqueueNewAfterHandlingExistingCommand() { // Given: - setupConsumerToReturnCommand(commandId, command); - expect(commandProducer.send(anyObject(ProducerRecord.class))).andAnswer( - () -> { - commandStore.getNewCommands(); - return recordMetadataFuture; - } - ).times(1); - expect(commandProducer.send(anyObject(ProducerRecord.class))).andReturn(recordMetadataFuture); - reset(recordMetadataFuture); - expect(recordMetadataFuture.get()).andReturn(recordMetadata).times(2); - replay(recordMetadataFuture, commandProducer); commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); + commandStore.getNewCommands(); // When: commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); - // Then: - verify(recordMetadataFuture, commandProducer, commandConsumer); + // Then: condition being verified is that the above call doesn't throw } @Test public void shouldRegisterBeforeDistributeAndReturnStatusOnGetNewCommands() { - givenCommandStoreThatAssignsSameId(commandId); - // Given: - setupConsumerToReturnCommand(commandId, command); - expect(commandProducer.send(anyObject(ProducerRecord.class))).andAnswer( - () -> { + when(commandProducer.send(any(ProducerRecord.class))).thenAnswer( + invocation -> { final QueuedCommand queuedCommand = commandStore.getNewCommands().get(0); assertThat(queuedCommand.getCommandId(), equalTo(commandId)); assertThat(queuedCommand.getStatus().isPresent(), equalTo(true)); @@ -223,34 +221,29 @@ public void shouldRegisterBeforeDistributeAndReturnStatusOnGetNewCommands() { return recordMetadataFuture; } ); - replay(commandProducer); // When: commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); // Then: - // verifying the commandProducer also verifies the assertions in its IAnswer were run - verify(recordMetadataFuture, commandProducer, commandConsumer); + // verifying the commandProducer also verifies the assertions in its Answer were run + verify(commandProducer).send(any(ProducerRecord.class)); } @Test public void shouldFilterNullCommands() { // Given: - final CommandId id = new CommandId(CommandId.Type.TABLE, "one", CommandId.Action.CREATE); - final Command command = new Command( - "some statement", Collections.emptyMap(), Collections.emptyMap()); final ConsumerRecords records = buildRecords( - id, null, - id, command); - expect(commandConsumer.poll(anyObject())).andReturn(records); - replay(commandConsumer); + commandId, null, + commandId, command); + when(commandConsumer.poll(any())).thenReturn(records); // When: final List commands = commandStore.getNewCommands(); // Then: assertThat(commands, hasSize(1)); - assertThat(commands.get(0).getCommandId(), equalTo(id)); + assertThat(commands.get(0).getCommandId(), equalTo(commandId)); assertThat(commands.get(0).getCommand(), equalTo(command)); } @@ -263,13 +256,9 @@ public void shouldFilterNullPriorCommand() { final ConsumerRecords records = buildRecords( id, null, id, command); - expect(commandConsumer.partitionsFor(COMMAND_TOPIC)) - .andReturn(ImmutableList.of( - new PartitionInfo(COMMAND_TOPIC, 0, node, new Node[]{node}, new Node[]{node}) - )); - expect(commandConsumer.poll(anyObject())).andReturn(records); - expect(commandConsumer.poll(anyObject())).andReturn(ConsumerRecords.empty()); - replay(commandConsumer); + when(commandConsumer.poll(any())) + .thenReturn(records) + .thenReturn(ConsumerRecords.empty()); // When: final List commands = commandStore.getRestoreCommands(); @@ -282,124 +271,62 @@ public void shouldFilterNullPriorCommand() { @Test public void shouldDistributeCommand() { - final KsqlConfig ksqlConfig = new KsqlConfig( - Collections.singletonMap(KsqlConfig.KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG, "foo")); - final Map overrideProperties = Collections.singletonMap( - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - final String statementText = "test-statement"; + ArgumentCaptor> recordCaptor = + ArgumentCaptor.forClass(ProducerRecord.class); - final Statement statement = mock(Statement.class); - final Capture> recordCapture = Capture.newInstance(); + when(commandProducer.send(recordCaptor.capture())).thenReturn(recordMetadataFuture); - expect(commandProducer.send(capture(recordCapture))).andReturn(recordMetadataFuture); - replay(commandProducer); - - givenCommandStoreThatAssignsSameId(commandId); - commandStore.enqueueCommand(statementText, statement, ksqlConfig, overrideProperties); - - verify(commandProducer, recordMetadataFuture); + // When: + commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); - final ProducerRecord record = recordCapture.getValue(); + // Then: + final ProducerRecord record = recordCaptor.getValue(); assertThat(record.key(), equalTo(commandId)); assertThat(record.value().getStatement(), equalTo(statementText)); - assertThat(record.value().getOverwriteProperties(), equalTo(overrideProperties)); - assertThat(record.value().getOriginalProperties(), equalTo(ksqlConfig.getAllConfigPropsWithSecretsObfuscated())); + assertThat(record.value().getOverwriteProperties(), equalTo(OVERRIDE_PROPERTIES)); + assertThat(record.value().getOriginalProperties(), + equalTo(KSQL_CONFIG.getAllConfigPropsWithSecretsObfuscated())); } @Test public void shouldIncludeCommandSequenceNumberInSuccessfulQueuedCommandStatus() { - // Given: - givenCommandStoreThatAssignsSameId(commandId); - - expect(commandProducer.send(anyObject(ProducerRecord.class))).andReturn(recordMetadataFuture); - replay(commandProducer); - // When: final QueuedCommandStatus commandStatus = commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); // Then: assertThat(commandStatus.getCommandSequenceNumber(), equalTo(recordMetadata.offset())); - - verify(commandProducer, recordMetadataFuture); } @Test public void shouldWaitOnSequenceNumberFuture() throws Exception { - // Given: - expect(sequenceNumberFutureStore.getFutureForSequenceNumber(EasyMock.anyLong())).andReturn(future); - expect(future.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))).andReturn(null); - replay(future, sequenceNumberFutureStore); - // When: commandStore.ensureConsumedUpThrough(2, TIMEOUT); // Then: - verify(sequenceNumberFutureStore, future); + verify(future).get(eq(TIMEOUT), eq(TimeUnit.MILLISECONDS)); } @Test public void shouldThrowExceptionOnTimeout() throws Exception { // Given: - expect(sequenceNumberFutureStore.getFutureForSequenceNumber(EasyMock.anyLong())).andReturn(future); - expect(future.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))) - .andThrow(new TimeoutException()); - replay(future, sequenceNumberFutureStore); + expectedException.expect(TimeoutException.class); + expectedException.expectMessage(String.format( + "Timeout reached while waiting for command sequence number of 2. (Timeout: %d ms)", TIMEOUT)); - try { - // When: - commandStore.ensureConsumedUpThrough(2, TIMEOUT); - - // Then: - fail("TimeoutException should be propagated."); - } catch (final TimeoutException e) { - assertThat(e.getMessage(), - is(String.format( - "Timeout reached while waiting for command sequence number of 2. (Timeout: %d ms)", TIMEOUT))); - } - verify(future, sequenceNumberFutureStore); + when(future.get(anyLong(), any(TimeUnit.class))).thenThrow(new TimeoutException()); + + // When: + commandStore.ensureConsumedUpThrough(2, TIMEOUT); } @Test public void shouldCompleteFuturesWhenGettingNewCommands() { - // Given: - sequenceNumberFutureStore.completeFuturesUpThroughSequenceNumber(EasyMock.anyLong()); - expectLastCall(); - expect(commandConsumer.poll(anyObject(Duration.class))).andReturn(buildRecords()); - replay(sequenceNumberFutureStore, commandConsumer); - // When: commandStore.getNewCommands(); // Then: - verify(sequenceNumberFutureStore); - } - - private void setupConsumerToReturnCommand(final CommandId commandId, final Command command) { - reset(commandConsumer); - expect(commandConsumer.poll(anyObject(Duration.class))).andReturn( - buildRecords(commandId, command) - ).times(1); - replay(commandConsumer); - } - - private void givenCommandStoreThatAssignsSameId(final CommandId commandId) { - commandIdAssigner = mock(CommandIdAssigner.class); - expect(commandIdAssigner.getCommandId(anyObject())).andStubAnswer( - () -> new CommandId(commandId.getType(), commandId.getEntity(), commandId.getAction()) - ); - replay(commandIdAssigner); - setUpCommandStore(); - } - - private void setUpCommandStore() { - commandStore = new CommandStore( - COMMAND_TOPIC, - commandConsumer, - commandProducer, - commandIdAssigner, - sequenceNumberFutureStore - ); + verify(sequenceNumberFutureStore).completeFuturesUpThroughSequenceNumber(eq(-1L)); } private static List> getPriorCommands(final CommandStore commandStore) { From c5167232d8527d7a025e0ae3510cf664444dcf88 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Wed, 5 Dec 2018 15:37:18 -0800 Subject: [PATCH 10/13] latest requested changes. --- docs/developer-guide/api.rst | 4 +- .../ksql/cli/console/ConsoleTest.java | 2 +- .../ksql/rest/entity/CommandStatusEntity.java | 12 +-- .../ksql/rest/server/KsqlRestApplication.java | 3 +- .../computation/CommandStatusFuture.java | 28 +++---- .../rest/server/computation/CommandStore.java | 28 ++++--- .../computation/QueuedCommandStatus.java | 6 +- .../computation/ReplayableCommandQueue.java | 4 +- .../SequenceNumberFutureStore.java | 10 +-- .../rest/server/resources/KsqlResource.java | 2 +- .../streaming/StreamedQueryResource.java | 5 +- .../resources/streaming/WSQueryEndpoint.java | 17 +++- .../ksql/rest/util/CommandStoreUtil.java | 13 ++- .../rest/entity/CommandStatusEntityTest.java | 55 +++++++++++-- .../server/computation/CommandStoreTest.java | 79 +++++++++++-------- .../rest/server/computation/RecoveryTest.java | 4 +- .../SequenceNumberFutureStoreTest.java | 11 +-- .../server/resources/KsqlResourceTest.java | 16 ++-- .../resources/StreamedQueryResourceTest.java | 9 ++- .../streaming/WSQueryEndpointTest.java | 10 ++- .../ksql/rest/util/CommandStoreUtilTest.java | 10 ++- 21 files changed, 206 insertions(+), 122 deletions(-) diff --git a/docs/developer-guide/api.rst b/docs/developer-guide/api.rst index 506609f112b3..00e66ccd6ce5 100644 --- a/docs/developer-guide/api.rst +++ b/docs/developer-guide/api.rst @@ -103,7 +103,7 @@ The KSQL resource runs a sequence of KSQL statements. All statements, except tho :json string ksql: A semicolon-delimited sequence of KSQL statements to run. :json map streamsProperties: Property overrides to run the statements with. Refer to the :ref:`Config Reference ` for details on properties that can be set. :json string streamsProperties[``property-name``]: The value of the property named by ``property-name``. Both the value and ``property-name`` should be strings. - :json long commandSequenceNumber: Optional. If specified, the statements will not be run until all existing commands up through the specified sequence number have completed. If unspecified, the statements are run immediately. + :json long commandSequenceNumber: Optional. If specified, the statements will not be run until all existing commands up to and including the specified sequence number have completed. If unspecified, the statements are run immediately. (Note: When a command is processed, the result object contains its sequence number.) The response JSON is an array of result objects. The result object contents depend on the statement that it is returning results for. The following sections detail the contents of the result objects by statement. @@ -113,7 +113,7 @@ The KSQL resource runs a sequence of KSQL statements. All statements, except tho :>json string commandId: A string that identifies the requested operation. You can use this ID to poll the result of the operation using the status endpoint. :>json string commandStatus.status: One of QUEUED, PARSING, EXECUTING, TERMINATED, SUCCESS, or ERROR. :>json string commandStatus.message: Detailed message regarding the status of the execution statement. - :>json long commandSequenceNumber: The sequence number of the requested operation in the command queue. + :>json long commandSequenceNumber: The sequence number of the requested operation in the command queue, or -1 if the operation was unsuccessful. **LIST STREAMS, SHOW STREAMS** diff --git a/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java b/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java index 0e8da810ee89..a543b532724d 100644 --- a/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java +++ b/ksql-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java @@ -133,7 +133,7 @@ public void testPrintKSqlEntityList() throws IOException { "e", CommandId.fromString("topic/1/create"), new CommandStatus(CommandStatus.Status.SUCCESS, "Success Message"), - 0), + 0L), new PropertiesList("e", properties, Collections.emptyList(), Collections.emptyList()), new Queries("e", queries), new SourceDescriptionEntity( diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java index 033275fc0270..4135cd97fdcd 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/CommandStatusEntity.java @@ -34,12 +34,12 @@ public CommandStatusEntity( @JsonProperty("statementText") final String statementText, @JsonProperty("commandId") final CommandId commandId, @JsonProperty("commandStatus") final CommandStatus commandStatus, - @JsonProperty("commandSequenceNumber") final long commandSequenceNumber + @JsonProperty("commandSequenceNumber") final Long commandSequenceNumber ) { super(statementText); - this.commandId = commandId; - this.commandStatus = commandStatus; - this.commandSequenceNumber = commandSequenceNumber; + this.commandId = Objects.requireNonNull(commandId, "commandId"); + this.commandStatus = Objects.requireNonNull(commandStatus, "commandStatus"); + this.commandSequenceNumber = commandSequenceNumber == null ? -1 : commandSequenceNumber; } public CommandId getCommandId() { @@ -51,7 +51,7 @@ public CommandStatus getCommandStatus() { return commandStatus; } - public long getCommandSequenceNumber() { + public Long getCommandSequenceNumber() { return commandSequenceNumber; } @@ -71,7 +71,7 @@ public boolean equals(final Object o) { @Override public int hashCode() { - return Objects.hash(getCommandId(), getCommandStatus(), getCommandSequenceNumber()); + return Objects.hash(commandId, commandStatus, commandSequenceNumber); } @Override 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 b82ba78db031..bd7957c16c2e 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 @@ -271,7 +271,8 @@ public T getEndpointInstance(final Class endpointClass) { replayableCommandQueue, exec, versionCheckerAgent::updateLastRequestTime, - config.getLong(KsqlRestConfig.DISTRIBUTED_COMMAND_RESPONSE_TIMEOUT_MS_CONFIG) + Duration.ofMillis(config.getLong( + KsqlRestConfig.DISTRIBUTED_COMMAND_RESPONSE_TIMEOUT_MS_CONFIG)) ); } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java index db82140269c0..727c95f225b4 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java @@ -29,39 +29,39 @@ public class CommandStatusFuture { CommandStatus.Status.QUEUED, "Statement written to command topic"); private final CommandId commandId; - private volatile CommandStatus commandStatus; - private final CompletableFuture future; + private volatile CommandStatus currentStatus; + private final CompletableFuture finalStatusFuture; public CommandStatusFuture(final CommandId commandId) { this.commandId = Objects.requireNonNull(commandId, "commandId cannot be null"); - this.commandStatus = INITIAL_STATUS; - this.future = new CompletableFuture<>(); + this.currentStatus = INITIAL_STATUS; + this.finalStatusFuture = new CompletableFuture<>(); } - public CommandId getCommandId() { + CommandId getCommandId() { return commandId; } - public CommandStatus getStatus() { - return commandStatus; + CommandStatus getStatus() { + return currentStatus; } - public CommandStatus tryWaitForFinalStatus(final Duration timeout) throws InterruptedException { + CommandStatus tryWaitForFinalStatus(final Duration timeout) throws InterruptedException { try { - return future.get(timeout.toMillis(), TimeUnit.MILLISECONDS); + return finalStatusFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS); } catch (final ExecutionException e) { throw new RuntimeException("Error executing command " + commandId, e.getCause()); } catch (final TimeoutException e) { - return commandStatus; + return currentStatus; } } - public void setStatus(final CommandStatus status) { - this.commandStatus = Objects.requireNonNull(status); + void setStatus(final CommandStatus status) { + this.currentStatus = Objects.requireNonNull(status); } - public void setFinalStatus(final CommandStatus status) { + void setFinalStatus(final CommandStatus status) { setStatus(status); - future.complete(status); + finalStatusFuture.complete(status); } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 80df96e26631..6614777f6d31 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -28,6 +28,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -86,13 +87,14 @@ public CommandStore( final CommandIdAssigner commandIdAssigner, final SequenceNumberFutureStore sequenceNumberFutureStore ) { - this.commandTopic = commandTopic; + this.commandTopic = Objects.requireNonNull(commandTopic, "commandTopic"); this.topicPartition = new TopicPartition(commandTopic, 0); - this.commandConsumer = commandConsumer; - this.commandProducer = commandProducer; - this.commandIdAssigner = commandIdAssigner; + this.commandConsumer = Objects.requireNonNull(commandConsumer, "commandConsumer"); + this.commandProducer = Objects.requireNonNull(commandProducer, "commandProducer"); + this.commandIdAssigner = Objects.requireNonNull(commandIdAssigner, "commandIdAssigner"); this.commandStatusMap = Maps.newConcurrentMap(); - this.sequenceNumberFutureStore = sequenceNumberFutureStore; + this.sequenceNumberFutureStore = + Objects.requireNonNull(sequenceNumberFutureStore, "sequenceNumberFutureStore"); commandConsumer.assign(Collections.singleton(topicPartition)); } @@ -215,24 +217,24 @@ public List getRestoreCommands() { } @Override - public void ensureConsumedUpThrough(final long seqNum, final long timeout) - throws TimeoutException { + public void ensureConsumedPast(final long seqNum, final Duration timeout) + throws InterruptedException, TimeoutException { final CompletableFuture future = sequenceNumberFutureStore.getFutureForSequenceNumber(seqNum); try { - future.get(timeout, TimeUnit.MILLISECONDS); + future.get(timeout.toMillis(), TimeUnit.MILLISECONDS); } catch (final ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException)e.getCause(); + } throw new RuntimeException( "Error waiting for command sequence number of " + seqNum, e.getCause()); - } catch (final InterruptedException e) { - throw new RuntimeException( - "Interrupted while waiting for command sequence number of " + seqNum, e); } catch (final TimeoutException e) { throw new TimeoutException( String.format( "Timeout reached while waiting for command sequence number of %d. (Timeout: %d ms)", seqNum, - timeout)); + timeout.toMillis())); } } @@ -248,6 +250,6 @@ private Collection getTopicPartitionsForTopic(final String topic private void completeSatisfiedSequenceNumberFutures() { final long consumerPosition = commandConsumer.position(topicPartition);; - sequenceNumberFutureStore.completeFuturesUpThroughSequenceNumber(consumerPosition - 1); + sequenceNumberFutureStore.completeFuturesUpToAndIncludingSequenceNumber(consumerPosition - 1); } } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java index 83f86e593c8f..add3c20cc121 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/QueuedCommandStatus.java @@ -19,6 +19,7 @@ import io.confluent.ksql.rest.entity.CommandStatus; import java.time.Duration; +import java.util.Objects; public class QueuedCommandStatus { private final CommandStatusFuture commandStatusFuture; @@ -26,8 +27,9 @@ public class QueuedCommandStatus { public QueuedCommandStatus( final long commandSequenceNumber, final CommandStatusFuture commandStatusFuture) { - this.commandSequenceNumber = commandSequenceNumber; - this.commandStatusFuture = commandStatusFuture; + this.commandSequenceNumber = + Objects.requireNonNull(commandSequenceNumber, "commandSequenceNumber"); + this.commandStatusFuture = Objects.requireNonNull(commandStatusFuture, "commandStatusFuture"); } public CommandStatus getStatus() { diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java index 0727c8a3326c..e84855f9ddbc 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ReplayableCommandQueue.java @@ -20,6 +20,7 @@ import io.confluent.ksql.util.KsqlConfig; import java.io.Closeable; +import java.time.Duration; import java.util.List; import java.util.Map; import java.util.concurrent.TimeoutException; @@ -36,5 +37,6 @@ QueuedCommandStatus enqueueCommand( List getRestoreCommands(); - void ensureConsumedUpThrough(long seqNum, long timeout) throws TimeoutException; + void ensureConsumedPast(long seqNum, Duration timeout) + throws InterruptedException, TimeoutException; } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java index 72f008508e75..8ecea895fe71 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStore.java @@ -19,23 +19,23 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -public class SequenceNumberFutureStore { +class SequenceNumberFutureStore { private final ConcurrentHashMap> sequenceNumberFutures; - private volatile long lastCompletedSequenceNumber; + private long lastCompletedSequenceNumber; - public SequenceNumberFutureStore() { + SequenceNumberFutureStore() { sequenceNumberFutures = new ConcurrentHashMap<>(); lastCompletedSequenceNumber = -1; } - public synchronized CompletableFuture getFutureForSequenceNumber(final long seqNum) { + synchronized CompletableFuture getFutureForSequenceNumber(final long seqNum) { if (seqNum <= lastCompletedSequenceNumber) { return CompletableFuture.completedFuture(null); } return sequenceNumberFutures.computeIfAbsent(seqNum, k -> new CompletableFuture<>()); } - public void completeFuturesUpThroughSequenceNumber(final long seqNum) { + void completeFuturesUpToAndIncludingSequenceNumber(final long seqNum) { synchronized (this) { lastCompletedSequenceNumber = seqNum; } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index b74f99bc177a..a26ad70456d1 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -192,7 +192,7 @@ public Response handleKsqlStatements(final KsqlRequest request) { try { CommandStoreUtil.httpWaitForCommandSequenceNumber( - replayableCommandQueue, request, distributedCommandResponseTimeout); + replayableCommandQueue, request, Duration.ofMillis(distributedCommandResponseTimeout)); final List> statements = parseStatements(request.getKsql()); 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 9742cb5a5822..90ca9b98576b 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 @@ -86,9 +86,10 @@ public Response streamQuery(final KsqlRequest request) throws Exception { return Errors.badRequest("\"ksql\" field must be populated"); } activenessRegistrar.updateLastRequestTime(); + + CommandStoreUtil.httpWaitForCommandSequenceNumber( + replayableCommandQueue, request, disconnectCheckInterval); try { - CommandStoreUtil.httpWaitForCommandSequenceNumber( - replayableCommandQueue, request, disconnectCheckInterval.toMillis()); statement = statementParser.parseSingleStatement(ksql); } catch (IllegalArgumentException | KsqlException e) { return Errors.badRequest(e); 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 0c1bf2442799..ee49f800b2fc 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 @@ -34,6 +34,7 @@ import io.confluent.ksql.util.HandlerMaps.ClassHandlerMap2; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.version.metrics.ActivenessRegistrar; +import java.time.Duration; import java.util.Collections; import java.util.List; import java.util.Map; @@ -72,7 +73,7 @@ public class WSQueryEndpoint { private final ActivenessRegistrar activenessRegistrar; private final QueryPublisher queryPublisher; private final PrintTopicPublisher topicPublisher; - private final long commandQueueCatchupTimeout; + private final Duration commandQueueCatchupTimeout; private WebSocketSubscriber subscriber; @@ -84,7 +85,7 @@ public WSQueryEndpoint( final ReplayableCommandQueue replayableCommandQueue, final ListeningScheduledExecutorService exec, final ActivenessRegistrar activenessRegistrar, - final long commandQueueCatchupTimeout + final Duration commandQueueCatchupTimeout ) { this(ksqlConfig, mapper, @@ -108,7 +109,7 @@ public WSQueryEndpoint( final QueryPublisher queryPublisher, final PrintTopicPublisher topicPublisher, final ActivenessRegistrar activenessRegistrar, - final long commandQueueCatchupTimeout + final Duration commandQueueCatchupTimeout ) { this.ksqlConfig = Objects.requireNonNull(ksqlConfig, "ksqlConfig"); this.mapper = Objects.requireNonNull(mapper, "mapper"); @@ -121,7 +122,8 @@ public WSQueryEndpoint( this.topicPublisher = Objects.requireNonNull(topicPublisher, "topicPublisher"); this.activenessRegistrar = Objects.requireNonNull(activenessRegistrar, "activenessRegistrar"); - this.commandQueueCatchupTimeout = commandQueueCatchupTimeout; + this.commandQueueCatchupTimeout = + Objects.requireNonNull(commandQueueCatchupTimeout, "commandQueueCatchupTimeout"); } @SuppressWarnings("unused") @@ -137,9 +139,16 @@ public void onOpen(final Session session, final EndpointConfig unused) { try { CommandStoreUtil.waitForCommandSequenceNumber(replayableCommandQueue, request, commandQueueCatchupTimeout); + } catch (final InterruptedException e) { + log.debug("Interrupted while waiting for command queue " + + "to reach specified command sequence number", + e); + SessionUtil.closeSilently(session, CloseCodes.UNEXPECTED_CONDITION, e.getMessage()); + return; } catch (final TimeoutException e) { log.debug("Timeout while processing request", e); SessionUtil.closeSilently(session, CloseCodes.TRY_AGAIN_LATER, e.getMessage()); + return; } final Statement statement = parseStatement(request); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java index 2925c0e9f501..03d5a76da35b 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandStoreUtil.java @@ -16,10 +16,12 @@ package io.confluent.ksql.rest.util; +import io.confluent.ksql.rest.entity.KsqlEntityList; import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; import io.confluent.ksql.rest.server.resources.Errors; import io.confluent.ksql.rest.server.resources.KsqlRestException; +import java.time.Duration; import java.util.Optional; import java.util.concurrent.TimeoutException; @@ -30,9 +32,14 @@ private CommandStoreUtil() { public static void httpWaitForCommandSequenceNumber( final ReplayableCommandQueue replayableCommandQueue, final KsqlRequest request, - final long timeout) { + final Duration timeout) { try { waitForCommandSequenceNumber(replayableCommandQueue, request, timeout); + } catch (final InterruptedException e) { + final String errorMsg = "Interrupted while waiting for command queue to reach " + + "specified command sequence number in request: " + request.getKsql(); + throw new KsqlRestException( + Errors.serverErrorForStatement(e, errorMsg, new KsqlEntityList())); } catch (final TimeoutException e) { throw new KsqlRestException(Errors.commandQueueCatchUpTimeout(e.getMessage())); } @@ -41,11 +48,11 @@ public static void httpWaitForCommandSequenceNumber( public static void waitForCommandSequenceNumber( final ReplayableCommandQueue replayableCommandQueue, final KsqlRequest request, - final long timeout) throws TimeoutException { + final Duration timeout) throws InterruptedException, TimeoutException { final Optional commandSequenceNumber = request.getCommandSequenceNumber(); if (commandSequenceNumber.isPresent()) { final long seqNum = commandSequenceNumber.get(); - replayableCommandQueue.ensureConsumedUpThrough(seqNum, timeout); + replayableCommandQueue.ensureConsumedPast(seqNum, timeout); } } } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java index 69913539f5af..245ebe6890e3 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java @@ -21,7 +21,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.confluent.ksql.rest.server.computation.CommandId; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; public class CommandStatusEntityTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @@ -34,25 +36,66 @@ public class CommandStatusEntityTest { + "\"status\":\"SUCCESS\"," + "\"message\":\"some success message\"" + "}," - + "\"commandSequenceNumber\":0" + + "\"commandSequenceNumber\":2" + "}"; - private static final CommandStatusEntity ENTITY = new CommandStatusEntity( - "sql", - CommandId.fromString("topic/1/create"), - new CommandStatus(CommandStatus.Status.SUCCESS, "some success message"), - 0); + private static final String STATEMENT_TEXT = "sql"; + private static final CommandId COMMAND_ID = CommandId.fromString("topic/1/create"); + private static final CommandStatus COMMAND_STATUS = + new CommandStatus(CommandStatus.Status.SUCCESS, "some success message"); + private static final long COMMAND_SEQUENCE_NUMBER = 2L; + private static final CommandStatusEntity ENTITY = + new CommandStatusEntity(STATEMENT_TEXT, COMMAND_ID, COMMAND_STATUS, COMMAND_SEQUENCE_NUMBER); + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); @Test public void shouldSerializeToJson() throws Exception { + // When: final String json = OBJECT_MAPPER.writeValueAsString(ENTITY); + + // Then: assertThat(json, is(JSON_ENTITY)); } @Test public void shouldDeserializeFromJson() throws Exception { + // When: final CommandStatusEntity entity = OBJECT_MAPPER.readValue(JSON_ENTITY, CommandStatusEntity.class); + + // Then: assertThat(entity, is(ENTITY)); } + + @Test + public void shouldHandleNullSequenceNumber() { + // When: + final CommandStatusEntity entity = + new CommandStatusEntity(STATEMENT_TEXT, COMMAND_ID, COMMAND_STATUS, null); + + // Then: + assertThat(entity.getCommandSequenceNumber(), is(-1L)); + } + + @Test + public void shouldThrowOnNullCommandId() { + // Given: + expectedException.expect(NullPointerException.class); + expectedException.expectMessage("commandId"); + + // When: + new CommandStatusEntity(STATEMENT_TEXT, null, COMMAND_STATUS, COMMAND_SEQUENCE_NUMBER); + } + + @Test + public void shouldThrowOnNullCommandStatus() { + // Given: + expectedException.expect(NullPointerException.class); + expectedException.expectMessage("commandStatus"); + + // When: + new CommandStatusEntity(STATEMENT_TEXT, COMMAND_ID, null, COMMAND_SEQUENCE_NUMBER); + } } \ No newline at end of file diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index 2fc2b8ed65e6..39a3831de776 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -27,6 +27,7 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -47,6 +48,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -65,6 +67,8 @@ import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.InOrder; import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; @@ -74,11 +78,14 @@ public class CommandStoreTest { private static final String COMMAND_TOPIC = "command"; + private static final TopicPartition COMMAND_TOPIC_PARTITION = + new TopicPartition(COMMAND_TOPIC, 0); private static final KsqlConfig KSQL_CONFIG = new KsqlConfig( Collections.singletonMap(KsqlConfig.KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG, "foo")); private static final Map OVERRIDE_PROPERTIES = Collections.singletonMap(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - private static final long TIMEOUT = 1000; + private static final Duration TIMEOUT = Duration.ofMillis(1000); + private static final AtomicInteger COUNTER = new AtomicInteger(); private static final String statementText = "test-statement"; @Rule @@ -101,18 +108,23 @@ public class CommandStoreTest { @Mock private CommandIdAssigner commandIdAssigner; + @Captor + private ArgumentCaptor> recordCaptor; + private final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); private final Command command = new Command(statementText, Collections.emptyMap(), Collections.emptyMap()); private final RecordMetadata recordMetadata = new RecordMetadata( - new TopicPartition("topic", 0), 0, 0, RecordBatch.NO_TIMESTAMP, 0L, 0, 0); + COMMAND_TOPIC_PARTITION, 0, 0, RecordBatch.NO_TIMESTAMP, 0L, 0, 0); private CommandStore commandStore; @Before public void setUp() throws Exception { - when(commandIdAssigner.getCommandId(any())).thenReturn(commandId); + when(commandIdAssigner.getCommandId(any())) + .thenAnswer(invocation -> new CommandId( + CommandId.Type.STREAM, "foo" + COUNTER.getAndIncrement(), CommandId.Action.CREATE)); when(commandProducer.send(any(ProducerRecord.class))).thenReturn(recordMetadataFuture); when(recordMetadataFuture.get()).thenReturn(recordMetadata); @@ -169,10 +181,12 @@ public void shouldHaveAllCreateCommandsInOrder() { @Test public void shouldFailEnqueueIfCommandWithSameIdRegistered() { // Given: - expectedException.expect(IllegalStateException.class); + when(commandIdAssigner.getCommandId(any())).thenReturn(commandId); commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); + expectedException.expect(IllegalStateException.class); + // When: commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); } @@ -189,29 +203,31 @@ public void shouldCleanupCommandStatusOnProduceError() { } catch (final KsqlException e) { } - // When: + // Should: commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); - - // Then: condition being verified is that the above call doesn't throw } @Test public void shouldEnqueueNewAfterHandlingExistingCommand() { // Given: + when(commandIdAssigner.getCommandId(any())).thenReturn(commandId); commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); commandStore.getNewCommands(); - // When: + // Should: commandStore.enqueueCommand(statementText, statement, KSQL_CONFIG, OVERRIDE_PROPERTIES); - - // Then: condition being verified is that the above call doesn't throw } @Test public void shouldRegisterBeforeDistributeAndReturnStatusOnGetNewCommands() { // Given: + when(commandIdAssigner.getCommandId(any())).thenReturn(commandId); when(commandProducer.send(any(ProducerRecord.class))).thenAnswer( invocation -> { + // Set up consumer to return record with matching key, i.e. force a race condition: + final ProducerRecord record = invocation.getArgument(0); + when(commandConsumer.poll(any())).thenReturn(buildRecords(record.key(), record.value())); + final QueuedCommand queuedCommand = commandStore.getNewCommands().get(0); assertThat(queuedCommand.getCommandId(), equalTo(commandId)); assertThat(queuedCommand.getStatus().isPresent(), equalTo(true)); @@ -250,12 +266,9 @@ public void shouldFilterNullCommands() { @Test public void shouldFilterNullPriorCommand() { // Given: - final CommandId id = new CommandId(CommandId.Type.TABLE, "one", CommandId.Action.CREATE); - final Command command = new Command( - "some statement", Collections.emptyMap(), Collections.emptyMap()); final ConsumerRecords records = buildRecords( - id, null, - id, command); + commandId, null, + commandId, command); when(commandConsumer.poll(any())) .thenReturn(records) .thenReturn(ConsumerRecords.empty()); @@ -265,15 +278,13 @@ public void shouldFilterNullPriorCommand() { // Then: assertThat(commands, hasSize(1)); - assertThat(commands.get(0).getCommandId(), equalTo(id)); + assertThat(commands.get(0).getCommandId(), equalTo(commandId)); assertThat(commands.get(0).getCommand(), equalTo(command)); } @Test public void shouldDistributeCommand() { - ArgumentCaptor> recordCaptor = - ArgumentCaptor.forClass(ProducerRecord.class); - + when(commandIdAssigner.getCommandId(any())).thenReturn(commandId); when(commandProducer.send(recordCaptor.capture())).thenReturn(recordMetadataFuture); // When: @@ -301,32 +312,39 @@ public void shouldIncludeCommandSequenceNumberInSuccessfulQueuedCommandStatus() @Test public void shouldWaitOnSequenceNumberFuture() throws Exception { // When: - commandStore.ensureConsumedUpThrough(2, TIMEOUT); + commandStore.ensureConsumedPast(2, TIMEOUT); // Then: - verify(future).get(eq(TIMEOUT), eq(TimeUnit.MILLISECONDS)); + verify(future).get(eq(TIMEOUT.toMillis()), eq(TimeUnit.MILLISECONDS)); } @Test public void shouldThrowExceptionOnTimeout() throws Exception { // Given: + when(future.get(anyLong(), any(TimeUnit.class))).thenThrow(new TimeoutException()); + expectedException.expect(TimeoutException.class); expectedException.expectMessage(String.format( - "Timeout reached while waiting for command sequence number of 2. (Timeout: %d ms)", TIMEOUT)); - - when(future.get(anyLong(), any(TimeUnit.class))).thenThrow(new TimeoutException()); + "Timeout reached while waiting for command sequence number of 2. (Timeout: %d ms)", + TIMEOUT.toMillis())); // When: - commandStore.ensureConsumedUpThrough(2, TIMEOUT); + commandStore.ensureConsumedPast(2, TIMEOUT); } @Test public void shouldCompleteFuturesWhenGettingNewCommands() { + // Given: + when(commandConsumer.position(COMMAND_TOPIC_PARTITION)).thenReturn(22L); + // When: commandStore.getNewCommands(); // Then: - verify(sequenceNumberFutureStore).completeFuturesUpThroughSequenceNumber(eq(-1L)); + final InOrder inOrder = inOrder(sequenceNumberFutureStore, commandConsumer); + inOrder.verify(sequenceNumberFutureStore) + .completeFuturesUpToAndIncludingSequenceNumber(eq(21L)); + inOrder.verify(commandConsumer).poll(any()); } private static List> getPriorCommands(final CommandStore commandStore) { @@ -344,13 +362,8 @@ private static ConsumerRecords buildRecords(final Object ... assertThat(args[i], instanceOf(CommandId.class)); assertThat(args[i + 1], anyOf(is(nullValue()), instanceOf(Command.class))); records.add( - new ConsumerRecord<>("topic", 0, 0, (CommandId) args[i], (Command) args[i + 1])); + new ConsumerRecord<>(COMMAND_TOPIC, 0, 0, (CommandId) args[i], (Command) args[i + 1])); } - return new ConsumerRecords<>( - Collections.singletonMap( - new TopicPartition("topic", 0), - records - ) - ); + return new ConsumerRecords<>(Collections.singletonMap(COMMAND_TOPIC_PARTITION, records)); } } diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index ae88619291a5..56d267585788 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -45,6 +45,7 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.PersistentQueryMetadata; import io.confluent.ksql.util.timestamp.TimestampExtractionPolicy; +import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -54,7 +55,6 @@ import java.util.Map.Entry; import java.util.Optional; import java.util.Set; -import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import javax.ws.rs.core.Response; import org.apache.kafka.connect.data.Schema; @@ -133,7 +133,7 @@ public List getRestoreCommands() { } @Override - public void ensureConsumedUpThrough(final long seqNum, final long timeout) { + public void ensureConsumedPast(final long seqNum, final Duration timeout) { } @Override diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java index c46ffcc140b2..14ae7178dbb1 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java @@ -63,16 +63,13 @@ public void shouldReturnFutureForExistingSequenceNumber() { @Test public void shouldReturnFutureForCompletedSequenceNumber() { // Given: - final CompletableFuture firstFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); - sequenceNumberFutureStore.completeFuturesUpThroughSequenceNumber(2); - assertFutureIsCompleted(firstFuture); + sequenceNumberFutureStore.completeFuturesUpToAndIncludingSequenceNumber(2); // When: - final CompletableFuture secondFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); + final CompletableFuture future = sequenceNumberFutureStore.getFutureForSequenceNumber(2); // Then: - assertFutureIsCompleted(secondFuture); - assertThat(secondFuture, is(not(sameInstance(firstFuture)))); + assertFutureIsCompleted(future); } @Test @@ -82,7 +79,7 @@ public void shouldCompleteFutures() { final CompletableFuture secondFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(3); // When: - sequenceNumberFutureStore.completeFuturesUpThroughSequenceNumber(2); + sequenceNumberFutureStore.completeFuturesUpToAndIncludingSequenceNumber(2); // Then: assertFutureIsCompleted(firstFuture); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index d4c3db3dd159..b11a01694af8 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -106,6 +106,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import javax.ws.rs.core.Response; import org.apache.avro.Schema.Type; @@ -198,7 +199,7 @@ public void shouldInstantRegisterTopic() { // Then: assertThat(result, is(new CommandStatusEntity( "REGISTER TOPIC FOO WITH (kafka_topic='bar', value_format='json');", - commandStatus.getCommandId(), commandStatus.getStatus(), 0))); + commandStatus.getCommandId(), commandStatus.getStatus(), 0L))); } @Test @@ -485,7 +486,7 @@ public void shouldReturnStatusEntityFromPersistentQuery() { // Then: assertThat(result, is(new CommandStatusEntity( "CREATE STREAM S AS SELECT * FROM test_stream;", - commandStatus.getCommandId(), commandStatus.getStatus(), 0))); + commandStatus.getCommandId(), commandStatus.getStatus(), 0L))); } @Test @@ -979,7 +980,7 @@ public void shouldNotWaitIfNoCommandSequenceNumberSpecified() throws Exception { makeSingleRequestWithSequenceNumber("list properties;", null, PropertiesList.class); // Then: - verify(commandStore, never()).ensureConsumedUpThrough(anyLong(), anyLong()); + verify(commandStore, never()).ensureConsumedPast(anyLong(), any()); } @Test @@ -988,14 +989,15 @@ public void shouldWaitIfCommandSequenceNumberSpecified() throws Exception { makeSingleRequestWithSequenceNumber("list properties;", 2L, PropertiesList.class); // Then: - verify(commandStore).ensureConsumedUpThrough(eq(2L), anyLong()); + verify(commandStore).ensureConsumedPast(eq(2L), any()); } @Test - public void shouldReturn503IfTimeoutWhileWaitingForCommandSequenceNumber() throws Exception { + public void shouldReturnServiceUnavailableIfTimeoutWaitingForCommandSequenceNumber() + throws Exception { // Given: - doThrow(new KsqlRestException(Errors.commandQueueCatchUpTimeout("timed out!"))) - .when(commandStore).ensureConsumedUpThrough(anyLong(), anyLong()); + doThrow(new TimeoutException("timed out!")) + .when(commandStore).ensureConsumedPast(anyLong(), any()); // When: final KsqlErrorMessage result = diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java index 1503f5d581a4..0d7bb6377a2a 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/StreamedQueryResourceTest.java @@ -19,6 +19,7 @@ import static org.easymock.EasyMock.anyLong; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.anyString; +import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.mock; @@ -68,7 +69,6 @@ import java.util.concurrent.atomic.AtomicReference; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; -import junit.framework.AssertionFailedError; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KafkaStreams.State; @@ -156,7 +156,7 @@ public void shouldNotWaitIfCommandSequenceNumberSpecified() throws Exception { @Test public void shouldWaitIfCommandSequenceNumberSpecified() throws Exception { // Given: - replayableCommandQueue.ensureConsumedUpThrough(anyLong(), anyLong()); + replayableCommandQueue.ensureConsumedPast(eq(3L), anyObject()); expectLastCall(); replay(replayableCommandQueue); @@ -169,9 +169,10 @@ public void shouldWaitIfCommandSequenceNumberSpecified() throws Exception { } @Test - public void shouldReturn503IfTimeoutWhileWaitingForCommandSequenceNumber() throws Exception { + public void shouldReturnServiceUnavailableIfTimeoutWaitingForCommandSequenceNumber() + throws Exception { // Given: - replayableCommandQueue.ensureConsumedUpThrough(anyLong(), anyLong()); + replayableCommandQueue.ensureConsumedPast(anyLong(), anyObject()); expectLastCall().andThrow(new TimeoutException("whoops")); replay(replayableCommandQueue); 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 fc46b97f5917..75b230138788 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 @@ -51,6 +51,7 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.version.metrics.ActivenessRegistrar; import java.io.IOException; +import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -89,7 +90,7 @@ public class WSQueryEndpointTest { private static final String VALID_VERSION = Versions.KSQL_V1_WS; private static final String[] NO_VERSION_PROPERTY = null; private static final KsqlRequest[] NO_REQUEST_PROPERTY = (KsqlRequest[]) null; - private static final long COMMAND_QUEUE_CATCHUP_TIMEOUT = 5000L; + private static final Duration COMMAND_QUEUE_CATCHUP_TIMEOUT = Duration.ofMillis(5000L); @Mock private KsqlConfig ksqlConfig; @@ -345,7 +346,7 @@ public void shouldNotWaitIfNoSequenceNumberSpecified() throws Exception { wsQueryEndpoint.onOpen(session, null); // Then: - verify(replayableCommandQueue, never()).ensureConsumedUpThrough(anyLong(), anyLong()); + verify(replayableCommandQueue, never()).ensureConsumedPast(anyLong(), any()); } @Test @@ -357,7 +358,7 @@ public void shouldWaitIfSequenceNumberSpecified() throws Exception { wsQueryEndpoint.onOpen(session, null); // Then: - verify(replayableCommandQueue).ensureConsumedUpThrough(eq(SEQUENCE_NUMBER), anyLong()); + verify(replayableCommandQueue).ensureConsumedPast(eq(SEQUENCE_NUMBER), any()); } @Test @@ -365,13 +366,14 @@ public void shouldReturnErrorIfCommandQueueCatchupTimeout() throws Exception { // Given: givenRequest(REQUEST_WITH_SEQUENCE_NUMBER); doThrow(new TimeoutException("yikes")) - .when(replayableCommandQueue).ensureConsumedUpThrough(eq(SEQUENCE_NUMBER), anyLong()); + .when(replayableCommandQueue).ensureConsumedPast(eq(SEQUENCE_NUMBER), any()); // When: wsQueryEndpoint.onOpen(session, null); // Then: verifyClosedWithReason("yikes", CloseCodes.TRY_AGAIN_LATER); + verify(statementParser, never()).parseSingleStatement(any()); } private PrintTopic printTopic(final String name, final boolean fromBeginning) { diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java index aa821a83d8fc..45f8178fca79 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/util/CommandStoreUtilTest.java @@ -5,6 +5,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.never; @@ -15,6 +16,7 @@ import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.server.computation.ReplayableCommandQueue; import io.confluent.ksql.rest.server.resources.KsqlRestException; +import java.time.Duration; import java.util.Optional; import java.util.concurrent.TimeoutException; import javax.ws.rs.core.Response; @@ -27,7 +29,7 @@ @RunWith(MockitoJUnitRunner.class) public class CommandStoreUtilTest { - private static final long TIMEOUT = 5000L; + private static final Duration TIMEOUT = Duration.ofMillis(5000L); private static final long SEQUENCE_NUMBER = 2; @Mock @@ -44,7 +46,7 @@ public void shouldNotWaitIfNoSequenceNumberSpecified() throws Exception { CommandStoreUtil.waitForCommandSequenceNumber(replayableCommandQueue, request, TIMEOUT); // Then: - verify(replayableCommandQueue, never()).ensureConsumedUpThrough(anyLong(), anyLong()); + verify(replayableCommandQueue, never()).ensureConsumedPast(anyLong(), any()); } @Test @@ -56,7 +58,7 @@ public void shouldWaitIfSequenceNumberSpecified() throws Exception { CommandStoreUtil.waitForCommandSequenceNumber(replayableCommandQueue, request, TIMEOUT); // Then: - verify(replayableCommandQueue).ensureConsumedUpThrough(SEQUENCE_NUMBER, TIMEOUT); + verify(replayableCommandQueue).ensureConsumedPast(SEQUENCE_NUMBER, TIMEOUT); } @Test @@ -64,7 +66,7 @@ public void shouldThrowKsqlRestExceptionOnTimeout() throws Exception { // Given: when(request.getCommandSequenceNumber()).thenReturn(Optional.of(SEQUENCE_NUMBER)); doThrow(new TimeoutException("uh oh")) - .when(replayableCommandQueue).ensureConsumedUpThrough(SEQUENCE_NUMBER, TIMEOUT); + .when(replayableCommandQueue).ensureConsumedPast(SEQUENCE_NUMBER, TIMEOUT); try { // When: From 8e15a757363f851d9061ea1a3fa0e2a36804aba2 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Wed, 5 Dec 2018 15:45:08 -0800 Subject: [PATCH 11/13] minor requested change that slipped through the cracks. --- .../java/io/confluent/ksql/rest/entity/KsqlRequest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java index e9f5624d93bd..64411e7f7ab5 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/KsqlRequest.java @@ -74,14 +74,14 @@ public boolean equals(final Object o) { } final KsqlRequest that = (KsqlRequest) o; - return Objects.equals(getKsql(), that.getKsql()) - && Objects.equals(getStreamsProperties(), that.getStreamsProperties()) - && Objects.equals(getCommandSequenceNumber(), that.getCommandSequenceNumber()); + return Objects.equals(ksql, that.ksql) + && Objects.equals(streamsProperties, that.streamsProperties) + && Objects.equals(commandSequenceNumber, that.commandSequenceNumber); } @Override public int hashCode() { - return Objects.hash(getKsql(), getStreamsProperties(), getCommandSequenceNumber()); + return Objects.hash(ksql, streamsProperties, commandSequenceNumber); } private static Map coerceTypes(final Map streamsProperties) { From c0f9886b16e9fd8ee546b873dfb9f05c82d63c64 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Thu, 13 Dec 2018 11:10:51 -0800 Subject: [PATCH 12/13] final round of andy's requested changes. --- .../computation/CommandStatusFuture.java | 3 ++ .../rest/entity/CommandStatusEntityTest.java | 20 ++++++++++ .../SequenceNumberFutureStoreTest.java | 40 ++++++++++++++----- 3 files changed, 53 insertions(+), 10 deletions(-) diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java index 727c95f225b4..d435dc48f02d 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStatusFuture.java @@ -50,6 +50,9 @@ CommandStatus tryWaitForFinalStatus(final Duration timeout) throws InterruptedEx try { return finalStatusFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS); } catch (final ExecutionException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException)e.getCause(); + } throw new RuntimeException("Error executing command " + commandId, e.getCause()); } catch (final TimeoutException e) { return currentStatus; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java index 245ebe6890e3..9e595a23dd8c 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/CommandStatusEntityTest.java @@ -38,6 +38,14 @@ public class CommandStatusEntityTest { + "}," + "\"commandSequenceNumber\":2" + "}"; + private static final String OLD_JSON_ENTITY = "{" + + "\"@type\":\"currentStatus\"," + + "\"statementText\":\"sql\"," + + "\"commandId\":\"topic/1/create\"," + + "\"commandStatus\":{" + + "\"status\":\"SUCCESS\"," + + "\"message\":\"some success message\"" + + "}}"; private static final String STATEMENT_TEXT = "sql"; private static final CommandId COMMAND_ID = CommandId.fromString("topic/1/create"); @@ -46,6 +54,8 @@ public class CommandStatusEntityTest { private static final long COMMAND_SEQUENCE_NUMBER = 2L; private static final CommandStatusEntity ENTITY = new CommandStatusEntity(STATEMENT_TEXT, COMMAND_ID, COMMAND_STATUS, COMMAND_SEQUENCE_NUMBER); + private static final CommandStatusEntity ENTITY_WITHOUT_SEQUENCE_NUMBER = + new CommandStatusEntity(STATEMENT_TEXT, COMMAND_ID, COMMAND_STATUS, null); @Rule public final ExpectedException expectedException = ExpectedException.none(); @@ -69,6 +79,16 @@ public void shouldDeserializeFromJson() throws Exception { assertThat(entity, is(ENTITY)); } + @Test + public void shouldBeAbleToDeserializeOlderServerMessage() throws Exception { + // When: + final CommandStatusEntity entity = + OBJECT_MAPPER.readValue(OLD_JSON_ENTITY, CommandStatusEntity.class); + + // Then: + assertThat(entity, is(ENTITY_WITHOUT_SEQUENCE_NUMBER)); + } + @Test public void shouldHandleNullSequenceNumber() { // When: diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java index 14ae7178dbb1..f4378a7d741a 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/SequenceNumberFutureStoreTest.java @@ -21,7 +21,10 @@ import static org.hamcrest.CoreMatchers.sameInstance; import static org.hamcrest.MatcherAssert.assertThat; +import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -32,17 +35,17 @@ public class SequenceNumberFutureStoreTest { @Rule public final ExpectedException expectedException = ExpectedException.none(); - private SequenceNumberFutureStore sequenceNumberFutureStore; + private SequenceNumberFutureStore futureStore; @Before public void setUp() { - sequenceNumberFutureStore = new SequenceNumberFutureStore(); + futureStore = new SequenceNumberFutureStore(); } @Test public void shouldReturnFutureForNewSequenceNumber() { // When: - final CompletableFuture future = sequenceNumberFutureStore.getFutureForSequenceNumber(2); + final CompletableFuture future = futureStore.getFutureForSequenceNumber(2); // Then: assertFutureIsNotCompleted(future); @@ -51,10 +54,10 @@ public void shouldReturnFutureForNewSequenceNumber() { @Test public void shouldReturnFutureForExistingSequenceNumber() { // Given: - final CompletableFuture existingFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); + final CompletableFuture existingFuture = futureStore.getFutureForSequenceNumber(2); // When: - final CompletableFuture newFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); + final CompletableFuture newFuture = futureStore.getFutureForSequenceNumber(2); // Then: assertThat(newFuture, is(sameInstance(existingFuture))); @@ -63,10 +66,10 @@ public void shouldReturnFutureForExistingSequenceNumber() { @Test public void shouldReturnFutureForCompletedSequenceNumber() { // Given: - sequenceNumberFutureStore.completeFuturesUpToAndIncludingSequenceNumber(2); + futureStore.completeFuturesUpToAndIncludingSequenceNumber(2); // When: - final CompletableFuture future = sequenceNumberFutureStore.getFutureForSequenceNumber(2); + final CompletableFuture future = futureStore.getFutureForSequenceNumber(2); // Then: assertFutureIsCompleted(future); @@ -75,17 +78,34 @@ public void shouldReturnFutureForCompletedSequenceNumber() { @Test public void shouldCompleteFutures() { // Given: - final CompletableFuture firstFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(2); - final CompletableFuture secondFuture = sequenceNumberFutureStore.getFutureForSequenceNumber(3); + final CompletableFuture firstFuture = futureStore.getFutureForSequenceNumber(2); + final CompletableFuture secondFuture = futureStore.getFutureForSequenceNumber(3); // When: - sequenceNumberFutureStore.completeFuturesUpToAndIncludingSequenceNumber(2); + futureStore.completeFuturesUpToAndIncludingSequenceNumber(2); // Then: assertFutureIsCompleted(firstFuture); assertFutureIsNotCompleted(secondFuture); } + @Test + public void shouldBeThreadSafe() { + // When: + final List> futures = IntStream.range(1, 11).parallel() + .mapToObj(idx -> { + final CompletableFuture f = futureStore.getFutureForSequenceNumber(idx); + if (idx % 10 == 0) { + futureStore.completeFuturesUpToAndIncludingSequenceNumber(idx); + } + return f; + }) + .collect(Collectors.toList()); + + // Then: + assertThat(futures.stream().allMatch(CompletableFuture::isDone), is(true)); + } + private static void assertFutureIsCompleted(CompletableFuture future) { assertThat(future.isDone(), is(true)); assertThat(future.isCancelled(), is(false)); From 807cbd55c04d8a4577132df954ed01d7dad3e369 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Thu, 13 Dec 2018 11:15:49 -0800 Subject: [PATCH 13/13] fix checkstyle issue. --- .../ksql/rest/server/computation/StatementExecutor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/StatementExecutor.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/StatementExecutor.java index f81819038ad1..353a2de4be14 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/StatementExecutor.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/StatementExecutor.java @@ -247,7 +247,8 @@ private void handleRunScript(final Command command, final Mode mode) { if (mode == Mode.EXECUTE) { for (final QueryMetadata queryMetadata : queryMetadataList) { if (queryMetadata instanceof PersistentQueryMetadata) { - final PersistentQueryMetadata persistentQueryMd = (PersistentQueryMetadata) queryMetadata; + final PersistentQueryMetadata persistentQueryMd = + (PersistentQueryMetadata) queryMetadata; persistentQueryMd.start(); } }