Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix: remove any rowtime or rowkey columns from query schema (MINOR) (Fixes 3039) #3043

Merged
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -63,11 +63,13 @@ public QueryDescription(
private QueryDescription(
final String id,
final QueryMetadata queryMetadata,
final Set<String> sinks) {
final Set<String> sinks,
final boolean valueSchemaOnly
) {
this(
new EntityQueryId(id),
queryMetadata.getStatementString(),
EntityUtil.buildSourceSchemaEntity(queryMetadata.getLogicalSchema()),
EntityUtil.buildSourceSchemaEntity(queryMetadata.getLogicalSchema(), valueSchemaOnly),
queryMetadata.getSourceNames(),
sinks,
queryMetadata.getTopologyDescription(),
Expand All @@ -77,11 +79,15 @@ private QueryDescription(

public static QueryDescription forQueryMetadata(final QueryMetadata queryMetadata) {
if (queryMetadata instanceof PersistentQueryMetadata) {
final PersistentQueryMetadata persistentQuery = (PersistentQueryMetadata) queryMetadata;
return new QueryDescription(
((PersistentQueryMetadata) queryMetadata).getQueryId().getId(), queryMetadata,
((PersistentQueryMetadata) queryMetadata).getSinkNames());
persistentQuery.getQueryId().getId(),
persistentQuery,
persistentQuery.getSinkNames(),
false
);
}
return new QueryDescription("", queryMetadata, Collections.emptySet());
return new QueryDescription("", queryMetadata, Collections.emptySet(), true);
}

public EntityQueryId getId() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ public SourceDescription(
dataSource.getName(),
readQueries,
writeQueries,
EntityUtil.buildSourceSchemaEntity(dataSource.getSchema()),
EntityUtil.buildSourceSchemaEntity(dataSource.getSchema(), false),
dataSource.getDataSourceType().getKsqlType(),
dataSource.getKeyField().name().orElse(""),
Optional.ofNullable(dataSource.getTimestampExtractionPolicy())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ public void onComplete() {
public void onSchema(final LogicalSchema schema) {
try {
session.getBasicRemote().sendText(
mapper.writeValueAsString(EntityUtil.buildSourceSchemaEntity(schema))
mapper.writeValueAsString(EntityUtil.buildSourceSchemaEntity(schema, true))
);
} catch (final IOException e) {
log.error("Error sending schema", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,16 @@ public final class EntityUtil {
private EntityUtil() {
}

public static List<FieldInfo> buildSourceSchemaEntity(final LogicalSchema schema) {
public static List<FieldInfo> buildSourceSchemaEntity(
final LogicalSchema schema,
final boolean valueSchemaOnly
) {

final List<FieldInfo> allFields = new ArrayList<>();
allFields.addAll(getFields(schema.metaFields(), "implicit"));
allFields.addAll(getFields(schema.keyFields(), "key"));
if (!valueSchemaOnly) {
allFields.addAll(getFields(schema.metaFields(), "implicit"));
allFields.addAll(getFields(schema.keyFields(), "key"));
}
allFields.addAll(getFields(schema.valueFields(), "value"));

return allFields;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,7 @@
import io.confluent.ksql.util.KsqlStatementException;
import io.confluent.ksql.util.PersistentQueryMetadata;
import io.confluent.ksql.util.QueryMetadata;
import io.confluent.ksql.util.QueuedQueryMetadata;
import io.confluent.ksql.util.Sandbox;
import io.confluent.ksql.util.timestamp.MetadataTimestampExtractionPolicy;
import io.confluent.ksql.version.metrics.ActivenessRegistrar;
Expand Down Expand Up @@ -1853,8 +1854,9 @@ private static void validateQueryDescription(
assertThat(entity, instanceOf(QueryDescriptionEntity.class));
final QueryDescriptionEntity queryDescriptionEntity = (QueryDescriptionEntity) entity;
final QueryDescription queryDescription = queryDescriptionEntity.getQueryDescription();
final boolean valueSchemaOnly = queryMetadata instanceof QueuedQueryMetadata;
assertThat(queryDescription.getFields(), is(
EntityUtil.buildSourceSchemaEntity(queryMetadata.getLogicalSchema())));
EntityUtil.buildSourceSchemaEntity(queryMetadata.getLogicalSchema(), valueSchemaOnly)));
assertThat(queryDescription.getOverriddenProperties(), is(overriddenProperties));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,7 @@
package io.confluent.ksql.rest.util;

import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertThat;

import io.confluent.ksql.rest.entity.FieldInfo;
Expand All @@ -32,7 +30,6 @@
import org.apache.kafka.connect.data.SchemaBuilder;
import org.junit.Test;

@SuppressWarnings("unchecked")
public class EntityUtilTest {

private static final FieldInfo ROWTIME_FIELD =
Expand Down Expand Up @@ -84,15 +81,14 @@ public void shouldBuildCorrectMapField() {
.build());

// When:
final List<FieldInfo> entity = EntityUtil.buildSourceSchemaEntity(schema);
final List<FieldInfo> fields = EntityUtil.buildSourceSchemaEntity(schema, true);

// Then:
final List<FieldInfo> valueFields = getValueFields(entity);
assertThat(valueFields, hasSize(1));
assertThat(valueFields.get(0).getName(), equalTo("field"));
assertThat(valueFields.get(0).getSchema().getTypeName(), equalTo("MAP"));
assertThat(valueFields.get(0).getSchema().getFields(), equalTo(Optional.empty()));
assertThat(valueFields.get(0).getSchema().getMemberSchema().get().getTypeName(),
assertThat(fields, hasSize(1));
assertThat(fields.get(0).getName(), equalTo("field"));
assertThat(fields.get(0).getSchema().getTypeName(), equalTo("MAP"));
assertThat(fields.get(0).getSchema().getFields(), equalTo(Optional.empty()));
assertThat(fields.get(0).getSchema().getMemberSchema().get().getTypeName(),
equalTo("INTEGER"));
}

Expand All @@ -108,15 +104,14 @@ public void shouldBuildCorrectArrayField() {
.build());

// When:
final List<FieldInfo> entity = EntityUtil.buildSourceSchemaEntity(schema);
final List<FieldInfo> fields = EntityUtil.buildSourceSchemaEntity(schema, true);

// Then:
final List<FieldInfo> valueFields = getValueFields(entity);
assertThat(valueFields, hasSize(1));
assertThat(valueFields.get(0).getName(), equalTo("field"));
assertThat(valueFields.get(0).getSchema().getTypeName(), equalTo("ARRAY"));
assertThat(valueFields.get(0).getSchema().getFields(), equalTo(Optional.empty()));
assertThat(valueFields.get(0).getSchema().getMemberSchema().get().getTypeName(),
assertThat(fields, hasSize(1));
assertThat(fields.get(0).getName(), equalTo("field"));
assertThat(fields.get(0).getSchema().getTypeName(), equalTo("ARRAY"));
assertThat(fields.get(0).getSchema().getFields(), equalTo(Optional.empty()));
assertThat(fields.get(0).getSchema().getMemberSchema().get().getTypeName(),
equalTo("BIGINT"));
}

Expand All @@ -135,17 +130,16 @@ public void shouldBuildCorrectStructField() {
.build());

// When:
final List<FieldInfo> entity = EntityUtil.buildSourceSchemaEntity(schema);
final List<FieldInfo> fields = EntityUtil.buildSourceSchemaEntity(schema, true);

// Then:
final List<FieldInfo> valueFields = getValueFields(entity);
assertThat(valueFields, hasSize(1));
assertThat(valueFields.get(0).getName(), equalTo("field"));
assertThat(valueFields.get(0).getSchema().getTypeName(), equalTo("STRUCT"));
assertThat(valueFields.get(0).getSchema().getFields().get().size(), equalTo(1));
final FieldInfo inner = valueFields.get(0).getSchema().getFields().get().get(0);
assertThat(fields, hasSize(1));
assertThat(fields.get(0).getName(), equalTo("field"));
assertThat(fields.get(0).getSchema().getTypeName(), equalTo("STRUCT"));
assertThat(fields.get(0).getSchema().getFields().get().size(), equalTo(1));
final FieldInfo inner = fields.get(0).getSchema().getFields().get().get(0);
assertThat(inner.getSchema().getTypeName(), equalTo("STRING"));
assertThat(valueFields.get(0).getSchema().getMemberSchema(), equalTo(Optional.empty()));
assertThat(fields.get(0).getSchema().getMemberSchema(), equalTo(Optional.empty()));
}

@Test
Expand All @@ -158,15 +152,53 @@ public void shouldBuildMiltipleFieldsCorrectly() {
.build());

// When:
final List<FieldInfo> entity = EntityUtil.buildSourceSchemaEntity(schema);
final List<FieldInfo> fields = EntityUtil.buildSourceSchemaEntity(schema, true);

// Then:
final List<FieldInfo> valueFields = getValueFields(entity);
assertThat(valueFields, hasSize(2));
assertThat(valueFields.get(0).getName(), equalTo("field1"));
assertThat(valueFields.get(0).getSchema().getTypeName(), equalTo("INTEGER"));
assertThat(valueFields.get(1).getName(), equalTo("field2"));
assertThat(valueFields.get(1).getSchema().getTypeName(), equalTo("BIGINT"));
assertThat(fields, hasSize(2));
assertThat(fields.get(0).getName(), equalTo("field1"));
assertThat(fields.get(0).getSchema().getTypeName(), equalTo("INTEGER"));
assertThat(fields.get(1).getName(), equalTo("field2"));
assertThat(fields.get(1).getSchema().getTypeName(), equalTo("BIGINT"));
}

@Test
public void shouldSupportRowTimeAndKeyInValueSchema() {
// Given:
final LogicalSchema schema = LogicalSchema.of(SchemaBuilder
.struct()
.field("ROWKEY", Schema.OPTIONAL_STRING_SCHEMA)
.field("ROWTIME", Schema.OPTIONAL_INT32_SCHEMA)
.field("field1", Schema.OPTIONAL_INT32_SCHEMA)
.build());

// When:
final List<FieldInfo> fields = EntityUtil.buildSourceSchemaEntity(schema, true);

// Then:
assertThat(fields, hasSize(3));
assertThat(fields.get(0).getName(), equalTo("ROWKEY"));
assertThat(fields.get(1).getName(), equalTo("ROWTIME"));
}

@Test
public void shouldSupportGettingFullSchema() {
// Given:
final LogicalSchema schema = LogicalSchema.of(SchemaBuilder
.struct()
.field("field1", Schema.OPTIONAL_INT32_SCHEMA)
.build());

// When:
final List<FieldInfo> fields = EntityUtil.buildSourceSchemaEntity(schema, false);

// Then:
assertThat(fields, hasSize(3));
assertThat(fields.get(0).getName(), equalTo("ROWTIME"));
assertThat(fields.get(0).getSchema().getTypeName(), equalTo("BIGINT"));
assertThat(fields.get(1).getName(), equalTo("ROWKEY"));
assertThat(fields.get(1).getSchema().getTypeName(), equalTo("STRING"));
assertThat(fields.get(2).getName(), equalTo("field1"));
}

private static void shouldBuildCorrectPrimitiveField(
Expand All @@ -180,21 +212,13 @@ private static void shouldBuildCorrectPrimitiveField(
.build());

// When:
final List<FieldInfo> entity = EntityUtil.buildSourceSchemaEntity(schema);
final List<FieldInfo> fields = EntityUtil.buildSourceSchemaEntity(schema, true);

// Then:
final List<FieldInfo> valueFields = getValueFields(entity);
assertThat(valueFields.get(0).getName(), equalTo("field"));
assertThat(valueFields.get(0).getSchema().getTypeName(), equalTo(schemaName));
assertThat(valueFields.get(0).getSchema().getFields(), equalTo(Optional.empty()));
assertThat(valueFields.get(0).getSchema().getMemberSchema(), equalTo(Optional.empty()));
}

private static List<FieldInfo> getValueFields(final List<FieldInfo> entity) {
assertThat(entity, hasSize(greaterThan(2)));
assertThat(entity.get(0), is(ROWTIME_FIELD));
assertThat(entity.get(1), is(ROWKEY_FIELD));
return entity.subList(2, entity.size());
assertThat(fields.get(0).getName(), equalTo("field"));
assertThat(fields.get(0).getSchema().getTypeName(), equalTo(schemaName));
assertThat(fields.get(0).getSchema().getFields(), equalTo(Optional.empty()));
assertThat(fields.get(0).getSchema().getMemberSchema(), equalTo(Optional.empty()));
}
}