Remove hard limitation that druid(after 0.15.0) only can consume Kafka version 0.11.x or better#10551
Conversation
|
@jihoonson Hi Jihoonson, sorry for bothering. Could you please help to review my code?Thanks! |
| props.put("group.id", StringUtils.format("kafka-supervisor-%s", IdUtils.getRandomId())); | ||
| props.put("auto.offset.reset", "none"); | ||
| props.put("enable.auto.commit", "false"); | ||
| props.put("isolation.level", "read_committed"); |
There was a problem hiding this comment.
I think to remove isolation.level here and leaves the configuration to users changes the current behavior.
Currently users care nothing about this kafka configuration when using a higher version of kafka such as 0.11. By removing it, they need to set this property in supervisor spec, or the default value, which is read_uncommitted, will be applied, which may be not what they expect.
If this is the root cause that limits the Druid to use Kafka lower than 0.11, I think maybe we can introduce another property, as the same way as pollTimeout property does, then we can unset isolation.level property according to the value of this new property. Only those who want to use Druid with older kafka need to set this new property.
There was a problem hiding this comment.
Yes, It will change the current behavior.
You are right, most people really don't care this Kafka configuration(Let it be default behavior). As I know the default logic of the Kafka Producer is not to enable transactions feature and the same as Kafka Consumer. So that maybe Druid Kafka indexing service keep the same default logic is more reasonable.
Furthermore, if a Druid user want Druid to consume transactional Kafka topics, I think it is more reasonable to let users set this parameter in consumerProperties like 'bootstrap.servers' because he knows what he is going to do and why.
By the way, Druid from 0.15 to better can't consume old version Kafka is really confused me(I believe it's not just me). Generally speaking, higher Kafka consumer client is able to consume old version Kafka cluster unless it involves high-version-specific api. And this hard limitation block us to upgrade Druid cluster from 0.14.2 for a long time. If we don't remove this config, we have to upgrade PRD Kafka cluster, which is a much more heavy work to do because there are too many consumers of Kafka such as Spark and Flink. Orz...
Thanks for reviewing!
There was a problem hiding this comment.
We don't know whether or not the users have enabled transactional message in their clusters, so changing the current behavior may cause backward compatibility problem.
There was a problem hiding this comment.
Make sense! I add a new property named consumeTransactionally in consumerProperties, which is a Kafka consumer level property to control isolation.level.
If users don't set consumeTransactionally or set consumeTransactionally true, druid will consume Kafka Transactionally by default.
Set consumeTransactionally false here can disable druid to consume Kafka Transactionally through unset isolation.level property, which means druid can consume lower version of Kafka now like 0.10.x
In this way, we don't change current Druid default behavior and provide a way to let druid consume lower version Kafka.
There was a problem hiding this comment.
@FrankChen021 Hi Frank, sorry to bother you. I have tested this change in our Druid cluster recently. What should I do next ?
There was a problem hiding this comment.
Putting this property in KafkaIndexTaskIOConfig.consumerProperties might cause some confusion. consumerProperties is designed to store kafka official properties. It's better to put the new property in KafkaIndexTaskIOConfig directly and put the code related to this property in KafkaRecordSupplier.addConsumerPropertiesFromConfig
There was a problem hiding this comment.
Hi FrankChen021, I have put the new property consumeTransactionally in the KafkaSupervisorIOConfig directly. If users ignore this config or set this config true, Druid would consume Kafka transactionally same as current Druid behavior.
Set consumeTransactionally false here can disable Druid to consume Kafka transactionally through unset isolation.level property, meanwhile Druid could consume lower version of Kafka now like 0.10.2.1 .
In this way, we don't change current Druid default behavior and provide a way to a way to make Druid have the ability to consume different versions of Kafka and remove the hard limitation that Druid(after 0.15.0) only can consume Kafka version 0.11.x or better.
I have tested this PR in our Dev Druid cluster. Please take a look.
4d6bded to
885985a
Compare
| |`lateMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps earlier than this period before the task was created; for example if this is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps earlier than *2016-01-01T11:00Z* will be dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments (e.g. a realtime and a nightly batch ingestion pipeline). Please note that only one of `lateMessageRejectionPeriod` or `lateMessageRejectionStartDateTime` can be specified.|no (default == none)| | ||
| |`earlyMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps later than this period after the task reached its taskDuration; for example if this is set to `PT1H`, the taskDuration is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps later than *2016-01-01T14:00Z* will be dropped. **Note:** Tasks sometimes run past their task duration, for example, in cases of supervisor failover. Setting earlyMessageRejectionPeriod too low may cause messages to be dropped unexpectedly whenever a task runs past its originally configured task duration.|no (default == none)| | ||
| |`consumeTransactionally`|Boolean|Set `consumeTransactionally` false here can disable druid to consume Kafka in a transactional way. And druid could consume lower version of Kafka now, such as 0.10.2.1 |no (default == true)| | ||
|
|
There was a problem hiding this comment.
At the beginning of this doc, there's paragraph that describes the version of kafka that are supported. Since this PR
provides a way to support those kafka clusters lower than 0.11, we should also make some changes to the doc
The Kafka indexing service supports transactional topics which were introduced in Kafka 0.11.x. These changes make the
Kafka consumer that Druid uses incompatible with older brokers. Ensure that your Kafka brokers are version 0.11.x or > Kafka consumer that Druid uses incompatible with older brokers. Ensure that your Kafka brokers are version 0.11.x or
better before using this functionality. Refer Kafka upgrade guide > better before using this functionality. Refer Kafka upgrade guide
if you are using older version of Kafka brokers. > if you are using older version of Kafka brokers.
| import org.apache.kafka.common.serialization.Deserializer; | ||
|
|
||
| import javax.annotation.Nonnull; | ||
|
|
There was a problem hiding this comment.
please revert this unnecessary change
| final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); | ||
| Collection workItems = new ArrayList<>(); | ||
| workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); | ||
|
|
There was a problem hiding this comment.
please revert this unnecessary change
| workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); | ||
|
|
||
| workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); | ||
|
|
There was a problem hiding this comment.
please revert this unnecessary change
|
Hi, @jihoonson would you like to take a look at this change ? |
|
Do |
|
Presently, the read_committed isolation level supports transactional producers as well as preserves the behavior for non-transactional producers. If this property is made user-configurable, it is possible that a cluster operator may unintentionally disable it for higher versions of Kafka that support transactional topics. Are there any issues that may arise if we disable this property for higher versions of Kafka? If so, I think we should have some sort of logging that makes the operator aware of it. Also with non-transactional producers in lower versions of Kafka, it may be reasonable to expect that offsets are sequential. Since we no longer have the offset gap check in Druid, the docs may need to make it clear that Druid doesn't perform this check. |
|
@zhangyue19921010 I think making Druid flexible to work with older versions of Kafka is good, so +1 on that. regarding the specific implementation here -- does that make sense? |
|
@himanshug Thanks for your review.
No problem will happen . When set I just change the strategy for setting
If users ignore this parameter, Druid will set This solution makes code changes lighter, safer and no need to add new parameters. This solution also does not change current behavior of Kafka consumption and provide a way to make Druid has the ability to consume lower versions of Kafka. All the changes have been tested. xxx__dev__010__committed supervisor got errors as expected while others work fine. Here is the error message in Overlord : |
According to above code, consumerProperties(from KafkaSupervisorIOConfig) will be overwritten by consumerConfigs which is set in KafkaConsumerConfigs class. |
|
Hi @pphust
I also considered the feasibility of this solution before. What I am concerned is that some configs such as |
| { | ||
|
|
||
| public static Map<String, Object> getConsumerProperties() | ||
| public static Map<String, Object> getConsumerProperties(Map<String, Object> customerConsumerProperties) |
There was a problem hiding this comment.
it would likely be simpler to leave this alone, remove the line props.put("isolation.level".. altogether
and put something like props.put("isolation.level", customerConsumerProperties.getOrDefault("isolation.level", "read_committed")); right after https://github.com/apache/druid/blob/master/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java#L78
There was a problem hiding this comment.
Hi @himanshug Thanks for your review!
I agree with it would likely be simpler to leave this alone, remove the line props.put("isolation.level".. altogether. But I think maybe it is not very appropriate to modify https://github.com/apache/druid/blob/master/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java#L78
I have tried to do changes in KafkaSupervisorIOConfig.java as you said (cc59d8b)
this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
consumerProperties.putIfAbsent("isolation.level", "read_committed");
Preconditions.checkNotNull(
and functional testing is fine while UT is failed at
These failure means changes in KafkaSupervisorIOConfig may cause compatibility issues when deserialize from old IoConfig to new IoConfig or deserialize from new IoConfig to old IoConfig. In other words, we need to ensure that the deserialize between new IoConfig and old IoConfig is completely consistent and smooth.
So I switched to a safer way in the latest commit.
All the changes is tested on Dev cluster.
PTAL :)
There was a problem hiding this comment.
we need to ensure that the deserialize between new IoConfig and old IoConfig is completely consistent and smooth
well, my main concern was the consumeTransactionally param which has been removed . Though I fail to see why simply adding the default to KafkaSupervisorIOConfig.java would not have same behavior functionally (or for backward compatibility as well)... but I am not so worried about the specifics at this point as it is easy to change later if needed.
…21010/druid into modify-kafka-Version
|
+1 after the build |
|
@FrankChen021 @nishantmonu51 @a2l007 @himanshug @pphust Thanks for your review! |
…a version 0.11.x or better (apache#10551) * remove build in kafka consumer config : * modify druid docs of kafka indexing service * yuezhang * modify doc * modify docs * fix kafkaindexTaskTest.java * revert uncessary change * add more logs and modify docs * revert jdk version * modify docs * modify-kafka-version v2 * modify docs * modify docs * modify docs * modify docs * modify docs * done * remove useless import * change code and add UT Co-authored-by: yuezhang <yuezhang@freewheel.tv>

Fixes #8279
Description
In our PRD Environment, we deployed two versions of Kafka, which are 0.10.2.1 and 2.4.1. And open source druid(from 0.15.0 to 0.20.0) only can support to consume messages from Kafka 0.11.x or better. And when consume Kafka 0.10.2.1, it throws UnsupportedVersionException :
And as the Druid docs says
The root cause of this exception (SDK 2.6.0 + Kafka 0.10.2.1) is Hard-coded ‘props.put("isolation.level", "read_committed");’ in the ‘KafkaConsumerConfigs’ which is unnecessary.
This PR adds a new config named
consumeTransactionallyinKafkaIndexTaskIOConfig. Default is true, druid can consume Kafka topics consumeTransactionally same as current Druid behavior. And users can set this config false. SetconsumeTransactionallyfalse here can disable druid to consume Kafka in a transactional way. And druid could consume lower version of Kafka now, such as 0.10.2.1, like{ "type": "kafka", "dataSchema": { "dataSource": "metrics-kafka", "timestampSpec": { "column": "timestamp", "format": "auto" }, "dimensionsSpec": { "dimensions": [], "dimensionExclusions": [ "timestamp", "value" ] }, "metricsSpec": [ { "name": "count", "type": "count" }, { "name": "value_sum", "fieldName": "value", "type": "doubleSum" }, { "name": "value_min", "fieldName": "value", "type": "doubleMin" }, { "name": "value_max", "fieldName": "value", "type": "doubleMax" } ], "granularitySpec": { "type": "uniform", "segmentGranularity": "HOUR", "queryGranularity": "NONE" } }, "ioConfig": { "topic": "metrics", "inputFormat": { "type": "json" }, "consumerProperties": { "bootstrap.servers": "localhost:9092" }, "consumeTransactionally": false, "taskCount": 1, "replicas": 1, "taskDuration": "PT1H" }, "tuningConfig": { "type": "kafka", "maxRowsPerSegment": 5000000 } }Enable this feature, Druid can both consumes Kafka 2.4.1 and Kafka 0.10.2.1 through different configurations of supervisors.
Here are logs of overlord service:
Ignore consumeTransactionally or set it true
Set consumeTransactionally false
This PR has:
Key changed/added classes in this PR
KafkaConsumerConfigs