Skip to content
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

Experimental kafa simple consumer based firehose #1609

Merged
merged 2 commits into from
Aug 28, 2015
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
46 changes: 27 additions & 19 deletions common/src/main/java/io/druid/common/utils/SerializerUtils.java
Original file line number Diff line number Diff line change
@@ -1,25 +1,28 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package io.druid.common.utils;

import com.google.common.io.ByteStreams;
import com.google.common.io.OutputSupplier;
import com.google.common.primitives.Ints;
import com.metamx.common.StringUtils;
import io.druid.collections.IntList;

import java.io.IOException;
Expand Down Expand Up @@ -61,15 +64,20 @@ public String readString(InputStream in) throws IOException
final int length = readInt(in);
byte[] stringBytes = new byte[length];
ByteStreams.readFully(in, stringBytes);
return new String(stringBytes, UTF8);
return StringUtils.fromUtf8(stringBytes);
}

public String readString(ByteBuffer in) throws IOException
{
final int length = in.getInt();
byte[] stringBytes = new byte[length];
in.get(stringBytes);
return new String(stringBytes, UTF8);
return StringUtils.fromUtf8(readBytes(in, length));
}

public byte[] readBytes(ByteBuffer in, int length) throws IOException
{
byte[] bytes = new byte[length];
in.get(bytes);
return bytes;
}

public void writeStrings(OutputStream out, String[] names) throws IOException
Expand Down
2 changes: 1 addition & 1 deletion docs/content/development/experimental.md
Original file line number Diff line number Diff line change
Expand Up @@ -11,4 +11,4 @@ To enable experimental features, include their artifacts in the configuration ru
druid.extensions.coordinates=["io.druid.extensions:druid-histogram:{VERSION}"]
```

The configuration for all the indexer and query nodes need to be updated with this.
The configuration for all the indexer and query nodes need to be updated with this.
30 changes: 30 additions & 0 deletions docs/content/development/kafka-simple-consumer-firehose.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
---
layout: doc_page
---
# KafkaSimpleConsumerFirehose
This is an experimental firehose to ingest data from kafka using kafka simple consumer api. Currently, this firehose would only work inside standalone realtime nodes.
The configuration for KafkaSimpleConsumerFirehose is similar to the KafkaFirehose [Kafka firehose example](realtime-ingestion.html#realtime-specfile), except `firehose` should be replaced with `firehoseV2` like this:
```json
"firehoseV2": {
"type" : "kafka-0.8-v2",
"brokerList" : ["localhost:4443"],
"queueBufferLength":10001,
"resetOffsetToEarliest":"true",
"partitionIdList" : ["0"],
"clientId" : "localclient",
"feed": "wikipedia"
}
```

|property|description|required?|
|--------|-----------|---------|
|type|kafka-0.8-v2|yes|
|brokerList|list of the kafka brokers|yes|
|queueBufferLength|the buffer length for kafka message queue|no default(20000)|
|resetOffsetToEarliest|in case of kafkaOffsetOutOfRange error happens, consumer should starts from the earliest or latest message available|true|
|partitionIdList|list of kafka partition ids|yes|
|clientId|the clientId for kafka SimpleConsumer|yes|
|feed|kafka topic|yes|

For using this firehose at scale and possibly in production, it is recommended to set replication factor to at least three, which means at least three Kafka brokers in the `brokerList`. For a 1*10^4 events per second kafka topic, keeping one partition can work properly, but more partitions could be added if higher throughput is required.

3 changes: 2 additions & 1 deletion docs/content/ingestion/firehose.md
Original file line number Diff line number Diff line change
Expand Up @@ -260,7 +260,6 @@ When using this firehose, events can be sent by submitting a POST request to the
|serviceName|name used to announce the event receiver service endpoint|yes|
|bufferSize| size of buffer used by firehose to store events|no default(100000)|


#### TimedShutoffFirehose

This can be used to start a firehose that will shut down at a specified time.
Expand All @@ -283,3 +282,5 @@ An example is shown below:
|type|This should be "timed"|yes|
|shutoffTime|time at which the firehose should shut down, in ISO8601 format|yes|
|delegate|firehose to use|yes|
=======

1 change: 1 addition & 0 deletions docs/content/toc.textile
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@ h2. Development
** "Select Query":../development/select-query.html
** "Approximate Histograms and Quantiles":../development/approximate-histograms.html
** "Router node":../development/router.html
** "New Kafka Firehose":../development/kafka-simple-consumer-firehose.html

h2. Misc
* "Papers & Talks":../misc/papers-and-talks.html
Expand Down
81 changes: 81 additions & 0 deletions extensions/kafka-eight-simpleConsumer/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ Druid - a distributed column store.
~ Copyright 2012 - 2015 Metamarkets Group Inc.
~
~ Licensed under the Apache License, Version 2.0 (the "License");
~ you may not use this file except in compliance with the License.
~ You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing, software
~ distributed under the License is distributed on an "AS IS" BASIS,
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->

<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<groupId>io.druid.extensions</groupId>
<artifactId>druid-kafka-eight-simple-consumer</artifactId>
<name>druid-kafka-eight-simple-consumer</name>
<description>druid-kafka-eight-simple-consumer</description>

<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.8.2-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>

<dependencies>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-api</artifactId>
</dependency>
<dependency>
<groupId>com.metamx</groupId>
<artifactId>emitter</artifactId>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka_2.10</artifactId>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(not blocking for this PR) we may need to consider better ways to handle the "different scala version require different artifact IDs" thing.

<version>0.8.2.1</version>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
</exclusion>
</exclusions>
</dependency>
<!-- Tests -->
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<archive>
<manifest>
<addDefaultImplementationEntries>true</addDefaultImplementationEntries>
<addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
</manifest>
</archive>
</configuration>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package io.druid.firehose.kafka;

import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import io.druid.initialization.DruidModule;

import java.util.List;

public class KafkaEightSimpleConsumerDruidModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(
new SimpleModule("KafkaEightSimpleConsumerFirehoseModule").registerSubtypes(
new NamedType(KafkaEightSimpleConsumerFirehoseFactory.class, "kafka-0.8-v2")
)
);
}

@Override
public void configure(Binder binder)
{

}
}
Loading