Skip to content

Commit

Permalink
adding UTs and addressing review comments to
Browse files Browse the repository at this point in the history
firehoseV2 addition to Realtime[Manager|Plumber],
essential segment metadata persist support,
kafka-simple-consumer-firehose extension patch
  • Loading branch information
himanshug committed Aug 28, 2015
1 parent 2237a8c commit 2e0dd1d
Show file tree
Hide file tree
Showing 40 changed files with 1,072 additions and 1,045 deletions.
37 changes: 20 additions & 17 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,13 +64,13 @@ 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();
return new String(readBytes(in, length), UTF8);
return StringUtils.fromUtf8(readBytes(in, length));
}

public byte[] readBytes(ByteBuffer in, int length) throws IOException
Expand Down
8 changes: 4 additions & 4 deletions docs/content/development/kafka-simple-consumer-firehose.md
Original file line number Diff line number Diff line change
Expand Up @@ -2,14 +2,14 @@
layout: doc_page
---
# KafkaSimpleConsumerFirehose
This firehose acts as a Kafka simple consumer and ingests data from Kafka, currently still in experimental section.
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,
"resetBehavior":"latest",
"resetOffsetToEarliest":"true",
"partitionIdList" : ["0"],
"clientId" : "localclient",
"feed": "wikipedia"
Expand All @@ -21,10 +21,10 @@ The configuration for KafkaSimpleConsumerFirehose is similar to the KafkaFirehos
|type|kafka-0.8-v2|yes|
|brokerList|list of the kafka brokers|yes|
|queueBufferLength|the buffer length for kafka message queue|no default(20000)|
|resetBehavior|in case of kafkaOffsetOutOfRange error happens, consumer should starts from the earliest or latest message available|no default(earliest)|
|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 topic, keeping one partition can work properly, but more partition could be added if higher throughput is required.
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.

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
6 changes: 5 additions & 1 deletion extensions/kafka-eight-simpleConsumer/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.8.0-SNAPSHOT</version>
<version>0.8.2-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>

Expand All @@ -36,6 +36,10 @@
<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>
Expand Down
Original file line number Diff line number Diff line change
@@ -1,32 +1,33 @@
/*
* 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.firehose.kafka;

import java.util.List;

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
Expand All @@ -35,8 +36,8 @@ public List<? extends Module> getJacksonModules()
return ImmutableList.of(
new SimpleModule("KafkaEightSimpleConsumerFirehoseModule").registerSubtypes(
new NamedType(KafkaEightSimpleConsumerFirehoseFactory.class, "kafka-0.8-v2")
)
);
)
);
}

@Override
Expand Down
Loading

0 comments on commit 2e0dd1d

Please sign in to comment.