Skip to content

Commit

Permalink
fix: create schemas at topic creation (#4717)
Browse files Browse the repository at this point in the history
  • Loading branch information
agavra committed Mar 10, 2020
1 parent f794ac0 commit 514025d
Show file tree
Hide file tree
Showing 5 changed files with 394 additions and 3 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* Copyright 2020 Confluent Inc.
*
* Licensed under the Confluent Community License (the "License"; you may not use
* this file except in compliance with the License. You may obtain a copy of the
* License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

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

import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
import io.confluent.ksql.KsqlExecutionContext;
import io.confluent.ksql.KsqlExecutionContext.ExecuteResult;
import io.confluent.ksql.parser.tree.CreateAsSelect;
import io.confluent.ksql.parser.tree.CreateSource;
import io.confluent.ksql.parser.tree.Statement;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.serde.Format;
import io.confluent.ksql.serde.FormatFactory;
import io.confluent.ksql.serde.FormatInfo;
import io.confluent.ksql.services.SandboxedServiceContext;
import io.confluent.ksql.services.ServiceContext;
import io.confluent.ksql.statement.ConfiguredStatement;
import io.confluent.ksql.statement.Injector;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.KsqlConstants;
import io.confluent.ksql.util.KsqlStatementException;
import io.confluent.ksql.util.PersistentQueryMetadata;
import java.io.IOException;
import java.util.Objects;

public class SchemaRegisterInjector implements Injector {

private final KsqlExecutionContext executionContext;
private final ServiceContext serviceContext;

public SchemaRegisterInjector(
final KsqlExecutionContext executionContext,
final ServiceContext serviceContext
) {
this.executionContext = Objects.requireNonNull(executionContext, "executionContext");
this.serviceContext = Objects.requireNonNull(serviceContext, "serviceContext");
}

@SuppressWarnings("unchecked")
@Override
public <T extends Statement> ConfiguredStatement<T> inject(
final ConfiguredStatement<T> statement
) {
if (statement.getStatement() instanceof CreateAsSelect) {
registerForCreateAs((ConfiguredStatement<? extends CreateAsSelect>) statement);
} else if (statement.getStatement() instanceof CreateSource) {
registerForCreateSource((ConfiguredStatement<? extends CreateSource>) statement);
}

return statement;
}

private void registerForCreateSource(final ConfiguredStatement<? extends CreateSource> cs) {
// since this injector is chained after the TopicCreateInjector,
// we can assume that the kafka topic is always present in the
// statement properties
registerSchema(
cs.getStatement().getElements().toLogicalSchema(false),
cs.getStatement().getProperties().getKafkaTopic(),
cs.getStatement().getProperties().getFormatInfo(),
cs.getConfig(),
cs.getStatementText()
);
}

private void registerForCreateAs(final ConfiguredStatement<? extends CreateAsSelect> cas) {
final ServiceContext sandboxServiceContext = SandboxedServiceContext.create(serviceContext);
final ExecuteResult executeResult = executionContext
.createSandbox(sandboxServiceContext)
.execute(sandboxServiceContext, cas);

final PersistentQueryMetadata queryMetadata = (PersistentQueryMetadata) executeResult
.getQuery()
.orElseThrow(() -> new KsqlStatementException(
"Could not determine output schema for query due to error: "
+ executeResult.getCommandResult(),
cas.getStatementText()
));

registerSchema(
queryMetadata.getLogicalSchema(),
queryMetadata.getResultTopic().getKafkaTopicName(),
queryMetadata.getResultTopic().getValueFormat().getFormatInfo(),
cas.getConfig(),
cas.getStatementText()
);
}

private void registerSchema(
final LogicalSchema schema,
final String topic,
final FormatInfo formatInfo,
final KsqlConfig config,
final String statementText
) {
final Format format = FormatFactory.of(formatInfo);
if (format.supportsSchemaInference()
&& 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.withoutMetaAndKeyColsInValue().value(), formatInfo)
);
} catch (IOException | RestClientException e) {
throw new KsqlStatementException("Could not register schema for topic.", statementText, e);
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

import io.confluent.ksql.KsqlExecutionContext;
import io.confluent.ksql.schema.ksql.inference.DefaultSchemaInjector;
import io.confluent.ksql.schema.ksql.inference.SchemaRegisterInjector;
import io.confluent.ksql.schema.ksql.inference.SchemaRegistryTopicSchemaSupplier;
import io.confluent.ksql.services.ServiceContext;
import io.confluent.ksql.topic.TopicCreateInjector;
Expand All @@ -29,7 +30,8 @@ public enum Injectors implements BiFunction<KsqlExecutionContext, ServiceContext
NO_TOPIC_DELETE((ec, sc) -> InjectorChain.of(
new DefaultSchemaInjector(
new SchemaRegistryTopicSchemaSupplier(sc.getSchemaRegistryClient())),
new TopicCreateInjector(ec, sc)
new TopicCreateInjector(ec, sc),
new SchemaRegisterInjector(ec, sc)
)),

DEFAULT((ec, sc) -> InjectorChain.of(
Expand Down
Loading

0 comments on commit 514025d

Please sign in to comment.