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: do not overwrite schemas from a CREATE STREAM/TABLE #5756

Merged
merged 1 commit into from
Jul 1, 2020
Merged
Show file tree
Hide file tree
Changes from all 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 @@ -15,6 +15,8 @@

package io.confluent.ksql.schema.ksql.inference;

import io.confluent.kafka.schemaregistry.ParsedSchema;
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
import io.confluent.ksql.KsqlExecutionContext;
import io.confluent.ksql.KsqlExecutionContext.ExecuteResult;
Expand Down Expand Up @@ -73,7 +75,8 @@ private void registerForCreateSource(final ConfiguredStatement<? extends CreateS
cs.getStatement().getProperties().getKafkaTopic(),
cs.getStatement().getProperties().getFormatInfo(),
cs.getConfig(),
cs.getStatementText()
cs.getStatementText(),
false
);
}

Expand All @@ -96,7 +99,8 @@ private void registerForCreateAs(final ConfiguredStatement<? extends CreateAsSel
queryMetadata.getResultTopic().getKafkaTopicName(),
queryMetadata.getResultTopic().getValueFormat().getFormatInfo(),
cas.getConfig(),
cas.getStatementText()
cas.getStatementText(),
true
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just to make sure, in which case to we want to overwrite existing schemas?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we want to overwrite existing schemas in the case of CREATE * AS SELECT

);
}

Expand All @@ -105,7 +109,8 @@ private void registerSchema(
final String topic,
final FormatInfo formatInfo,
final KsqlConfig config,
final String statementText
final String statementText,
final boolean overwriteExisting
) {
final Format format = FormatFactory.of(formatInfo);
if (!format.supportsSchemaInference()) {
Expand All @@ -115,10 +120,23 @@ private void registerSchema(
if (config.getString(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY) != null
&& !config.getString(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY).isEmpty()) {
try {
serviceContext.getSchemaRegistryClient().register(
topic + KsqlConstants.SCHEMA_REGISTRY_VALUE_SUFFIX,
format.toParsedSchema(schema.withoutPseudoAndKeyColsInValue().value(), formatInfo)
);
final SchemaRegistryClient srClient = serviceContext.getSchemaRegistryClient();
final String subject = topic + KsqlConstants.SCHEMA_REGISTRY_VALUE_SUFFIX;
final ParsedSchema parsedSchema =
format.toParsedSchema(schema.withoutPseudoAndKeyColsInValue().value(), formatInfo);

if (!overwriteExisting && srClient.getAllSubjects().contains(subject)) {
if (!srClient.testCompatibility(subject, parsedSchema)) {
throw new KsqlStatementException(
String.format(
"Could not register schema for subject "
+ "'%s' because it is incompatible with existing schema.",
subject),
statementText);
}
} else {
srClient.register(subject, parsedSchema);
}
} catch (IOException | RestClientException e) {
throw new KsqlStatementException("Could not register schema for topic.", statementText, e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ static SchemaRegistryClient createProxy(final SchemaRegistryClient delegate) {

return LimitedProxyBuilder.forClass(SchemaRegistryClient.class)
.swallow("register", anyParams(), 123)
.forward("getAllSubjects", methodParams(), delegate)
.forward("getLatestSchemaMetadata", methodParams(String.class), delegate)
.forward("getSchemaBySubjectAndId", methodParams(String.class, int.class), delegate)
.forward("testCompatibility",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import static org.mockito.Mockito.when;

import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import io.confluent.kafka.schemaregistry.ParsedSchema;
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
Expand Down Expand Up @@ -178,6 +179,38 @@ public void shouldRegisterSchemaForSchemaRegistryEnabledFormatCreateSource()
verify(schemaRegistryClient).register("expectedName-value", AVRO_SCHEMA);
}

@Test
public void shouldNotReplaceExistingSchemaForSchemaRegistryEnabledFormatCreateSource()
throws IOException, RestClientException {
// Given:
givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH (kafka_topic='expectedName', value_format='AVRO', partitions=1);");
when(schemaRegistryClient.getAllSubjects()).thenReturn(ImmutableSet.of("expectedName-value"));
when(schemaRegistryClient.testCompatibility(eq("expectedName-value"), any(ParsedSchema.class))).thenReturn(true);

// When:
injector.inject(statement);

// Then:
verify(schemaRegistryClient).getAllSubjects();
verify(schemaRegistryClient).testCompatibility(eq("expectedName-value"), any(ParsedSchema.class));
verifyNoMoreInteractions(schemaRegistryClient);
}

@Test
public void shouldThrowOnExistingSchemaForSchemaRegistryEnabledFormatCreateSourceIncompatible()
throws IOException, RestClientException {
// Given:
givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH (kafka_topic='expectedName', value_format='AVRO', partitions=1);");
when(schemaRegistryClient.getAllSubjects()).thenReturn(ImmutableSet.of("expectedName-value"));
when(schemaRegistryClient.testCompatibility(eq("expectedName-value"), any(ParsedSchema.class))).thenReturn(false);

// When:
final KsqlStatementException e = assertThrows(KsqlStatementException.class, () -> injector.inject(statement));

// Then:
assertThat(e.getMessage(), containsString("Could not register schema for subject 'expectedName-value' because it is incompatible with existing schema."));
}

@Test
public void shouldNotRegisterSchemaForSchemaRegistryDisabledFormatCreateAsSelect() {
// Given:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static org.mockito.Mockito.verifyZeroInteractions;
import static org.mockito.Mockito.when;

import com.google.common.collect.ImmutableSet;
import io.confluent.kafka.schemaregistry.ParsedSchema;
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
import io.confluent.kafka.schemaregistry.client.SchemaMetadata;
Expand Down Expand Up @@ -61,6 +62,7 @@ public static Collection<TestCase<SchemaRegistryClient>> getMethodsToTest() {
.ignore("testCompatibility", String.class, Schema.class)
.ignore("testCompatibility", String.class, ParsedSchema.class)
.ignore("deleteSubject", String.class)
.ignore("getAllSubjects")
.build();
}

Expand Down Expand Up @@ -127,6 +129,18 @@ public void shouldTestCompatibility() throws Exception {
assertThat(second, is(false));
}

@Test
public void shouldGetAllSubjects() throws Exception {
// Given:
when(delegate.getAllSubjects()).thenReturn(ImmutableSet.of("foo"));

// When:
final Collection<String> subjects = sandboxedClient.getAllSubjects();

// Then:
assertThat(subjects, is(ImmutableSet.of("foo")));
}

@Test
public void shouldSwallowDeleteSubject() throws Exception {
// When:
Expand Down