From 19c24f1e1fc3d1293c40dd285cd30c702002b1f0 Mon Sep 17 00:00:00 2001 From: Almog Gavra Date: Mon, 31 Aug 2020 12:36:44 -0700 Subject: [PATCH] chore: leverage SELECT * style formatting for testing tool (#6117) --- .../confluent/ksql/cli/console/Console.java | 17 +++- .../confluent/ksql/util/TabularRowTest.java | 41 +++++----- .../io/confluent/ksql/util/TabularRow.java | 30 +++---- .../ksql/test/driver/AssertExecutor.java | 78 +++++++++++++++---- .../ksql/test/driver/TestDriverPipeline.java | 3 +- 5 files changed, 118 insertions(+), 51 deletions(-) rename {ksqldb-cli => ksqldb-common}/src/main/java/io/confluent/ksql/util/TabularRow.java (89%) diff --git a/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java b/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java index cff317632c7b..600a4c78200e 100644 --- a/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java +++ b/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import io.confluent.ksql.GenericRow; +import io.confluent.ksql.cli.console.CliConfig.OnOff; import io.confluent.ksql.cli.console.KsqlTerminal.HistoryEntry; import io.confluent.ksql.cli.console.KsqlTerminal.StatusClosable; import io.confluent.ksql.cli.console.cmd.CliSpecificCommand; @@ -384,7 +385,14 @@ private void printRowHeader(final LogicalSchema schema) { case JSON: break; case TABULAR: - writer().println(TabularRow.createHeader(getWidth(), schema, config)); + writer().println( + TabularRow.createHeader( + getWidth(), + schema.columns(), + config.getString(CliConfig.WRAP_CONFIG).equalsIgnoreCase(OnOff.ON.toString()), + config.getInt(CliConfig.COLUMN_WIDTH_CONFIG) + ) + ); break; default: throw new RuntimeException(String.format( @@ -431,7 +439,12 @@ private Optional getCliCommand(final String line) { private void printAsTable(final GenericRow row) { rowCaptor.addRow(row); - writer().println(TabularRow.createRow(getWidth(), row, config)); + writer().println(TabularRow.createRow( + getWidth(), + row, + config.getString(CliConfig.WRAP_CONFIG).equalsIgnoreCase(OnOff.ON.toString()), + config.getInt(CliConfig.COLUMN_WIDTH_CONFIG)) + ); flush(); } diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/util/TabularRowTest.java b/ksqldb-cli/src/test/java/io/confluent/ksql/util/TabularRowTest.java index 131a2c8ffde5..2a9925f83545 100644 --- a/ksqldb-cli/src/test/java/io/confluent/ksql/util/TabularRowTest.java +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/util/TabularRowTest.java @@ -19,24 +19,27 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.isEmptyString; -import static org.mockito.Mockito.when; import io.confluent.ksql.GenericRow; -import io.confluent.ksql.cli.console.CliConfig; -import io.confluent.ksql.cli.console.CliConfig.OnOff; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.types.SqlTypes; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; @RunWith(MockitoJUnitRunner.class) public class TabularRowTest { - @Mock - private CliConfig config; + private boolean shouldWrap; + private int width; + + @Before + public void setUp() { + shouldWrap = false; + width = 0; + } @Test public void shouldFormatHeader() { @@ -47,7 +50,7 @@ public void shouldFormatHeader() { .build(); // When: - final String formatted = TabularRow.createHeader(20, schema, config).toString(); + final String formatted = TabularRow.createHeader(20, schema.columns(), shouldWrap, width).toString(); // Then: assertThat(formatted, is("" @@ -65,7 +68,7 @@ public void shouldMultilineFormatHeader() { .build(); // When: - final String formatted = TabularRow.createHeader(20, schema, config).toString(); + final String formatted = TabularRow.createHeader(20, schema.columns(), shouldWrap, width).toString(); // Then: assertThat(formatted, is("" @@ -84,7 +87,7 @@ public void shouldFormatRow() { final GenericRow value = genericRow("foo", "bar"); // When: - final String formatted = TabularRow.createRow(20, value, config).toString(); + final String formatted = TabularRow.createRow(20, value, shouldWrap, width).toString(); // Then: assertThat(formatted, is("|foo |bar |")); @@ -98,7 +101,7 @@ public void shouldMultilineFormatRow() { final GenericRow value = genericRow("foo", "bar is a long string"); // When: - final String formatted = TabularRow.createRow(20, value, config).toString(); + final String formatted = TabularRow.createRow(20, value, shouldWrap, width).toString(); // Then: assertThat(formatted, is("" @@ -115,7 +118,7 @@ public void shouldClipMultilineFormatRow() { final GenericRow value = genericRow("foo", "bar is a long string"); // When: - final String formatted = TabularRow.createRow(20, value, config).toString(); + final String formatted = TabularRow.createRow(20, value, shouldWrap, width).toString(); // Then: assertThat(formatted, is("" @@ -133,7 +136,7 @@ public void shouldClipMultilineFormatRowWithLotsOfWhitespace() { ); // When: - final String formatted = TabularRow.createRow(20, value, config).toString(); + final String formatted = TabularRow.createRow(20, value, shouldWrap, width).toString(); // Then: assertThat(formatted, is("" @@ -151,7 +154,7 @@ public void shouldNotAddEllipsesMultilineFormatRowWithLotsOfWhitespace() { ); // When: - final String formatted = TabularRow.createRow(20, value, config).toString(); + final String formatted = TabularRow.createRow(20, value, shouldWrap, width).toString(); // Then: assertThat(formatted, is("" @@ -166,7 +169,7 @@ public void shouldFormatNoColumnsHeader() { .build(); // When: - final String formatted = TabularRow.createHeader(20, schema, config).toString(); + final String formatted = TabularRow.createHeader(20, schema.columns(), shouldWrap, width).toString(); // Then: assertThat(formatted, isEmptyString()); @@ -182,7 +185,7 @@ public void shouldFormatMoreColumnsThanWidth() { .build(); // When: - final String formatted = TabularRow.createHeader(3, schema, config).toString(); + final String formatted = TabularRow.createHeader(3, schema.columns(), shouldWrap, width).toString(); // Then: assertThat(formatted, @@ -204,7 +207,7 @@ public void shouldFormatCustomColumnWidth() { .build(); // When: - final String formatted = TabularRow.createHeader(999, schema, config).toString(); + final String formatted = TabularRow.createHeader(999, schema.columns(), shouldWrap, width).toString(); // Then: assertThat(formatted, @@ -215,14 +218,14 @@ public void shouldFormatCustomColumnWidth() { } private void givenWrappingEnabled() { - when(config.getString(CliConfig.WRAP_CONFIG)).thenReturn(OnOff.ON.toString()); + shouldWrap = true; } private void givenWrappingDisabled() { - when(config.getString(CliConfig.WRAP_CONFIG)).thenReturn("Not ON"); + shouldWrap = false; } private void givenCustomColumnWidth(int width) { - when(config.getInt(CliConfig.COLUMN_WIDTH_CONFIG)).thenReturn(width); + this.width = width; } } diff --git a/ksqldb-cli/src/main/java/io/confluent/ksql/util/TabularRow.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/TabularRow.java similarity index 89% rename from ksqldb-cli/src/main/java/io/confluent/ksql/util/TabularRow.java rename to ksqldb-common/src/main/java/io/confluent/ksql/util/TabularRow.java index 925deaaebbcf..7d938a6a9fae 100644 --- a/ksqldb-cli/src/main/java/io/confluent/ksql/util/TabularRow.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/TabularRow.java @@ -19,11 +19,8 @@ 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.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; @@ -41,10 +38,11 @@ public final class TabularRow { public static TabularRow createHeader( final int width, - final LogicalSchema schema, - final CliConfig config + final List columns, + final boolean shouldWrap, + final int configuredCellWidth ) { - final List headings = schema.columns().stream() + final List headings = columns.stream() .map(Column::name) .map(ColumnName::text) .collect(Collectors.toList()); @@ -53,20 +51,23 @@ public static TabularRow createHeader( width, headings, true, - config + shouldWrap, + configuredCellWidth ); } public static TabularRow createRow( final int width, final GenericRow value, - final CliConfig config + final boolean shouldWrap, + final int configuredCellWidth ) { return new TabularRow( width, value.values().stream().map(Objects::toString).collect(Collectors.toList()), false, - config + shouldWrap, + configuredCellWidth ); } @@ -74,16 +75,15 @@ private TabularRow( final int width, final List columns, final boolean isHeader, - final CliConfig config + final boolean shouldWrap, + final int configuredCellWidth ) { this.columns = ImmutableList.copyOf(Objects.requireNonNull(columns, "columns")); this.isHeader = isHeader; - this.shouldWrap = isHeader - || config.getString(CliConfig.WRAP_CONFIG).equalsIgnoreCase(OnOff.ON.toString()); + this.shouldWrap = isHeader || shouldWrap; - final int configCellWidth = config.getInt(CliConfig.COLUMN_WIDTH_CONFIG); - if (configCellWidth > 0) { - this.cellWidth = configCellWidth; + if (configuredCellWidth > 0) { + this.cellWidth = configuredCellWidth; } else if (!columns.isEmpty()) { this.cellWidth = Math.max(width / columns.size() - 2, MIN_CELL_WIDTH); } else { diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/AssertExecutor.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/AssertExecutor.java index f5659ae4ff38..e5baec68c788 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/AssertExecutor.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/AssertExecutor.java @@ -15,21 +15,29 @@ package io.confluent.ksql.test.driver; +import com.google.common.collect.ImmutableList; import io.confluent.ksql.GenericRow; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.engine.generic.GenericRecordFactory; import io.confluent.ksql.engine.generic.KsqlGenericRecord; import io.confluent.ksql.metastore.model.DataSource; +import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.parser.AssertTable; import io.confluent.ksql.parser.tree.AssertStream; import io.confluent.ksql.parser.tree.AssertTombstone; import io.confluent.ksql.parser.tree.AssertValues; import io.confluent.ksql.parser.tree.InsertValues; +import io.confluent.ksql.schema.ksql.Column; +import io.confluent.ksql.schema.ksql.Column.Namespace; import io.confluent.ksql.schema.ksql.SystemColumns; +import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.TabularRow; import java.util.Iterator; +import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.streams.test.TestRecord; @@ -94,13 +102,14 @@ private static void assertContent( final Iterator> records = driverPipeline .getRecordsForTopic(dataSource.getKafkaTopicName()); if (!records.hasNext()) { - throw new KsqlException( - String.format( - "Expected another record (%s) for %s but already read all records: %s", - expected, - dataSource.getName(), - driverPipeline.getAllRecordsForTopic(dataSource.getKafkaTopicName()) - ) + throwAssertionError( + "Expected another record, but all records have already been read:", + dataSource, + expected, + driverPipeline.getAllRecordsForTopic(dataSource.getKafkaTopicName()) + .stream() + .map(rec -> KsqlGenericRecord.of(rec.key(), rec.value(), rec.timestamp())) + .collect(Collectors.toList()) ); } @@ -112,14 +121,55 @@ private static void assertContent( ); if (!actual.equals(expected)) { - throw new KsqlException( - String.format( - "Expected record does not match actual. Expected: %s vs. Actual: %s", - expected, - actual - ) - ); + throwAssertionError( + "Expected record does not match actual.", + dataSource, + expected, + ImmutableList.of(actual)); + } + } + + private static void throwAssertionError( + final String message, + final DataSource dataSource, + final KsqlGenericRecord expected, + final List actual + ) { + final List columns = ImmutableList.builder() + .add(Column.of(ColumnName.of("."), SqlTypes.STRING, Namespace.KEY, 0)) + .add(Column.of(SystemColumns.ROWTIME_NAME, SqlTypes.BIGINT, Namespace.KEY, 0)) + .addAll(dataSource.getSchema().columns()) + .build(); + + final TabularRow headerRow = TabularRow.createHeader(80, columns, false, 0); + + final StringBuilder actualRows = new StringBuilder(); + actual.forEach(a -> actualRows.append(fromGenericRow(false, dataSource, a)).append('\n')); + throw new KsqlException( + String.format( + "%s%n%s%n%s%n%s", + message, + headerRow, + fromGenericRow(true, dataSource, expected), + actualRows.toString() + ) + ); + } + + private static TabularRow fromGenericRow( + final boolean expected, + final DataSource source, + final KsqlGenericRecord row + ) { + final GenericRow contents = new GenericRow(); + contents.append(expected ? "EXPECTED" : "ACTUAL"); + for (final Column key : source.getSchema().key()) { + contents.append(row.key.get(key.name().text())); } + contents.append(row.ts); + contents.appendAll(row.value.values()); + + return TabularRow.createRow(80, contents, false, 0); } public static void assertStream(final AssertStream assertStatement) { diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/TestDriverPipeline.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/TestDriverPipeline.java index e1c9bb9ff1c4..71368e05d6f1 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/TestDriverPipeline.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/TestDriverPipeline.java @@ -221,7 +221,8 @@ public boolean hasNext() { @Override public TestRecord next() { - final int idx = assertPositions.merge(topic, 0, (old, zero) -> old + 1); + final int idx = assertPositions.getOrDefault(topic, 0); + assertPositions.put(topic, idx + 1); return outputCache.get(topic).get(idx); } };