mirror of https://github.com/apache/druid.git
Removed hard-coded Kafka Deserializer in Web-UI Kafka data import such that users can supply a custom deserializer in the UI as well as in hand-built ingestion specs. (#8364)
This commit is contained in:
parent
aa51668ea7
commit
a4d1219184
|
@ -31,7 +31,6 @@ import org.apache.druid.indexing.overlord.sampler.FirehoseSampler;
|
|||
import org.apache.druid.indexing.overlord.sampler.SamplerConfig;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamSamplerSpec;
|
||||
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
|
||||
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
@ -77,8 +76,6 @@ public class KafkaSamplerSpec extends SeekableStreamSamplerSpec
|
|||
|
||||
props.put("enable.auto.commit", "false");
|
||||
props.put("auto.offset.reset", "none");
|
||||
props.put("key.deserializer", ByteArrayDeserializer.class.getName());
|
||||
props.put("value.deserializer", ByteArrayDeserializer.class.getName());
|
||||
props.put("request.timeout.ms", Integer.toString(samplerConfig.getTimeoutMs()));
|
||||
|
||||
return new KafkaRecordSupplier(props, objectMapper);
|
||||
|
|
Loading…
Reference in New Issue