-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
fix: preserve old schema behavior for protobuf wrapped primitives (#8934
- Loading branch information
Showing
384 changed files
with
56,705 additions
and
60 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
123 changes: 123 additions & 0 deletions
123
ksqldb-engine/src/main/java/io/confluent/ksql/statement/SourcePropertyInjector.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,123 @@ | ||
/* | ||
* Copyright 2022 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community 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.statement; | ||
|
||
import io.confluent.ksql.parser.KsqlParser.PreparedStatement; | ||
import io.confluent.ksql.parser.SqlFormatter; | ||
import io.confluent.ksql.parser.properties.with.CreateSourceAsProperties; | ||
import io.confluent.ksql.parser.properties.with.CreateSourceProperties; | ||
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.util.ErrorMessageUtil; | ||
import io.confluent.ksql.util.KsqlException; | ||
import io.confluent.ksql.util.KsqlStatementException; | ||
|
||
/** | ||
* An injector which injects information into {@code CreateSourceProperties} | ||
* or {@code CreateSourceAsProperties}. | ||
* | ||
* <p>The only information injected currently is that new queries using the | ||
* protobuf format should use the new default converter behavior to unwrap | ||
* wrapped primitives. This injector causes new queries to use the new behavior | ||
* while old (existing) queries will continue to use the old behavior. | ||
* | ||
* <p>If a statement that is not {@code CreateAsSelect} or {@code CreateSource} | ||
* is passed in, this results in a no-op that returns the incoming statement.</p> | ||
*/ | ||
public class SourcePropertyInjector implements Injector { | ||
|
||
@SuppressWarnings("unchecked") | ||
public <T extends Statement> ConfiguredStatement<T> inject( | ||
final ConfiguredStatement<T> statement | ||
) { | ||
if (!(statement.getStatement() instanceof CreateSource) | ||
&& !(statement.getStatement() instanceof CreateAsSelect)) { | ||
return statement; | ||
} | ||
|
||
try { | ||
if (statement.getStatement() instanceof CreateAsSelect) { | ||
return (ConfiguredStatement<T>) injectForCreateAsSelect( | ||
(ConfiguredStatement<? extends CreateAsSelect>) statement); | ||
} else { | ||
return (ConfiguredStatement<T>) injectForCreateSource( | ||
(ConfiguredStatement<? extends CreateSource>) statement); | ||
} | ||
} catch (final KsqlStatementException e) { | ||
throw e; | ||
} catch (final KsqlException e) { | ||
throw new KsqlStatementException( | ||
ErrorMessageUtil.buildErrorMessage(e), | ||
statement.getStatementText(), | ||
e.getCause()); | ||
} | ||
} | ||
|
||
private ConfiguredStatement<? extends CreateSource> injectForCreateSource( | ||
final ConfiguredStatement<? extends CreateSource> original | ||
) { | ||
final CreateSource statement = original.getStatement(); | ||
final CreateSourceProperties properties = statement.getProperties(); | ||
|
||
final CreateSourceProperties injectedProps = properties.withUnwrapProtobufPrimitives(true); | ||
|
||
return buildConfiguredStatement(original, injectedProps); | ||
} | ||
|
||
private ConfiguredStatement<? extends CreateAsSelect> injectForCreateAsSelect( | ||
final ConfiguredStatement<? extends CreateAsSelect> original | ||
) { | ||
final CreateAsSelect createAsSelect = original.getStatement(); | ||
final CreateSourceAsProperties properties = createAsSelect.getProperties(); | ||
|
||
final CreateSourceAsProperties injectedProps = properties.withUnwrapProtobufPrimitives(true); | ||
|
||
return buildConfiguredStatement(original, injectedProps); | ||
} | ||
|
||
private static ConfiguredStatement<CreateSource> buildConfiguredStatement( | ||
final ConfiguredStatement<? extends CreateSource> original, | ||
final CreateSourceProperties injectedProps | ||
) { | ||
final CreateSource statement = original.getStatement(); | ||
|
||
final CreateSource withProps = statement.copyWith( | ||
original.getStatement().getElements(), | ||
injectedProps | ||
); | ||
|
||
final PreparedStatement<CreateSource> prepared = buildPreparedStatement(withProps); | ||
return ConfiguredStatement.of(prepared, original.getSessionConfig()); | ||
} | ||
|
||
private static ConfiguredStatement<CreateAsSelect> buildConfiguredStatement( | ||
final ConfiguredStatement<? extends CreateAsSelect> original, | ||
final CreateSourceAsProperties injectedProps | ||
) { | ||
final CreateAsSelect statement = original.getStatement(); | ||
|
||
final CreateAsSelect withProps = statement.copyWith(injectedProps); | ||
|
||
final PreparedStatement<CreateAsSelect> prepared = buildPreparedStatement(withProps); | ||
return ConfiguredStatement.of(prepared, original.getSessionConfig()); | ||
} | ||
|
||
private static <T extends Statement> PreparedStatement<T> buildPreparedStatement( | ||
final T stmt | ||
) { | ||
return PreparedStatement.of(SqlFormatter.formatSql(stmt), stmt); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
104 changes: 104 additions & 0 deletions
104
ksqldb-engine/src/test/java/io/confluent/ksql/statement/SourcePropertyInjectorTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,104 @@ | ||
/* | ||
* Copyright 2022 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.statement; | ||
|
||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.is; | ||
import static org.mockito.Mockito.verify; | ||
import static org.mockito.Mockito.when; | ||
|
||
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; | ||
import io.confluent.ksql.config.SessionConfig; | ||
import io.confluent.ksql.parser.properties.with.CreateSourceAsProperties; | ||
import io.confluent.ksql.parser.properties.with.CreateSourceProperties; | ||
import io.confluent.ksql.parser.tree.CreateAsSelect; | ||
import io.confluent.ksql.parser.tree.CreateSource; | ||
import io.confluent.ksql.parser.tree.TableElements; | ||
import org.junit.Before; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
import org.mockito.Mock; | ||
import org.mockito.junit.MockitoJUnitRunner; | ||
|
||
@RunWith(MockitoJUnitRunner.class) | ||
public class SourcePropertyInjectorTest { | ||
|
||
@Mock | ||
private SessionConfig sessionConfig; | ||
|
||
@Mock | ||
private CreateSource createSource; | ||
@Mock | ||
private ConfiguredStatement<CreateSource> csStatement; | ||
@Mock | ||
private CreateSourceProperties originalCSProps; | ||
@Mock | ||
private CreateSourceProperties csPropsWithUnwrapping; | ||
@Mock | ||
private TableElements tableElements; | ||
@Mock | ||
private CreateSource csWithUnwrapping; | ||
|
||
@Mock | ||
private CreateAsSelect createAsSelect; | ||
@Mock | ||
private ConfiguredStatement<CreateAsSelect> csasStatement; | ||
@Mock | ||
private CreateSourceAsProperties originalCsasProps; | ||
@Mock | ||
private CreateSourceAsProperties csasPropsWithUnwrapping; | ||
@Mock | ||
private CreateAsSelect csasWithUnwrapping; | ||
|
||
private SourcePropertyInjector injector; | ||
|
||
@Before | ||
public void setUp() { | ||
when(csStatement.getStatement()).thenReturn(createSource); | ||
when(csStatement.getSessionConfig()).thenReturn(sessionConfig); | ||
when(createSource.getProperties()).thenReturn(originalCSProps); | ||
when(createSource.getElements()).thenReturn(tableElements); | ||
when(originalCSProps.withUnwrapProtobufPrimitives(true)).thenReturn(csPropsWithUnwrapping); | ||
when(createSource.copyWith(tableElements, csPropsWithUnwrapping)).thenReturn(csWithUnwrapping); | ||
|
||
when(csasStatement.getStatement()).thenReturn(createAsSelect); | ||
when(csasStatement.getSessionConfig()).thenReturn(sessionConfig); | ||
when(createAsSelect.getProperties()).thenReturn(originalCsasProps); | ||
when(originalCsasProps.withUnwrapProtobufPrimitives(true)).thenReturn(csasPropsWithUnwrapping); | ||
when(createAsSelect.copyWith(csasPropsWithUnwrapping)).thenReturn(csasWithUnwrapping); | ||
|
||
injector = new SourcePropertyInjector(); | ||
} | ||
|
||
@Test | ||
public void shouldInjectForCreateSource() { | ||
// When: | ||
final ConfiguredStatement<CreateSource> configured = injector.inject(csStatement); | ||
|
||
// Then: | ||
assertThat(configured.getStatement(), is(csWithUnwrapping)); | ||
} | ||
|
||
@Test | ||
public void shouldInjectForCreateAsSelect() { | ||
// When: | ||
final ConfiguredStatement<CreateAsSelect> configured = injector.inject(csasStatement); | ||
|
||
// Then: | ||
assertThat(configured.getStatement(), is(csasWithUnwrapping)); | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.