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

Add kafka header #666

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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
1 change: 1 addition & 0 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,7 @@ project(':datastream-common') {
compile "com.linkedin.pegasus:restli-server:$pegasusVersion"
compile "com.intellij:annotations:$intellijAnnotationsVersion"
compile "com.google.guava:guava:$guavaVersion"
compile "org.apache.kafka:kafka-clients:$kafkaVersion"
testCompile "org.mockito:mockito-core:$mockitoVersion"
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@

import org.apache.avro.reflect.Nullable;
import org.apache.commons.lang.Validate;
import org.apache.kafka.common.header.Headers;


/**
Expand All @@ -29,14 +30,16 @@ public class BrooklinEnvelope {

private Map<String, String> _metadata;

private Headers _headers;

/**
* Construct a BrooklinEnvelope using record key, value, and metadata
* @param key The record key (e.g. primary key)
* @param value The new record value
* @param metadata Additional metadata to associate with the change event
*/
public BrooklinEnvelope(Object key, Object value, Map<String, String> metadata) {
this(key, value, null, metadata);
this(key, value, null, metadata, null);
}

/**
Expand All @@ -45,14 +48,28 @@ public BrooklinEnvelope(Object key, Object value, Map<String, String> metadata)
* @param previousValue The old record value
* @param value The new record value
* @param metadata Additional metadata to associate with the change event
* @param headers Kafka Headers
*/
public BrooklinEnvelope(@Nullable Object key, @Nullable Object value, @Nullable Object previousValue,
Map<String, String> metadata) {
Map<String, String> metadata, Headers headers) {
Validate.notNull(metadata, "metadata cannot be null");
setKey(key);
setValue(value);
setPreviousValue(previousValue);
setMetadata(metadata);
setHeaders(headers);
}

/**
* Construct a BrooklinEnvelope using record key, value, and metadata
* @param key The record key (e.g. primary key)
* @param previousValue The old record value
* @param value The new record value
* @param metadata Additional metadata to associate with the change event
*/
public BrooklinEnvelope(@Nullable Object key, @Nullable Object value, @Nullable Object previousValue,
Map<String, String> metadata) {
this(key, value, previousValue, metadata, null);
}

/**
Expand Down Expand Up @@ -129,6 +146,21 @@ public void setMetadata(Map<String, String> metadata) {
_metadata = metadata;
}


/**
* Get the kafka headers
*/
public Headers getHeaders() {
return _headers;
}

/**
* Set the kafka headers
*/
public void setHeaders(Headers headers) {
_headers = headers;
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand All @@ -139,17 +171,17 @@ public boolean equals(Object o) {
}
BrooklinEnvelope task = (BrooklinEnvelope) o;
return Objects.equals(_previousValue, task._previousValue) && Objects.equals(_key, task._key) && Objects.equals(
_value, task._value) && Objects.equals(_metadata, task._metadata);
_value, task._value) && Objects.equals(_metadata, task._metadata) && Objects.equals(_headers, task._headers);
}

@Override
public int hashCode() {
return Objects.hash(_key, _value, _previousValue, _metadata);
return Objects.hash(_key, _value, _previousValue, _metadata, _headers);
}

@Override
public String toString() {
return String.format("Key:(%s), Value:(%s), PreviousValue:(%s), Metadata=(%s)", _key, _value, _previousValue,
_metadata);
return String.format("Key:(%s), Value:(%s), PreviousValue:(%s), Metadata=(%s), Headers=(%s)", _key, _value, _previousValue,
_metadata, _headers);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -144,7 +144,7 @@ protected DatastreamProducerRecord translate(ConsumerRecord<?, ?> fromKafka, Ins
eventsSourceTimestamp = fromKafka.timestamp();
}

BrooklinEnvelope envelope = new BrooklinEnvelope(fromKafka.key(), fromKafka.value(), null, metadata);
BrooklinEnvelope envelope = new BrooklinEnvelope(fromKafka.key(), fromKafka.value(), null, metadata, fromKafka.headers());
DatastreamProducerRecordBuilder builder = new DatastreamProducerRecordBuilder();
builder.addEvent(envelope);
builder.setEventsSourceTimestamp(eventsSourceTimestamp);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -228,7 +228,7 @@ protected DatastreamProducerRecord translate(ConsumerRecord<?, ?> fromKafka, Ins
String offsetStr = String.valueOf(offset);
metadata.put(KAFKA_ORIGIN_OFFSET, offsetStr);
metadata.put(BrooklinEnvelopeMetadataConstants.EVENT_TIMESTAMP, String.valueOf(eventsSourceTimestamp));
BrooklinEnvelope envelope = new BrooklinEnvelope(fromKafka.key(), fromKafka.value(), null, metadata);
BrooklinEnvelope envelope = new BrooklinEnvelope(fromKafka.key(), fromKafka.value(), null, metadata, fromKafka.headers());
DatastreamProducerRecordBuilder builder = new DatastreamProducerRecordBuilder();
builder.addEvent(envelope);
builder.setEventsSourceTimestamp(eventsSourceTimestamp);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.header.Headers;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -99,6 +100,7 @@ private ProducerRecord<byte[], byte[]> convertToProducerRecord(String topicName,

byte[] keyValue = new byte[0];
byte[] payloadValue = new byte[0];
Headers recordHeader = null;
if (event instanceof BrooklinEnvelope) {
BrooklinEnvelope envelope = (BrooklinEnvelope) event;
if (envelope.key().isPresent() && envelope.key().get() instanceof byte[]) {
Expand All @@ -108,19 +110,23 @@ private ProducerRecord<byte[], byte[]> convertToProducerRecord(String topicName,
if (envelope.value().isPresent() && envelope.value().get() instanceof byte[]) {
payloadValue = (byte[]) envelope.value().get();
}

if (envelope.getHeaders() != null) {
recordHeader = envelope.getHeaders();
}
} else if (event instanceof byte[]) {
payloadValue = (byte[]) event;
}

if (partition.isPresent() && partition.get() >= 0) {
// If the partition is specified. We send the record to the specific partition
return new ProducerRecord<>(topicName, partition.get(), keyValue, payloadValue);
return new ProducerRecord<>(topicName, partition.get(), keyValue, payloadValue, recordHeader);
} else {
// If the partition is not specified. We use the partitionKey as the key. Kafka will use the hash of that
// to determine the partition. If partitionKey does not exist, use the key value.
keyValue = record.getPartitionKey().isPresent()
? record.getPartitionKey().get().getBytes(StandardCharsets.UTF_8) : null;
return new ProducerRecord<>(topicName, keyValue, payloadValue);
return new ProducerRecord<>(topicName, null, keyValue, payloadValue, recordHeader);
}
}

Expand Down