Skip to content

Commit

Permalink
Add support for multiple topic in avro format
Browse files Browse the repository at this point in the history
  • Loading branch information
LiuHongliang committed Aug 4, 2016
1 parent 63eb64e commit 8782e66
Show file tree
Hide file tree
Showing 4 changed files with 31 additions and 10 deletions.
1 change: 1 addition & 0 deletions docs/kafka.md
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ These Kafka-specific properties, if used, may be specified either at the global
|`topicPattern`|A regular expression used to match Kafka topics to dataSource configurations. See "Matching Topics to Data Sources" for details.|{match nothing}, must be provided|
|`topicPattern.priority`|If multiple topicPatterns match the same topic name, the highest priority dataSource configuration will be used. A higher number indicates a higher priority. See "Matching Topics to Data Sources" for details.|1|
|`useTopicAsDataSource`|Use the Kafka topic as the dataSource name instead of the one provided in the configuration file. Useful when combined with a topicPattern that matches more than one Kafka topic. See "Matching Topics to Data Sources" for details.|false|
|`useInputTopicAsDecodeTopic`|Use the topic which used by kafka consumer as the avro stream decoder's topic. Useful when consume multiply topic with different avro schema.|true|
|`reportDropsAsExceptions`|Whether or not dropped messages will cause an exception and terminate the application.|false|

### Running
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,10 @@ public PropertiesBasedKafkaConfig()
@Default("false")
public abstract Boolean useTopicAsDataSource();

@Config("useInputTopicAsDecodeTopic")
@Default("false")
public abstract Boolean useInputTopicAsDecodeTopic();

@Config("topicPattern.priority")
@Default("1")
public abstract Integer getTopicPatternPriority();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,10 +60,14 @@ public TranquilityEventWriter(
FinagleRegistry finagleRegistry
)
{
this.dataSourceConfig = dataSourceConfig;
if(dataSourceConfig.propertiesBasedConfig().useInputTopicAsDecodeTopic()){
this.dataSourceConfig = KafkaBeamUtils.useInputTopicAsDecodeTopic(topic, dataSourceConfig);
} else {
this.dataSourceConfig = dataSourceConfig;
}
this.tranquilizer = KafkaBeamUtils.createTranquilizer(
topic,
dataSourceConfig,
this.dataSourceConfig,
curator,
finagleRegistry
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package com.metamx.tranquility.kafka

import com.metamx.common.scala.untyped.Dict
import com.metamx.tranquility.config.DataSourceConfig
import com.metamx.tranquility.druid.DruidBeams
import com.metamx.tranquility.druid.DruidLocation
Expand All @@ -28,15 +29,13 @@ import com.metamx.tranquility.tranquilizer.Tranquilizer
import org.apache.curator.framework.CuratorFramework
import scala.reflect.runtime.universe.typeTag

object KafkaBeamUtils
{
object KafkaBeamUtils {
def createTranquilizer(
topic: String,
config: DataSourceConfig[PropertiesBasedKafkaConfig],
curator: CuratorFramework,
finagleRegistry: FinagleRegistry
): Tranquilizer[Array[Byte]] =
{
topic: String,
config: DataSourceConfig[PropertiesBasedKafkaConfig],
curator: CuratorFramework,
finagleRegistry: FinagleRegistry
): Tranquilizer[Array[Byte]] = {
DruidBeams.fromConfig(config, typeTag[Array[Byte]])
.location(
DruidLocation.create(
Expand All @@ -48,4 +47,17 @@ object KafkaBeamUtils
.finagleRegistry(finagleRegistry)
.buildTranquilizer(config.tranquilizerBuilder())
}

def useInputTopicAsDecodeTopic(topic: String, config: DataSourceConfig[PropertiesBasedKafkaConfig]): DataSourceConfig[PropertiesBasedKafkaConfig] = {
val dataSchema = config.specMap.get("dataSchema").get.asInstanceOf[Dict]
val parser = dataSchema.get("parser").get.asInstanceOf[Dict]
if ("avro_stream".equals(parser.get("type").toString)) {
val avroBytesDecoder = parser.get("avroBytesDecoder").get.asInstanceOf[Dict]
val subjectAndIdConverter = avroBytesDecoder.get("subjectAndIdConverter").get.asInstanceOf[Dict]
val map = config.specMap.updated("dataSchema", dataSchema.updated("parser", parser.updated("avroBytesDecoder", avroBytesDecoder.updated("subjectAndIdConverter", subjectAndIdConverter.updated("topic", topic)))))
config.copy(config.dataSource, config.propertiesBasedConfig, map)
} else {
config
}
}
}

0 comments on commit 8782e66

Please sign in to comment.