Skip to content

Commit

Permalink
feat(static): unordered table elements and meta columns serialization (
Browse files Browse the repository at this point in the history
…#3428)

* feat(static): unordered table elements and meta columns serialization

A query might contain key and value columns in any order. Hence the `TableElements` class has been changed to:

 - not fail if key columns come after value columns
 - not change the order of columns

When serializing a logical schema across the wire, (which is currently only used for static queries), we should't yet serialize the ROWTIME meta column. This is mainly because static queries don't support it, and our syntax doesn't support meta columns, i.e. you can't have a `CREATE TABLE X (ROWTIME BIGINT META, ...`. So if we did include it in the serialized form, we wouldn't be able to parse it.
  • Loading branch information
big-andy-coates authored Sep 27, 2019
1 parent 7ad3248 commit 3b23fd6
Show file tree
Hide file tree
Showing 8 changed files with 165 additions and 85 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,24 @@ public static void classSetUp() {
}

@Test
public void shouldSchemaAsString() throws Exception {
public void shouldSerializeSchemaWithImplicitColumns() throws Exception {
// Given:
final LogicalSchema schema = LogicalSchema.builder()
.valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER)
.build();

// When:
final String json = MAPPER.writeValueAsString(schema);

// Then:
assertThat(json, is("\"`ROWKEY` STRING KEY, `v0` INTEGER\""));
}

@Test
public void shouldSerializeSchemaWithOutImplicitColumns() throws Exception {
// Given:
final LogicalSchema schema = LogicalSchema.builder()
.noImplicitColumns()
.keyColumn(ColumnName.of("key0"), SqlTypes.STRING)
.valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER)
.build();
Expand All @@ -50,6 +65,21 @@ public void shouldSchemaAsString() throws Exception {
assertThat(json, is("\"`key0` STRING KEY, `v0` INTEGER\""));
}

@Test
public void shouldSerializeSchemaWithKeyAfterValue() throws Exception {
// Given:
final LogicalSchema schema = LogicalSchema.builder()
.valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER)
.keyColumn(ColumnName.of("key0"), SqlTypes.STRING)
.build();

// When:
final String json = MAPPER.writeValueAsString(schema);

// Then:
assertThat(json, is("\"`v0` INTEGER, `key0` STRING KEY\""));
}

private static final class TestModule extends SimpleModule {

TestModule() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ public CommandFactories(final ServiceContext serviceContext, final MetaStore met
this(serviceContext, metaStore, SerdeOptions::buildForCreateStatement, new GenericRowSerDe());
}

public CommandFactories(
CommandFactories(
final ServiceContext serviceContext,
final MetaStore metaStore,
final SerdeOptionsSupplier serdeOptionsSupplier,
Expand Down Expand Up @@ -198,7 +198,6 @@ private CreateTableCommand handleCreateTable(
);
}

@SuppressWarnings("MethodMayBeStatic")
private DropSourceCommand handleDropStream(final DropStream statement) {
return handleDropSource(
statement.getName(),
Expand All @@ -207,7 +206,6 @@ private DropSourceCommand handleDropStream(final DropStream statement) {
);
}

@SuppressWarnings("MethodMayBeStatic")
private DropSourceCommand handleDropTable(final DropTable statement) {
return handleDropSource(
statement.getName(),
Expand All @@ -224,6 +222,7 @@ private RegisterTypeCommand handleRegisterType(final RegisterType statement) {
);
}

@SuppressWarnings("MethodMayBeStatic")
private DropTypeCommand handleDropType(final DropType statement) {
return new DropTypeCommand(statement.getTypeName());
}
Expand Down Expand Up @@ -294,7 +293,7 @@ private static LogicalSchema buildSchema(final TableElements tableElements) {
}
});

return tableElements.toLogicalSchema();
return tableElements.toLogicalSchema(true);
}

private static KsqlTopic buildTopic(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ private static Topic createTopicFromStatement(
final KeyFormat keyFormat = ksqlTopic.getKeyFormat();

final Supplier<LogicalSchema> logicalSchemaSupplier =
statement.getElements()::toLogicalSchema;
() -> statement.getElements().toLogicalSchema(true);

final SerdeSupplier<?> keySerdeSupplier =
SerdeUtil.getKeySerdeSupplier(keyFormat, logicalSchemaSupplier);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
package io.confluent.ksql.parser.tree;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.errorprone.annotations.Immutable;
import io.confluent.ksql.name.ColumnName;
Expand All @@ -25,11 +24,8 @@
import io.confluent.ksql.schema.ksql.LogicalSchema.Builder;
import io.confluent.ksql.schema.ksql.types.SqlType;
import io.confluent.ksql.util.KsqlException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.stream.Collectors;
Expand All @@ -42,11 +38,11 @@ public final class TableElements implements Iterable<TableElement> {
private final ImmutableList<TableElement> elements;

public static TableElements of(final TableElement... elements) {
return build(Arrays.stream(elements));
return new TableElements(ImmutableList.copyOf(elements));
}

public static TableElements of(final List<TableElement> elements) {
return build(elements.stream());
return new TableElements(ImmutableList.copyOf(elements));
}

@Override
Expand Down Expand Up @@ -80,12 +76,18 @@ public String toString() {
return elements.toString();
}

public LogicalSchema toLogicalSchema() {
/**
* @param withImplicitColumns controls if schema has implicit columns such as ROWTIME or ROWKEY.
* @return the logical schema.
*/
public LogicalSchema toLogicalSchema(final boolean withImplicitColumns) {
if (Iterables.isEmpty(this)) {
throw new KsqlException("No columns supplied.");
}

final Builder builder = LogicalSchema.builder();
final Builder builder = withImplicitColumns
? LogicalSchema.builder()
: LogicalSchema.builder().noImplicitColumns();

for (final TableElement tableElement : this) {
final ColumnName fieldName = tableElement.getName();
Expand All @@ -103,58 +105,12 @@ public LogicalSchema toLogicalSchema() {

private TableElements(final ImmutableList<TableElement> elements) {
this.elements = Objects.requireNonNull(elements, "elements");
}

private static TableElements build(final Stream<TableElement> elements) {
final Map<Boolean, List<TableElement>> split = splitByElementType(elements);

final List<TableElement> keyColumns = split.getOrDefault(Boolean.TRUE, ImmutableList.of());
final List<TableElement> valueColumns = split.getOrDefault(Boolean.FALSE, ImmutableList.of());

throwOnDuplicateNames(keyColumns, "KEY");
throwOnDuplicateNames(valueColumns, "non-KEY");

final ImmutableList.Builder<TableElement> builder = ImmutableList.builder();

builder.addAll(keyColumns);
builder.addAll(valueColumns);

return new TableElements(builder.build());
}

private static Map<Boolean, List<TableElement>> splitByElementType(
final Stream<TableElement> elements
) {
final List<TableElement> keyFields = new ArrayList<>();
final List<TableElement> valueFields = new ArrayList<>();

elements.forEach(element -> {
if (element.getNamespace() == Namespace.VALUE) {
valueFields.add(element);
return;
}

if (!valueFields.isEmpty()) {
throw new KsqlException("KEY column declared after VALUE column: "
+ element.getName().name()
+ System.lineSeparator()
+ "All KEY columns must be declared before any VALUE column(s).");
}

keyFields.add(element);
});

return ImmutableMap.of(
Boolean.TRUE, keyFields,
Boolean.FALSE, valueFields
);
throwOnDuplicateNames();
}

private static void throwOnDuplicateNames(
final List<TableElement> columns,
final String type
) {
final String duplicates = columns.stream()
private void throwOnDuplicateNames() {
final String duplicates = elements.stream()
.collect(Collectors.groupingBy(TableElement::getName, Collectors.counting()))
.entrySet()
.stream()
Expand All @@ -164,7 +120,7 @@ private static void throwOnDuplicateNames(
.collect(Collectors.joining(", "));

if (!duplicates.isEmpty()) {
throw new KsqlException("Duplicate " + type + " column names: " + duplicates);
throw new KsqlException("Duplicate column names: " + duplicates);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.schema.ksql.types.SqlTypes;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.SchemaUtil;
import java.util.List;
import java.util.stream.Collectors;
import org.junit.Rule;
Expand Down Expand Up @@ -58,19 +59,17 @@ public void shouldImplementHashCodeAndEqualsProperty() {
}

@Test
public void shouldThrowOnOutOfOrderKeyColumns() {
public void shouldSupportKeyColumnsAfterValues() {
// Given:
final List<TableElement> elements = ImmutableList.of(
tableElement(VALUE, "v0", INT_TYPE),
tableElement(KEY, "key", STRING_TYPE)
);

// Then:
expectedException.expect(KsqlException.class);
expectedException.expectMessage("KEY column declared after VALUE column: key");
final TableElement key = tableElement(KEY, "key", STRING_TYPE);
final TableElement value = tableElement(VALUE, "v0", INT_TYPE);
final List<TableElement> elements = ImmutableList.of(value, key);

// When:
TableElements.of(elements);
final TableElements result = TableElements.of(elements);

// Then:
assertThat(result, contains(value, key));
}

@Test
Expand All @@ -85,7 +84,7 @@ public void shouldThrowOnDuplicateKeyColumns() {

// Then:
expectedException.expect(KsqlException.class);
expectedException.expectMessage("Duplicate KEY column names:");
expectedException.expectMessage("Duplicate column names:");
expectedException.expectMessage("k0");
expectedException.expectMessage("k1");

Expand All @@ -105,7 +104,27 @@ public void shouldThrowOnDuplicateValueColumns() {

// Then:
expectedException.expect(KsqlException.class);
expectedException.expectMessage("Duplicate non-KEY column names:");
expectedException.expectMessage("Duplicate column names:");
expectedException.expectMessage("v0");
expectedException.expectMessage("v1");

// When:
TableElements.of(elements);
}

@Test
public void shouldThrowOnDuplicateKeyValueColumns() {
// Given:
final List<TableElement> elements = ImmutableList.of(
tableElement(KEY, "v0", INT_TYPE),
tableElement(VALUE, "v0", INT_TYPE),
tableElement(KEY, "v1", INT_TYPE),
tableElement(VALUE, "v1", INT_TYPE)
);

// Then:
expectedException.expect(KsqlException.class);
expectedException.expectMessage("Duplicate column names:");
expectedException.expectMessage("v0");
expectedException.expectMessage("v1");

Expand Down Expand Up @@ -182,23 +201,65 @@ public void shouldThrowWhenBuildLogicalSchemaIfNoElements() {
expectedException.expectMessage("No columns supplied.");

// When:
tableElements.toLogicalSchema();
tableElements.toLogicalSchema(true);
}

@Test
public void shouldBuildLogicalSchema() {
public void shouldBuildLogicalSchemaWithImplicits() {
// Given:
final TableElement element0 = tableElement(KEY, "k0", STRING_TYPE);
final TableElement element1 = tableElement(VALUE, "v0", INT_TYPE);
final TableElements tableElements = TableElements.of(element0, element1);
final TableElements tableElements = TableElements.of(
tableElement(VALUE, "v0", INT_TYPE)
);

// When:
final LogicalSchema schema = tableElements.toLogicalSchema(true);

// Then:
assertThat(schema, is(LogicalSchema.builder()
.keyColumn(SchemaUtil.ROWKEY_NAME, SqlTypes.STRING)
.valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER)
.build()
));
}

@Test
public void shouldBuildLogicalSchemaWithOutImplicits() {
// Given:
final TableElements tableElements = TableElements.of(
tableElement(VALUE, "v0", INT_TYPE)
);

// When:
final LogicalSchema schema = tableElements.toLogicalSchema(false);

// Then:
assertThat(schema, is(LogicalSchema.builder()
.noImplicitColumns()
.valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER)
.build()
));
}

@Test
public void shouldBuildLogicalSchemaWithKeyAndValueColumnsInterleaved() {
// Given:
final TableElements tableElements = TableElements.of(
tableElement(VALUE, "v0", INT_TYPE),
tableElement(KEY, "k0", INT_TYPE),
tableElement(VALUE, "v1", STRING_TYPE),
tableElement(KEY, "k1", INT_TYPE)
);

// When:
final LogicalSchema schema = tableElements.toLogicalSchema();
final LogicalSchema schema = tableElements.toLogicalSchema(false);

// Then:
assertThat(schema, is(LogicalSchema.builder()
.keyColumn(ColumnName.of("k0"), SqlTypes.STRING)
.noImplicitColumns()
.valueColumn(ColumnName.of("v0"), SqlTypes.INTEGER)
.keyColumn(ColumnName.of("k0"), SqlTypes.INTEGER)
.valueColumn(ColumnName.of("v1"), SqlTypes.STRING)
.keyColumn(ColumnName.of("k1"), SqlTypes.INTEGER)
.build()
));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,6 @@ public LogicalSchema deserialize(

final TableElements tableElements = SchemaParser.parse(text, TypeRegistry.EMPTY);

return tableElements.toLogicalSchema();
return tableElements.toLogicalSchema(false);
}
}
Loading

0 comments on commit 3b23fd6

Please sign in to comment.