Skip to content

Commit

Permalink
KafkaIndexTask.
Browse files Browse the repository at this point in the history
Reads a specific offset range from specific partitions, and can use dataSource metadata
transactions to guarantee exactly-once ingestion.

Each task has a finite lifecycle, so it is expected that some process will be supervising
existing tasks and creating new ones when needed.
  • Loading branch information
gianm committed Mar 11, 2016
1 parent 187569e commit f22fb2c
Show file tree
Hide file tree
Showing 13 changed files with 2,674 additions and 0 deletions.
2 changes: 2 additions & 0 deletions distribution/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,8 @@
<argument>-c</argument>
<argument>io.druid.extensions:druid-kafka-extraction-namespace</argument>
<argument>-c</argument>
<argument>io.druid.extensions:druid-kafka-indexing-service</argument>
<argument>-c</argument>
<argument>io.druid.extensions:mysql-metadata-storage</argument>
<argument>-c</argument>
<argument>io.druid.extensions:druid-namespace-lookup</argument>
Expand Down
91 changes: 91 additions & 0 deletions extensions-core/kafka-indexing-service/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->

<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-indexing-service</artifactId>
<name>druid-kafka-indexing-service</name>
<description>druid-kafka-indexing-service</description>

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

<dependencies>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-api</artifactId>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-indexing-service</artifactId>
<version>0.9.0-SNAPSHOT</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>0.9.0.0</version>
</dependency>

<!-- Tests -->
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka_2.11</artifactId>
<version>0.9.0.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-server</artifactId>
<version>0.9.0-SNAPSHOT</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-indexing-service</artifactId>
<version>0.9.0-SNAPSHOT</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.curator</groupId>
<artifactId>curator-test</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* 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.indexing.kafka;

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Maps;
import com.metamx.common.IAE;
import io.druid.indexing.overlord.DataSourceMetadata;

import java.util.Map;
import java.util.Objects;

public class KafkaDataSourceMetadata implements DataSourceMetadata
{
private final KafkaPartitions kafkaPartitions;

@JsonCreator
public KafkaDataSourceMetadata(
@JsonProperty("partitions") KafkaPartitions kafkaPartitions
)
{
this.kafkaPartitions = kafkaPartitions;
}

@JsonProperty("partitions")
public KafkaPartitions getKafkaPartitions()
{
return kafkaPartitions;
}

@Override
public boolean isValidStart()
{
return true;
}

@Override
public boolean matches(DataSourceMetadata other)
{
if (getClass() != other.getClass()) {
return false;
}

return plus(other).equals(other.plus(this));
}

@Override
public DataSourceMetadata plus(DataSourceMetadata other)
{
if (!(other instanceof KafkaDataSourceMetadata)) {
throw new IAE(
"Expected instance of %s, got %s",
KafkaDataSourceMetadata.class.getCanonicalName(),
other.getClass().getCanonicalName()
);
}

final KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) other;

if (that.getKafkaPartitions().getTopic().equals(kafkaPartitions.getTopic())) {
// Same topic, merge offsets.
final Map<Integer, Long> newMap = Maps.newHashMap();

for (Map.Entry<Integer, Long> entry : kafkaPartitions.getPartitionOffsetMap().entrySet()) {
newMap.put(entry.getKey(), entry.getValue());
}

for (Map.Entry<Integer, Long> entry : that.getKafkaPartitions().getPartitionOffsetMap().entrySet()) {
newMap.put(entry.getKey(), entry.getValue());
}

return new KafkaDataSourceMetadata(new KafkaPartitions(kafkaPartitions.getTopic(), newMap));
} else {
// Different topic, prefer "other".
return other;
}
}

@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) o;
return Objects.equals(kafkaPartitions, that.kafkaPartitions);
}

@Override
public int hashCode()
{
return Objects.hash(kafkaPartitions);
}

@Override
public String toString()
{
return "KafkaDataSourceMetadata{" +
"kafkaPartitions=" + kafkaPartitions +
'}';
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* 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.indexing.kafka;

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import io.druid.segment.indexing.IOConfig;

import java.util.Map;

public class KafkaIOConfig implements IOConfig
{
private static final boolean DEFAULT_USE_TRANSACTION = true;

private final String sequenceName;
private final KafkaPartitions startPartitions;
private final KafkaPartitions endPartitions;
private final Map<String, String> consumerProperties;
private final boolean useTransaction;

@JsonCreator
public KafkaIOConfig(
@JsonProperty("sequenceName") String sequenceName,
@JsonProperty("startPartitions") KafkaPartitions startPartitions,
@JsonProperty("endPartitions") KafkaPartitions endPartitions,
@JsonProperty("consumerProperties") Map<String, String> consumerProperties,
@JsonProperty("useTransaction") Boolean useTransaction
)
{
this.sequenceName = Preconditions.checkNotNull(sequenceName, "sequenceName");
this.startPartitions = Preconditions.checkNotNull(startPartitions, "startPartitions");
this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions");
this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION;

Preconditions.checkArgument(
startPartitions.getTopic().equals(endPartitions.getTopic()),
"start topic and end topic must match"
);

Preconditions.checkArgument(
startPartitions.getPartitionOffsetMap().keySet().equals(endPartitions.getPartitionOffsetMap().keySet()),
"start partition set and end partition set must match"
);

for (int partition : endPartitions.getPartitionOffsetMap().keySet()) {
Preconditions.checkArgument(
endPartitions.getPartitionOffsetMap().get(partition) >= startPartitions.getPartitionOffsetMap()
.get(partition),
"end offset must be >= start offset for partition[%d]",
partition
);
}
}

@JsonProperty
public String getSequenceName()
{
return sequenceName;
}

@JsonProperty
public KafkaPartitions getStartPartitions()
{
return startPartitions;
}

@JsonProperty
public KafkaPartitions getEndPartitions()
{
return endPartitions;
}

@JsonProperty
public Map<String, String> getConsumerProperties()
{
return consumerProperties;
}

@JsonProperty
public boolean isUseTransaction()
{
return useTransaction;
}

@Override
public String toString()
{
return "KafkaIOConfig{" +
"sequenceName='" + sequenceName + '\'' +
", startPartitions=" + startPartitions +
", endPartitions=" + endPartitions +
", consumerProperties=" + consumerProperties +
", useTransaction=" + useTransaction +
'}';
}
}
Loading

0 comments on commit f22fb2c

Please sign in to comment.