-
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: Use a SandboxedPersistentQueryMetadata to not interact with Kafk…
…Streams (#6066)
- Loading branch information
Showing
8 changed files
with
293 additions
and
26 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
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
54 changes: 54 additions & 0 deletions
54
ksqldb-engine/src/main/java/io/confluent/ksql/util/SandboxedPersistentQueryMetadata.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,54 @@ | ||
/* | ||
* 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.util; | ||
|
||
import java.util.function.Consumer; | ||
|
||
/** | ||
* Sandboxed {@link PersistentQueryMetadata} that prevents to modify the state of the internal | ||
* {@link org.apache.kafka.streams.KafkaStreams}. | ||
*/ | ||
public final class SandboxedPersistentQueryMetadata extends PersistentQueryMetadata { | ||
public static SandboxedPersistentQueryMetadata of( | ||
final PersistentQueryMetadata queryMetadata, | ||
final Consumer<QueryMetadata> closeCallback | ||
) { | ||
return new SandboxedPersistentQueryMetadata(queryMetadata, closeCallback); | ||
} | ||
|
||
private SandboxedPersistentQueryMetadata( | ||
final PersistentQueryMetadata queryMetadata, | ||
final Consumer<QueryMetadata> closeCallback | ||
) { | ||
super(queryMetadata, closeCallback); | ||
} | ||
|
||
@Override | ||
public void close() { | ||
closed = true; | ||
closeCallback.accept(this); | ||
} | ||
|
||
@Override | ||
public void start() { | ||
// no-op | ||
} | ||
|
||
@Override | ||
protected void closeKafkaStreams() { | ||
// no-op | ||
} | ||
} |
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
154 changes: 154 additions & 0 deletions
154
ksqldb-engine/src/test/java/io/confluent/ksql/util/SandboxedPersistentQueryMetadataTest.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,154 @@ | ||
/* | ||
* 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.util; | ||
|
||
import io.confluent.ksql.execution.plan.ExecutionStep; | ||
import io.confluent.ksql.execution.streams.materialization.MaterializationProvider; | ||
import io.confluent.ksql.metastore.model.DataSource; | ||
import io.confluent.ksql.query.KafkaStreamsBuilder; | ||
import io.confluent.ksql.query.MaterializationProviderBuilderFactory; | ||
import io.confluent.ksql.query.QueryErrorClassifier; | ||
import io.confluent.ksql.query.QueryId; | ||
import io.confluent.ksql.schema.ksql.LogicalSchema; | ||
import io.confluent.ksql.schema.ksql.PhysicalSchema; | ||
import org.apache.kafka.streams.KafkaStreams; | ||
import org.apache.kafka.streams.Topology; | ||
import org.junit.Before; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
import org.mockito.Mock; | ||
import org.mockito.junit.MockitoJUnitRunner; | ||
|
||
import java.util.Collections; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
import java.util.function.Consumer; | ||
|
||
import static org.mockito.ArgumentMatchers.any; | ||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.never; | ||
import static org.mockito.Mockito.reset; | ||
import static org.mockito.Mockito.verify; | ||
import static org.mockito.Mockito.verifyZeroInteractions; | ||
import static org.mockito.Mockito.when; | ||
|
||
@RunWith(MockitoJUnitRunner.class) | ||
public class SandboxedPersistentQueryMetadataTest { | ||
private static final String SQL = "sql"; | ||
private static final String EXECUTION_PLAN = "execution plan"; | ||
private static final QueryId QUERY_ID = new QueryId("queryId"); | ||
private static final String APPLICATION_ID = "applicationId"; | ||
private static final long CLOSE_TIMEOUT = 10L; | ||
|
||
@Mock | ||
private KafkaStreamsBuilder kafkaStreamsBuilder; | ||
@Mock | ||
private KafkaStreams kafkaStreams; | ||
@Mock | ||
private PhysicalSchema physicalSchema; | ||
@Mock | ||
private DataSource sinkDataSource; | ||
@Mock | ||
private MaterializationProviderBuilderFactory.MaterializationProviderBuilder | ||
materializationProviderBuilder; | ||
@Mock | ||
private MaterializationProvider materializationProvider; | ||
@Mock | ||
private Topology topology; | ||
@Mock | ||
private QuerySchemas schemas; | ||
@Mock | ||
private Map<String, Object> props; | ||
@Mock | ||
private Map<String, Object> overrides; | ||
@Mock | ||
private Consumer<QueryMetadata> closeCallback; | ||
@Mock | ||
private QueryErrorClassifier queryErrorClassifier; | ||
@Mock | ||
private ExecutionStep<?> physicalPlan; | ||
|
||
private PersistentQueryMetadata query; | ||
private SandboxedPersistentQueryMetadata sandbox; | ||
|
||
@Before | ||
public void setUp() { | ||
when(kafkaStreamsBuilder.build(any(), any())).thenReturn(kafkaStreams); | ||
when(physicalSchema.logicalSchema()).thenReturn(mock(LogicalSchema.class)); | ||
when(materializationProviderBuilder.apply(kafkaStreams)) | ||
.thenReturn(Optional.of(materializationProvider)); | ||
|
||
query = new PersistentQueryMetadata( | ||
SQL, | ||
physicalSchema, | ||
Collections.emptySet(), | ||
sinkDataSource, | ||
EXECUTION_PLAN, | ||
QUERY_ID, | ||
Optional.of(materializationProviderBuilder), | ||
APPLICATION_ID, | ||
topology, | ||
kafkaStreamsBuilder, | ||
schemas, | ||
props, | ||
overrides, | ||
closeCallback, | ||
CLOSE_TIMEOUT, | ||
queryErrorClassifier, | ||
physicalPlan, | ||
10 | ||
); | ||
|
||
sandbox = SandboxedPersistentQueryMetadata.of(query, closeCallback); | ||
reset(kafkaStreams); | ||
} | ||
|
||
@Test | ||
public void shouldNotStartKafkaStreamsOnStart() { | ||
// When: | ||
sandbox.start(); | ||
|
||
// Then: | ||
verifyZeroInteractions(kafkaStreams); | ||
} | ||
|
||
@Test | ||
public void shouldNotCloseKafkaStreamsOnStop() { | ||
// When: | ||
sandbox.stop(); | ||
|
||
// Then: | ||
verifyZeroInteractions(kafkaStreams); | ||
} | ||
|
||
@Test | ||
public void shouldNotCloseKafkaStreamsOnClose() { | ||
// When: | ||
sandbox.close(); | ||
|
||
// Then: | ||
verifyZeroInteractions(kafkaStreams); | ||
} | ||
|
||
@Test | ||
public void shouldNotCloseStateListenerOnClose() { | ||
// When: | ||
sandbox.stop(); | ||
|
||
// Then: | ||
verifyZeroInteractions(closeCallback); | ||
} | ||
} |
Oops, something went wrong.