Skip to content

Commit

Permalink
fix: pull queries available on /query rest & ws endpoint
Browse files Browse the repository at this point in the history
fixes: confluentinc#3672 by providing alternative way of issuing pull queries that does NOT log

Makes pull queries available on the `/query` RESTful and Websocket endpoints, in the same way that push queries are.

Note: this change does not _remove_ pull query support from the `/ksql` endpoint, nor does it switch the CLI over to use
the `/query` endpoint. The CLI continues to use the `/ksql` endpoint for pull queries.

Push and pull queries to the `/query` rest endpoint now return the schema of the rows in the first message.
This is required as the 'DESCRIBE' that CLI was previously running to get column headers doesn't work for pull queries yet. (Known issue: confluentinc#3495).
This is similar to the pattern used by the websocket endpoint, which also sends the schema in the first message.

In addition, I've hidden null fields and added a 'header' row to return the schema of the data. The output now looks like:

```json
[{"header":{"queryId":"someId","schema":"`USERID` STRING, `PAGEID` STRING, `VIEWTIME` BIGINT, `ROWKEY` STRING"}},
{"row":{"columns":["USER_1","PAGE_1",1,"1"]}},
{"row":{"columns":["USER_2","PAGE_2",2,"2"]}},
{"finalMessage":"Limit Reached"}]"
```

BREAKING CHANGE: the response from the RESTful API for push queries has changed: it now returns a line with the schema and query id in a `header` field and null fields are not included in the payload.

The CLI is backwards compatible with older versions of the server, though it won't output column headings from older versions.
  • Loading branch information
big-andy-coates committed Nov 12, 2019
1 parent 9f2ba2b commit 8e7cba9
Show file tree
Hide file tree
Showing 19 changed files with 1,106 additions and 281 deletions.
30 changes: 9 additions & 21 deletions ksql-cli/src/main/java/io/confluent/ksql/cli/Cli.java
Original file line number Diff line number Diff line change
Expand Up @@ -37,10 +37,8 @@
import io.confluent.ksql.rest.client.RestResponse;
import io.confluent.ksql.rest.entity.CommandStatus;
import io.confluent.ksql.rest.entity.CommandStatusEntity;
import io.confluent.ksql.rest.entity.FieldInfo;
import io.confluent.ksql.rest.entity.KsqlEntity;
import io.confluent.ksql.rest.entity.KsqlEntityList;
import io.confluent.ksql.rest.entity.QueryDescriptionEntity;
import io.confluent.ksql.rest.entity.StreamedRow;
import io.confluent.ksql.util.ErrorMessageUtil;
import io.confluent.ksql.util.HandlerMaps;
Expand Down Expand Up @@ -316,18 +314,6 @@ private void handleStreamedQuery(
final String query,
final SqlBaseParser.QueryStatementContext ignored
) {
final RestResponse<KsqlEntityList> explainResponse = restClient
.makeKsqlRequest("EXPLAIN " + query);
if (!explainResponse.isSuccessful()) {
terminal.printErrorMessage(explainResponse.getErrorMessage());
return;
}

final QueryDescriptionEntity description =
(QueryDescriptionEntity) explainResponse.getResponse().get(0);
final List<FieldInfo> fields = description.getQueryDescription().getFields();
terminal.printRowHeader(fields);

final RestResponse<QueryStream> queryResponse =
makeKsqlRequest(query, restClient::makeQueryRequest);

Expand All @@ -338,22 +324,24 @@ private void handleStreamedQuery(
} else {
try (QueryStream queryStream = queryResponse.getResponse();
StatusClosable toClose = terminal.setStatusMessage("Press CTRL-C to interrupt")) {
streamResults(queryStream, fields);
streamResults(queryStream);
}
}
}

private void streamResults(
final QueryStream queryStream,
final List<FieldInfo> fields
) {
private void streamResults(final QueryStream queryStream) {
final Future<?> queryStreamFuture = queryStreamExecutorService.submit(() -> {
for (long rowsRead = 0; limitNotReached(rowsRead) && queryStream.hasNext(); rowsRead++) {
for (long rowsRead = 0; limitNotReached(rowsRead) && queryStream.hasNext(); ) {
final StreamedRow row = queryStream.next();
terminal.printStreamedRow(row, fields);

terminal.printStreamedRow(row);
if (row.isTerminal()) {
break;
}

if (row.getRow().isPresent()) {
rowsRead++;
}
}
});

Expand Down
55 changes: 24 additions & 31 deletions ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@
import io.confluent.ksql.rest.entity.TablesList;
import io.confluent.ksql.rest.entity.TopicDescription;
import io.confluent.ksql.rest.entity.TypeList;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.util.CmdLineUtil;
import io.confluent.ksql.util.HandlerMaps;
import io.confluent.ksql.util.HandlerMaps.ClassHandlerMap1;
Expand Down Expand Up @@ -324,32 +325,27 @@ public void printError(final String shortMsg, final String fullMsg) {
writer().println(shortMsg);
}

public void printStreamedRow(
final StreamedRow row,
final List<FieldInfo> fields
) {
if (row.getErrorMessage() != null) {
printErrorMessage(row.getErrorMessage());
return;
}
public void printStreamedRow(final StreamedRow row) {
row.getErrorMessage().ifPresent(this::printErrorMessage);

if (row.getFinalMessage() != null) {
writer().println(row.getFinalMessage());
return;
}
row.getFinalMessage().ifPresent(finalMsg -> writer().println(finalMsg));

switch (outputFormat) {
case JSON:
printAsJson(row.getRow().getColumns());
break;
case TABULAR:
printAsTable(row.getRow(), fields);
break;
default:
throw new RuntimeException(String.format(
"Unexpected output format: '%s'",
outputFormat.name()
));
row.getHeader().ifPresent(header -> printRowHeader(header.getSchema()));

if (row.getRow().isPresent()) {
switch (outputFormat) {
case JSON:
printAsJson(row.getRow().get().getColumns());
break;
case TABULAR:
printAsTable(row.getRow().get());
break;
default:
throw new RuntimeException(String.format(
"Unexpected output format: '%s'",
outputFormat.name()
));
}
}
}

Expand All @@ -376,12 +372,12 @@ public void printKsqlEntityList(final List<KsqlEntity> entityList) {
}
}

public void printRowHeader(final List<FieldInfo> fields) {
private void printRowHeader(final LogicalSchema schema) {
switch (outputFormat) {
case JSON:
break;
case TABULAR:
writer().println(TabularRow.createHeader(getWidth(), fields));
writer().println(TabularRow.createHeader(getWidth(), schema));
break;
default:
throw new RuntimeException(String.format(
Expand Down Expand Up @@ -426,12 +422,9 @@ private Optional<CliCmdExecutor> getCliCommand(final String line) {
.findFirst();
}

private void printAsTable(
final GenericRow row,
final List<FieldInfo> fields
) {
private void printAsTable(final GenericRow row) {
rowCaptor.addRow(row);
writer().println(TabularRow.createRow(getWidth(), fields, row, config));
writer().println(TabularRow.createRow(getWidth(), row, config));
flush();
}

Expand Down
40 changes: 21 additions & 19 deletions ksql-cli/src/main/java/io/confluent/ksql/util/TabularRow.java
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,15 @@

package io.confluent.ksql.util;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Splitter;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
import io.confluent.ksql.GenericRow;
import io.confluent.ksql.cli.console.CliConfig;
import io.confluent.ksql.cli.console.CliConfig.OnOff;
import io.confluent.ksql.rest.entity.FieldInfo;
import io.confluent.ksql.name.ColumnName;
import io.confluent.ksql.schema.ksql.Column;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
Expand All @@ -33,51 +35,51 @@ public class TabularRow {
private static final int MIN_CELL_WIDTH = 5;

private final int width;
private final List<String> value;
private final List<String> header;
private final List<String> columns;
private final boolean isHeader;
private final boolean shouldWrap;

public static TabularRow createHeader(final int width, final List<FieldInfo> header) {
public static TabularRow createHeader(final int width, final LogicalSchema schema) {
final List<String> headings = schema.columns().stream()
.map(Column::name)
.map(ColumnName::name)
.collect(Collectors.toList());

return new TabularRow(
width,
header.stream().map(FieldInfo::getName).collect(Collectors.toList()),
null,
true);
headings,
true,
true
);
}

public static TabularRow createRow(
final int width,
final List<FieldInfo> header,
final GenericRow value,
final CliConfig config
) {
return new TabularRow(
width,
header.stream().map(FieldInfo::getName).collect(Collectors.toList()),
value.getColumns().stream().map(Objects::toString).collect(Collectors.toList()),
false,
config.getString(CliConfig.WRAP_CONFIG).equalsIgnoreCase(OnOff.ON.toString())
);
}

@VisibleForTesting
TabularRow(
private TabularRow(
final int width,
final List<String> header,
final List<String> value,
final List<String> columns,
final boolean isHeader,
final boolean shouldWrap
) {
this.header = Objects.requireNonNull(header, "header");
this.columns = ImmutableList.copyOf(Objects.requireNonNull(columns, "columns"));
this.width = width;
this.value = value;
this.isHeader = value == null;
this.isHeader = isHeader;
this.shouldWrap = shouldWrap;
}

@Override
public String toString() {
final List<String> columns = isHeader ? header : value;

if (columns.isEmpty()) {
return "";
}
Expand Down
22 changes: 16 additions & 6 deletions ksql-cli/src/test/java/io/confluent/ksql/cli/CliTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.either;
import static org.hamcrest.Matchers.emptyIterable;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
Expand Down Expand Up @@ -603,22 +602,33 @@ public void testTransientSelect() {
}

@Test
public void testTransientStaticSelectStar() {
public void shouldHandlePullQuery() {
// Given:
run("CREATE TABLE X AS SELECT COUNT(1) AS COUNT "
+ "FROM " + orderDataProvider.kstreamName()
+ " GROUP BY ITEMID;",
localCli
);

// When:
final Supplier<String> runner = () -> {
// It's possible that the state store is not warm on the first invocation, hence the retry
run("SELECT * FROM X WHERE ROWKEY='ITEM_1';", localCli);
return terminal.getOutputString();
};

// Wait for warm store:
assertThatEventually(runner, containsString("ROWKEY"));
assertRunCommand(
"SELECT * FROM X WHERE ROWKEY='unknowwn';",
is(emptyIterable())
"SELECT * FROM X WHERE ROWKEY='ITEM_1';",
containsRows(
row("ITEM_1", "1")
)
);
}

@Test
public void testTransientStaticHeader() {
public void shouldOutputPullQueryHeader() {
// Given:
run("CREATE TABLE Y AS SELECT COUNT(1) AS COUNT "
+ "FROM " + orderDataProvider.kstreamName()
Expand Down Expand Up @@ -655,7 +665,7 @@ public void testTransientContinuousSelectStar() {
}

@Test
public void testTransientContinuousHeader() {
public void shouldOutputPushQueryHeader() {
// When:
run("SELECT * FROM " + orderDataProvider.kstreamName() + " EMIT CHANGES LIMIT 1", localCli);

Expand Down
Loading

0 comments on commit 8e7cba9

Please sign in to comment.