-
Notifications
You must be signed in to change notification settings - Fork 329
SAMZA-1921: upgrade to use the latest java AdminClient. #789
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
Changes from all commits
a31a7aa
410ce78
d4620d6
bbffb79
010fa16
5e6f5fb
06b1ac3
1ad58d4
dd39d08
67e611e
0edf343
88f8559
78ad578
7887d88
afb34d9
57fca52
7f7b559
8ab04b2
add733b
728dc18
952dbbe
927adff
ceebdc3
4f0a746
cdd811d
f6966a8
450ea2e
5f5d7a3
058217e
7821072
74dd138
92180fd
712e68a
824e19a
f1f4eba
1e40147
abb215d
d56feee
de5385e
b3b4183
8964955
57bf054
353f903
ee95637
15f62c4
11c4dba
21a16a3
7d60510
8410445
70c6f2b
fcb7839
bc7def0
83c2a85
31474da
7ba6061
6f6cf92
e40adda
84062e1
b23f31b
3b30f86
8aa77a1
1016240
f554c32
d57a878
c7d94c8
1fe59fd
e2bbb0a
bdc69ae
1445e5d
3b5b3dc
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 |
|---|---|---|
|
|
@@ -27,21 +27,29 @@ | |
| import java.util.Map; | ||
| import java.util.Properties; | ||
| import java.util.Set; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.function.Supplier; | ||
| import java.util.stream.Collectors; | ||
| import kafka.admin.AdminClient; | ||
| import kafka.utils.ZkUtils; | ||
| import org.apache.commons.lang3.NotImplementedException; | ||
| import org.apache.commons.lang3.StringUtils; | ||
| import org.apache.kafka.clients.admin.AdminClient; | ||
| import org.apache.kafka.clients.admin.CreateTopicsResult; | ||
| import org.apache.kafka.clients.admin.DeleteTopicsResult; | ||
| import org.apache.kafka.clients.admin.DescribeTopicsResult; | ||
| import org.apache.kafka.clients.admin.NewTopic; | ||
| import org.apache.kafka.clients.admin.TopicDescription; | ||
| import org.apache.kafka.clients.consumer.Consumer; | ||
| import org.apache.kafka.clients.consumer.ConsumerConfig; | ||
| import org.apache.kafka.common.PartitionInfo; | ||
| import org.apache.kafka.common.TopicPartition; | ||
| import org.apache.kafka.common.errors.TopicExistsException; | ||
| import org.apache.samza.Partition; | ||
| import org.apache.samza.SamzaException; | ||
| import org.apache.samza.config.Config; | ||
| import org.apache.samza.config.KafkaConfig; | ||
| import org.apache.samza.config.MapConfig; | ||
| import org.apache.samza.config.SystemConfig; | ||
| import org.apache.samza.system.StreamSpec; | ||
| import org.apache.samza.system.StreamValidationException; | ||
|
|
@@ -73,16 +81,17 @@ public class KafkaSystemAdmin implements SystemAdmin { | |
| protected static final long DEFAULT_EXPONENTIAL_SLEEP_MAX_DELAY_MS = 10000; | ||
| protected static final int MAX_RETRIES_ON_EXCEPTION = 5; | ||
| protected static final int DEFAULT_REPL_FACTOR = 2; | ||
| private static final int KAFKA_ADMIN_OPS_TIMEOUT_MS = 50000; | ||
|
|
||
| // used in TestRepartitionJoinWindowApp TODO - remove SAMZA-1945 | ||
| @VisibleForTesting | ||
| public static volatile boolean deleteMessageCalled = false; | ||
|
|
||
| protected final String systemName; | ||
| protected final Consumer metadataConsumer; | ||
| protected final Config config; | ||
| protected final Config config; | ||
|
|
||
| protected AdminClient adminClient = null; | ||
| protected kafka.admin.AdminClient adminClientForDelete = null; | ||
|
|
||
| // Custom properties to create a new coordinator stream. | ||
| private final Properties coordinatorStreamProperties; | ||
|
|
@@ -99,6 +108,9 @@ public class KafkaSystemAdmin implements SystemAdmin { | |
| // used for intermediate streams | ||
| protected final boolean deleteCommittedMessages; | ||
|
|
||
| // admin client for create/remove topics | ||
| final AdminClient adminClient; | ||
|
|
||
| private final AtomicBoolean stopped = new AtomicBoolean(false); | ||
|
|
||
| public KafkaSystemAdmin(String systemName, Config config, Consumer metadataConsumer) { | ||
|
|
@@ -111,6 +123,10 @@ public KafkaSystemAdmin(String systemName, Config config, Consumer metadataConsu | |
| } | ||
| this.metadataConsumer = metadataConsumer; | ||
|
|
||
| Properties props = createAdminClientProperties(); | ||
| LOG.info("New admin client with props:" + props); | ||
|
Contributor
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 do we need to use two different adminClient?
Contributor
Author
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. Because deleteMessages() is only provided by the old kafka.admin.AdminClient. We will get rid of it when we move to the new Kafka (1.0) |
||
| adminClient = AdminClient.create(props); | ||
|
|
||
| KafkaConfig kafkaConfig = new KafkaConfig(config); | ||
| coordinatorStreamReplicationFactor = Integer.valueOf(kafkaConfig.getCoordinatorReplicationFactor()); | ||
| coordinatorStreamProperties = KafkaSystemAdminUtilsScala.getCoordinatorTopicProperties(kafkaConfig); | ||
|
|
@@ -167,13 +183,17 @@ public void stop() { | |
| LOG.warn("metadataConsumer.close for system " + systemName + " failed with exception.", e); | ||
| } | ||
| } | ||
| if (adminClient != null) { | ||
| if (adminClientForDelete != null) { | ||
| try { | ||
| adminClient.close(); | ||
| adminClientForDelete.close(); | ||
| } catch (Exception e) { | ||
| LOG.warn("adminClient.close for system " + systemName + " failed with exception.", e); | ||
| LOG.warn("AdminClient.close() for system {} failed with exception {}.", systemName, e); | ||
| } | ||
| } | ||
|
|
||
| if (adminClient != null) { | ||
| adminClient.close(); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -481,18 +501,62 @@ public Integer offsetComparator(String offset1, String offset2) { | |
| @Override | ||
| public boolean createStream(StreamSpec streamSpec) { | ||
| LOG.info("Creating Kafka topic: {} on system: {}", streamSpec.getPhysicalName(), streamSpec.getSystemName()); | ||
| final String REPL_FACTOR = "replication.factor"; | ||
|
Contributor
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: Would be better to move these static final constants to beginning of the class.
Contributor
Author
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. Usually yes, but this is kind of unfortunate hack, and it should be used only in this method. So I'd prefer to keep it here. |
||
|
|
||
| KafkaStreamSpec kSpec = toKafkaSpec(streamSpec); | ||
| String topicName = kSpec.getPhysicalName(); | ||
|
|
||
| // create topic. | ||
| NewTopic newTopic = new NewTopic(topicName, kSpec.getPartitionCount(), (short) kSpec.getReplicationFactor()); | ||
|
|
||
| // specify the configs | ||
| Map<String, String> streamConfig = new HashMap(streamSpec.getConfig()); | ||
| // HACK - replication.factor is invalid config for AdminClient.createTopics | ||
| if (streamConfig.containsKey(REPL_FACTOR)) { | ||
| String repl = streamConfig.get(REPL_FACTOR); | ||
| LOG.warn("Configuration {}={} for topic={} is invalid. Using kSpec repl factor {}", | ||
| REPL_FACTOR, repl, kSpec.getPhysicalName(), kSpec.getReplicationFactor()); | ||
| streamConfig.remove(REPL_FACTOR); | ||
|
Contributor
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 preserve the replicationFactor value defined in the configuration.
Contributor
Author
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 pass it in the spec. |
||
| } | ||
| newTopic.configs(new MapConfig(streamConfig)); | ||
| CreateTopicsResult result = adminClient.createTopics(ImmutableSet.of(newTopic)); | ||
| try { | ||
| result.all().get(KAFKA_ADMIN_OPS_TIMEOUT_MS, TimeUnit.MILLISECONDS); | ||
| } catch (Exception e) { | ||
| if (e instanceof TopicExistsException || e.getCause() instanceof TopicExistsException) { | ||
| LOG.info("Topic {} already exists.", topicName); | ||
| return false; | ||
| } | ||
|
|
||
| return KafkaSystemAdminUtilsScala.createStream(toKafkaSpec(streamSpec), getZkConnection()); | ||
| throw new SamzaException(String.format("Creation of topic %s failed.", topicName), e); | ||
| } | ||
| LOG.info("Successfully created topic {}", topicName); | ||
| DescribeTopicsResult desc = adminClient.describeTopics(ImmutableSet.of(topicName)); | ||
| try { | ||
| TopicDescription td = desc.all().get(KAFKA_ADMIN_OPS_TIMEOUT_MS, TimeUnit.MILLISECONDS).get(topicName); | ||
| LOG.info("Topic {} created with {}", topicName, td); | ||
| return true; | ||
| } catch (Exception e) { | ||
| LOG.error("'Describe after create' failed for topic " + topicName, e); | ||
| return false; | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public boolean clearStream(StreamSpec streamSpec) { | ||
| LOG.info("Creating Kafka topic: {} on system: {}", streamSpec.getPhysicalName(), streamSpec.getSystemName()); | ||
|
|
||
| KafkaSystemAdminUtilsScala.clearStream(streamSpec, getZkConnection()); | ||
| String topicName = streamSpec.getPhysicalName(); | ||
|
|
||
| Map<String, List<PartitionInfo>> topicsMetadata = getTopicMetadata(ImmutableSet.of(streamSpec.getPhysicalName())); | ||
| return topicsMetadata.get(streamSpec.getPhysicalName()).isEmpty(); | ||
| try { | ||
| DeleteTopicsResult deleteTopicsResult = adminClient.deleteTopics(ImmutableSet.of(topicName)); | ||
| deleteTopicsResult.all().get(KAFKA_ADMIN_OPS_TIMEOUT_MS, TimeUnit.MILLISECONDS); | ||
| } catch (Exception e) { | ||
| LOG.error("Failed to delete topic {} with exception {}.", topicName, e); | ||
| return false; | ||
| } | ||
|
|
||
| return true; | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -566,10 +630,10 @@ Map<String, List<PartitionInfo>> getTopicMetadata(Set<String> topics) { | |
| @Override | ||
| public void deleteMessages(Map<SystemStreamPartition, String> offsets) { | ||
| if (deleteCommittedMessages) { | ||
| if (adminClient == null) { | ||
| adminClient = AdminClient.create(createAdminClientProperties()); | ||
| if (adminClientForDelete == null) { | ||
| adminClientForDelete = kafka.admin.AdminClient.create(createAdminClientProperties()); | ||
| } | ||
| KafkaSystemAdminUtilsScala.deleteMessages(adminClient, offsets); | ||
| KafkaSystemAdminUtilsScala.deleteMessages(adminClientForDelete, offsets); | ||
| deleteMessageCalled = true; | ||
| } | ||
| } | ||
|
|
@@ -594,7 +658,6 @@ protected Properties createAdminClientProperties() { | |
| } | ||
| props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); | ||
|
|
||
|
|
||
| // kafka.admin.AdminUtils requires zkConnect | ||
| // this will change after we move to the new org.apache..AdminClient | ||
| String zkConnect = | ||
|
|
||
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.
50 seconds? Isn't that too high?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.
I don't think user need to configure it. But I think you are right - it is too high. I will change it to 10 seconds (in case brokers are loaded), to match today's max delay.