-
Notifications
You must be signed in to change notification settings - Fork 1k
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
Set the clean up policy to compact for non-windowed tables. #1042
Changes from 2 commits
91c2193
fcaff17
c7e94f0
1273ba2
915c19e
226a837
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,7 +16,6 @@ | |
|
||
package io.confluent.ksql.util; | ||
|
||
import org.apache.kafka.clients.admin.DescribeConfigsResult; | ||
import org.apache.kafka.clients.admin.TopicDescription; | ||
|
||
import java.io.Closeable; | ||
|
@@ -74,11 +73,12 @@ void createTopic( | |
Map<String, TopicDescription> describeTopics(Collection<String> topicNames); | ||
|
||
/** | ||
* [warn] synchronous call to get the response | ||
* [warn] Asynchronous call to get the response | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this call looks synchronous to me |
||
* | ||
* @param topicName topicName to describe | ||
* @param topicName topicNames to describe | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why make this plural. It is still a single topic. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This calls There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes. But this method just takes a single topic and passes a list with a single element and also assumes the output contains a single pair. So it is more appropriate for javadoc for this method to be singular rather than plural. |
||
*/ | ||
DescribeConfigsResult describeConfigs(String topicName); | ||
public String getTopicCleanupPolicy(String topicName); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Would be better if this was an enum with three values: |
||
|
||
|
||
/** | ||
* Delete the list of the topics in the given list. | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
|
||
import org.apache.kafka.clients.admin.AdminClient; | ||
import org.apache.kafka.clients.admin.Config; | ||
import org.apache.kafka.clients.admin.ConfigEntry; | ||
import org.apache.kafka.clients.admin.DeleteTopicsResult; | ||
import org.apache.kafka.clients.admin.DescribeClusterResult; | ||
import org.apache.kafka.clients.admin.DescribeConfigsResult; | ||
|
@@ -137,10 +138,34 @@ public Map<String, TopicDescription> describeTopics(final Collection<String> top | |
} | ||
} | ||
|
||
|
||
|
||
@Override | ||
public DescribeConfigsResult describeConfigs(String topicName) { | ||
return adminClient.describeConfigs( | ||
Collections.singleton(new ConfigResource(ConfigResource.Type.TOPIC, topicName))); | ||
public String getTopicCleanupPolicy(String topicName) { | ||
RetryHelper<Map<ConfigResource, Config>> retryHelper = new RetryHelper<>(); | ||
Map<ConfigResource, Config> configMap = null; | ||
try { | ||
configMap = retryHelper.executeWithRetries( | ||
() -> { | ||
return adminClient.describeConfigs(Collections.singleton( | ||
new ConfigResource(ConfigResource.Type.TOPIC, topicName))) | ||
.all(); | ||
}); | ||
} catch (Exception e) { | ||
throw new KsqlException("Could not get the topic configs for : " + topicName, e); | ||
} | ||
if (configMap == null) { | ||
throw new KsqlException("Could not get the topic configs for : " + topicName); | ||
} | ||
Object[] configValues = configMap.values().stream().findFirst().get() | ||
.entries() | ||
.stream() | ||
.filter(configEntry -> configEntry.name().equalsIgnoreCase("cleanup.policy")) | ||
.toArray(); | ||
if (configValues == null || configValues.length ==0) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: == 0 |
||
throw new KsqlException("Could not get the topic configs for : " + topicName); | ||
} | ||
return ((ConfigEntry) configValues[0]).value(); | ||
} | ||
|
||
@Override | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -133,7 +133,7 @@ private void execInitCreateStreamQueries() throws Exception { | |
|
||
String usersTableStr = String.format("CREATE TABLE %s (userid varchar, age integer) WITH " | ||
+ "(value_format = 'json', kafka_topic='%s', " | ||
+ "key='userid');", | ||
+ "KEY='userid');", | ||
usersTable, usersTopic); | ||
|
||
String messageStreamStr = String.format("CREATE STREAM %s (message varchar) WITH (value_format = 'json', " | ||
|
@@ -203,21 +203,11 @@ public void testSinkProperties() throws Exception { | |
assertThat( | ||
topicClient.describeTopics(ImmutableList.of(streamName)).get(streamName).partitions(), | ||
hasSize(3)); | ||
Map<ConfigResource, Config> configResourceConfigMap = topicClient.describeConfigs | ||
(streamName).all().get(1000, TimeUnit.MILLISECONDS); | ||
assertThat(configResourceConfigMap.values().size(), equalTo(1)); | ||
Object[] configEntries = | ||
configResourceConfigMap.values().stream().findFirst().get() | ||
.entries() | ||
.stream() | ||
.filter | ||
(configEntry -> configEntry.name().equalsIgnoreCase("cleanup.policy")).toArray(); | ||
assertThat(configEntries[0], instanceOf(ConfigEntry.class)); | ||
assertThat(((ConfigEntry) configEntries[0]).value(), equalTo("delete")); | ||
assertThat(topicClient.getTopicCleanupPolicy(streamName), equalTo("delete")); | ||
} | ||
|
||
@Test | ||
public void testSinkCleanupPolicy() throws Exception { | ||
public void testTableSinkCleanupProperty() throws Exception { | ||
final String tableName = "SinkCleanupTable".toUpperCase(); | ||
final int resultPartitionCount = 3; | ||
final String queryString = String.format("CREATE TABLE %s AS SELECT * " | ||
|
@@ -230,17 +220,7 @@ public void testSinkCleanupPolicy() throws Exception { | |
"Wait for async topic creation" | ||
); | ||
|
||
Map<ConfigResource, Config> configResourceConfigMap = topicClient.describeConfigs | ||
(tableName).all().get(1000, TimeUnit.MILLISECONDS); | ||
assertThat(configResourceConfigMap.values().size(), equalTo(1)); | ||
Object[] configEntries = | ||
configResourceConfigMap.values().stream().findFirst().get() | ||
.entries() | ||
.stream() | ||
.filter | ||
(configEntry -> configEntry.name().equalsIgnoreCase("cleanup.policy")).toArray(); | ||
assertThat(configEntries[0], instanceOf(ConfigEntry.class)); | ||
assertThat(((ConfigEntry) configEntries[0]).value(), equalTo("compact")); | ||
assertThat(topicClient.getTopicCleanupPolicy(tableName), equalTo("compact")); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. here and elsewhere we should be checking equivalence to |
||
} | ||
|
||
@Test | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -37,10 +37,12 @@ | |
import io.confluent.ksql.ddl.DdlConfig; | ||
import io.confluent.ksql.function.FunctionRegistry; | ||
import io.confluent.ksql.metastore.KsqlStream; | ||
import io.confluent.ksql.metastore.KsqlTable; | ||
import io.confluent.ksql.metastore.KsqlTopic; | ||
import io.confluent.ksql.metastore.MetastoreUtil; | ||
import io.confluent.ksql.serde.json.KsqlJsonTopicSerDe; | ||
import io.confluent.ksql.structured.SchemaKStream; | ||
import io.confluent.ksql.structured.SchemaKTable; | ||
import io.confluent.ksql.util.KafkaTopicClient; | ||
import io.confluent.ksql.util.KsqlConfig; | ||
|
||
|
@@ -189,4 +191,47 @@ private SchemaKStream buildStream() { | |
new HashMap<>(), new MockSchemaRegistryClient()); | ||
} | ||
|
||
@Test | ||
public void shouldCreateSinkWithCorrectCleanupPolicy() { | ||
KafkaTopicClient topicClientForNonWindowTable = EasyMock.createNiceMock(KafkaTopicClient.class); | ||
final Map<String, Object> props = new HashMap<>(); | ||
props.put(KsqlConfig.SINK_NUMBER_OF_PARTITIONS_PROPERTY, 4); | ||
props.put(KsqlConfig.SINK_NUMBER_OF_REPLICAS_PROPERTY, (short)3); | ||
|
||
StructuredDataSourceNode tableSourceNode = new StructuredDataSourceNode( | ||
new PlanNodeId("0"), | ||
new KsqlTable("sqlExpression", "datasource", | ||
schema, | ||
schema.field("key"), | ||
schema.field("timestamp"), | ||
new KsqlTopic("input", "input", | ||
new KsqlJsonTopicSerDe()), | ||
"TableStateStore", | ||
false), | ||
schema); | ||
|
||
outputNode = new KsqlStructuredDataOutputNode(new PlanNodeId("0"), | ||
tableSourceNode, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. indentation. |
||
schema, | ||
schema.field("timestamp"), | ||
schema.field("key"), | ||
new KsqlTopic("output", "output", new KsqlJsonTopicSerDe()), | ||
"output", | ||
props, | ||
Optional.empty()); | ||
|
||
StreamsBuilder streamsBuilder = new StreamsBuilder(); | ||
SchemaKTable schemaKTable = (SchemaKTable) outputNode.buildStream(streamsBuilder, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We should first have a check |
||
ksqlConfig, | ||
topicClientForNonWindowTable, | ||
new MetastoreUtil(), | ||
new FunctionRegistry(), | ||
new HashMap<>(), | ||
new MockSchemaRegistryClient()); | ||
topicClientForNonWindowTable.createTopic(eq("output"), anyInt(), anyShort(), eq(true)); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Would be better to have There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shouldn't this plus the next 2 lines come before buildStream? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, indeed this needed more changes. |
||
EasyMock.expectLastCall(); | ||
EasyMock.replay(topicClientForNonWindowTable); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This seems off. We should be doing: topicClientForNonWindowTable.createTopic(...)
Easymock.expectLastCall();
EasyMock.replay(topicClientForNonWindownTable);
outputNode.buildStream(...);
EasyMock.verify(topicClientForNonWindowTable); This way we are actually checking that the right call was made to the topic client when trying to create the sink topic. Also, we should add tests which check that the compaction policy is delete for streams and windowed tables. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Made changes and added tests for both windowed table and stream. |
||
|
||
} | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -84,7 +84,7 @@ public void testCreateTopic() { | |
} | ||
|
||
@Test | ||
public void shouldUseExistingTopicWIthTheSameSpecsInsteadOfCreate() { | ||
public void shouldUseExistingTopicWithTheSameSpecsInsteadOfCreate() { | ||
AdminClient adminClient = mock(AdminClient.class); | ||
expect(adminClient.describeCluster()).andReturn(getDescribeClusterResult()); | ||
expect(adminClient.listTopics()).andReturn(getListTopicsResult()); | ||
|
@@ -320,11 +320,8 @@ private DescribeClusterResult getDescribeClusterResult() { | |
private DescribeConfigsResult getDescribeConfigsResult(boolean isCompacted) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why this change? This variable doesn't seem to be used at all and really doesn't need to be changed relative to the other changes in this PR. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, it was left from previous changes. Removed it. |
||
DescribeConfigsResult describeConfigsResult = mock(DescribeConfigsResult.class); | ||
ConfigEntry configEntryDeleteEnable = new ConfigEntry("delete.topic.enable", "true"); | ||
ConfigEntry configEntryIsCompacted = new ConfigEntry("cleanup.policy", | ||
isCompacted? "compact": "delete"); | ||
List<ConfigEntry> configEntries = new ArrayList<>(); | ||
configEntries.add(configEntryDeleteEnable); | ||
configEntries.add(configEntryIsCompacted); | ||
Map<ConfigResource, Config> config = new HashMap<>(); | ||
config.put(new ConfigResource(ConfigResource.Type.BROKER, "1"), new Config(configEntries)); | ||
expect(describeConfigsResult.all()).andReturn(KafkaFuture.completedFuture(config)); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a synchronous call. It blocks until it gets a successful response or will throw an exception if it didn't get a successful response after 5 retries.