Skip to content

Commit

Permalink
fix: fail AVRO/Protobuf/JSON Schema statements if SR is missing (#5597)
Browse files Browse the repository at this point in the history
  • Loading branch information
agavra committed Jun 11, 2020
1 parent 49cfbb3 commit 85a0320
Show file tree
Hide file tree
Showing 10 changed files with 45 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -107,8 +107,11 @@ private void registerSchema(
final String statementText
) {
final Format format = FormatFactory.of(formatInfo);
if (format.supportsSchemaInference()
&& config.getString(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY) != null
if (!format.supportsSchemaInference()) {
return;
}

if (config.getString(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY) != null
&& !config.getString(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY).isEmpty()) {
try {
serviceContext.getSchemaRegistryClient().register(
Expand All @@ -118,6 +121,12 @@ private void registerSchema(
} catch (IOException | RestClientException e) {
throw new KsqlStatementException("Could not register schema for topic.", statementText, e);
}
} else {
throw new KsqlStatementException(
String.format(
"Cannot create topic '%s' with format %s without configuring '%s'",
topic, format.name(), KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY),
statementText);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import io.confluent.ksql.query.BlockingRowQueue;
import io.confluent.ksql.query.QueryId;
import io.confluent.ksql.serde.Format;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.KsqlConstants;
import io.confluent.ksql.util.PageViewDataProvider;
import io.confluent.ksql.util.PersistentQueryMetadata;
Expand Down Expand Up @@ -116,6 +117,9 @@ public class EndToEndIntegrationTest {
KSQL_FUNCTIONS_PROPERTY_PREFIX + "_global_.expected-param",
"expected-value"
)
.withAdditionalConfig(
KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY,
"http://foo:8080")
.build();

@Rule
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import io.confluent.ksql.serde.FormatFactory;
import io.confluent.ksql.test.util.TopicTestUtil;
import io.confluent.ksql.util.ItemDataProvider;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.OrderDataProvider;
import java.util.HashMap;
import java.util.Map;
Expand Down Expand Up @@ -61,7 +62,9 @@ public class JoinIntTest {
.around(TEST_HARNESS);

@Rule
public final TestKsqlContext ksqlContext = TEST_HARNESS.buildKsqlContext();
public final TestKsqlContext ksqlContext = TEST_HARNESS.ksqlContextBuilder()
.withAdditionalConfig(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY, "http://foo:8080")
.build();

private final long now = System.currentTimeMillis();
private String itemTableTopicJson = "ItemTopicJson";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import io.confluent.ksql.serde.Format;
import io.confluent.ksql.test.util.KsqlIdentifierTestUtil;
import io.confluent.ksql.test.util.TopicTestUtil;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.OrderDataProvider;
import java.text.SimpleDateFormat;
import java.util.HashMap;
Expand Down Expand Up @@ -65,7 +66,9 @@ public class StreamsSelectAndProjectIntTest {
.around(TEST_HARNESS);

@Rule
public final TestKsqlContext ksqlContext = TEST_HARNESS.buildKsqlContext();
public final TestKsqlContext ksqlContext = TEST_HARNESS.ksqlContextBuilder()
.withAdditionalConfig(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY, "http://foo:8080")
.build();

private String jsonTopicName;
private String avroTopicName;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import io.confluent.ksql.serde.json.JsonFormat;
import io.confluent.ksql.test.util.KsqlIdentifierTestUtil;
import io.confluent.ksql.util.ItemDataProvider;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.OrderDataProvider;
import java.util.Collection;
import java.util.Collections;
Expand Down Expand Up @@ -79,7 +80,9 @@ public class UdfIntTest {
.around(TEST_HARNESS);

@Rule
public final TestKsqlContext ksqlContext = TEST_HARNESS.buildKsqlContext();
public final TestKsqlContext ksqlContext = TEST_HARNESS.ksqlContextBuilder()
.withAdditionalConfig(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY, "http://foo:8080")
.build();

private final TestData testData;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,10 +146,10 @@ public void shouldNotRegisterSchemaIfSchemaRegistryIsDisabled() {
givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH(kafka_topic='expectedName', value_format='AVRO', partitions=1);");

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

// Then:
verifyNoMoreInteractions(schemaRegistryClient);
assertThat(e.getMessage(), containsString("Cannot create topic 'expectedName' with format AVRO without configuring"));
}

@Test
Expand Down Expand Up @@ -184,10 +184,10 @@ public void shouldNotRegisterSchemaForSchemaRegistryDisabledFormatCreateAsSelect
givenStatement("CREATE STREAM sink WITH(value_format='DELIMITED') AS SELECT * FROM SOURCE;");

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

// Then:
verifyNoMoreInteractions(schemaRegistryClient);
assertThat(e.getMessage(), containsString("Cannot create topic 'SINK' with format AVRO without configuring"));
}

@Test
Expand Down Expand Up @@ -242,8 +242,7 @@ public void shouldPropagateErrorOnSRClientError() throws IOException, RestClient
@Test
public void shouldNotExecuteQueryOnOriginalExecutionContext() {
// Given:
config = new KsqlConfig(ImmutableMap.of());
givenStatement("CREATE STREAM sink WITH(value_format='DELIMITED') AS SELECT * FROM SOURCE;");
givenStatement("CREATE STREAM sink WITH(value_format='AVRO') AS SELECT * FROM SOURCE;");

// When:
injector.inject(statement);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import io.confluent.ksql.rest.entity.ClusterTerminateRequest;
import io.confluent.ksql.rest.entity.KsqlErrorMessage;
import io.confluent.ksql.rest.server.TestKsqlRestApp;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.KsqlConstants;
import io.confluent.ksql.util.PageViewDataProvider;
import io.vertx.core.buffer.Buffer;
Expand Down Expand Up @@ -58,6 +59,7 @@ public class ClusterTerminationTest {
private static final TestKsqlRestApp REST_APP = TestKsqlRestApp
.builder(TEST_HARNESS::kafkaBootstrapServers)
.withStaticServiceContext(TEST_HARNESS::getServiceContext)
.withProperty(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY, "http://foo:8080")
.build();

@ClassRule
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import io.confluent.ksql.serde.FormatFactory;
import io.confluent.ksql.serde.SerdeOption;
import io.confluent.ksql.serde.avro.AvroSchemas;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.KsqlConstants;
import io.confluent.ksql.util.PageViewDataProvider;
import java.util.List;
Expand Down Expand Up @@ -69,6 +70,7 @@ public class KsqlResourceFunctionalTest {
private static final TestKsqlRestApp REST_APP = TestKsqlRestApp
.builder(TEST_HARNESS::kafkaBootstrapServers)
.withStaticServiceContext(TEST_HARNESS::getServiceContext)
.withProperty(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY, "http://foo:8080")
.build();

@ClassRule
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,15 +51,20 @@
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.FixMethodOrder;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.MethodSorters;
import org.mockito.Mock;
import org.mockito.junit.MockitoJUnitRunner;

@Category({IntegrationTest.class})
@RunWith(MockitoJUnitRunner.class)
// shouldFailOnAvroWithoutSchemasIfSchemaNotEvolvable fails if run after shouldHandleJsonWithSchemas
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class StandaloneExecutorFunctionalTest {

@ClassRule
Expand Down Expand Up @@ -100,6 +105,7 @@ public void setUp() throws Exception {
final Map<String, Object> properties = ImmutableMap.<String, Object>builder()
.putAll(KsqlConfigTestUtil.baseTestConfig())
.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, TEST_HARNESS.kafkaBootstrapServers())
.put(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY, "http://foo:8080")
.build();

final Function<KsqlConfig, ServiceContext> serviceContextFactory = config ->
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,9 @@ public class StandaloneExecutorTest {
ProcessingLogConfig.TOPIC_AUTO_CREATE, true,
ProcessingLogConfig.TOPIC_NAME, PROCESSING_LOG_TOPIC_NAME
));
private static final KsqlConfig ksqlConfig = new KsqlConfig(emptyMap());
private static final KsqlConfig ksqlConfig = new KsqlConfig(
ImmutableMap.of(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY, "http://foo:8080")
);

private static final TableElements SOME_ELEMENTS = TableElements.of(
new TableElement(Namespace.VALUE, ColumnName.of("bob"), new Type(SqlTypes.STRING)));
Expand Down

0 comments on commit 85a0320

Please sign in to comment.