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

Send and receive kafka headers #749

Open
wants to merge 2 commits 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
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,15 @@

package io.confluent.kafkarest;

import io.confluent.kafkarest.entities.ForwardHeader;
import io.confluent.kafkarest.entities.ProduceRecord;
import java.util.Collection;
import java.util.List;
import java.util.stream.Collectors;

import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.header.Header;

/**
* Wrapper producer for content types which have no associated schema (e.g. binary or JSON).
Expand All @@ -43,8 +48,17 @@ public void produce(
if (recordPartition == null) {
recordPartition = record.getPartition();
}
List<Header> headers = null;
if (record.getHeaders() != null && record.getHeaders().size() > 0) {
headers = record
.getHeaders()
.stream()
.filter(m -> m.value != null && m.value.length > 0)
.map(ForwardHeader::toHeader)
.collect(Collectors.toList());
}
producer.send(
new ProducerRecord<>(topic, recordPartition, record.getKey(), record.getValue()),
new ProducerRecord<>(topic, recordPartition, record.getKey(), record.getValue(), headers),
task.createCallback()
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,17 +22,22 @@
import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDe;
import io.confluent.kafkarest.converters.ConversionException;
import io.confluent.kafkarest.converters.SchemaConverter;
import io.confluent.kafkarest.entities.ForwardHeader;
import io.confluent.kafkarest.entities.ProduceRecord;
import io.confluent.kafkarest.entities.ProduceRequest;
import io.confluent.rest.exceptions.RestException;
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;

import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.header.Header;

public class SchemaRestProducer implements RestProducer<JsonNode, JsonNode> {

Expand Down Expand Up @@ -131,7 +136,16 @@ public void produce(
if (recordPartition == null) {
recordPartition = record.getPartition();
}
kafkaRecords.add(new ProducerRecord(topic, recordPartition, key, value));
List<Header> headers = null;
if (record.getHeaders() != null && record.getHeaders().size() > 0) {
headers = record
.getHeaders()
.stream()
.filter(m -> m.value != null && m.value.length > 0)
.map(ForwardHeader::toHeader)
.collect(Collectors.toList());
}
kafkaRecords.add(new ProducerRecord(topic, recordPartition, key, value, headers));
}
} catch (ConversionException e) {
throw Errors.jsonConversionException(e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,9 @@
package io.confluent.kafkarest.entities;

import com.google.auto.value.AutoValue;

import javax.annotation.Nullable;
import java.util.List;

@AutoValue
public abstract class ConsumerRecord<K, V> {
Expand All @@ -36,8 +38,13 @@ public abstract class ConsumerRecord<K, V> {

public abstract long getOffset();

@Nullable
public abstract List<ForwardHeader> getHeaders();

public static <K, V> ConsumerRecord<K, V> create(
String topic, @Nullable K key, @Nullable V value, int partition, long offset) {
return new AutoValue_ConsumerRecord<>(topic, key, value, partition, offset);
String topic,
@Nullable K key, @Nullable V value, int partition, long offset,
@Nullable List<ForwardHeader> headers) {
return new AutoValue_ConsumerRecord<>(topic, key, value, partition, offset, headers);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
/*
* Copyright 2018 Confluent Inc.
*
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.kafkarest.entities;

import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
import com.fasterxml.jackson.databind.ser.std.StdSerializer;
import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.internals.RecordHeader;

import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Objects;

@JsonSerialize(using = ForwardHeader.HeaderSerializer.class)
@JsonDeserialize(using = ForwardHeader.HeaderDeserializer.class)
public final class ForwardHeader {
private static final String NULL_KEY_MESSAGE = "Null header keys are not permitted";
public final String key;
public final byte[] value;

public ForwardHeader(Header header) {
this(header.key(), header.value());
}

public ForwardHeader(String key, byte[] value) {
this.key = Objects.requireNonNull(key, NULL_KEY_MESSAGE);
this.value = value;
}

public ForwardHeader(String key, String value) {
this(key, Objects.requireNonNull(
value, "Null header string value").getBytes(StandardCharsets.UTF_8));
}

public Header toHeader() {
return new RecordHeader(key, value);
}

protected static final class HeaderDeserializer extends StdDeserializer<ForwardHeader> {

public HeaderDeserializer() {
super(ForwardHeader.class);
}

protected HeaderDeserializer(Class<?> vc) {
super(vc);
}

@Override
public ForwardHeader deserialize(JsonParser p, DeserializationContext ctx) throws IOException {
String key = p.nextFieldName();
String value = p.nextTextValue();
//noinspection StatementWithEmptyBody
while (p.nextToken() != JsonToken.END_OBJECT) {
}
if (value != null) {
return new ForwardHeader(key, value.getBytes(StandardCharsets.UTF_8));
}
return null;
}
}

protected static final class HeaderSerializer extends StdSerializer<ForwardHeader> {

public HeaderSerializer() {
super(ForwardHeader.class);
}

protected HeaderSerializer(Class<ForwardHeader> t) {
super(t);
}

@Override
public void serialize(ForwardHeader value, JsonGenerator gen, SerializerProvider provider)
throws IOException {
if (value != null && value.value != null) {
gen.writeStartObject();
gen.writeStringField(value.key, new String(value.value, StandardCharsets.UTF_8));
gen.writeEndObject();
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

import com.google.auto.value.AutoValue;
import javax.annotation.Nullable;
import java.util.List;

@AutoValue
public abstract class ProduceRecord<K, V> {
Expand All @@ -33,8 +34,12 @@ public abstract class ProduceRecord<K, V> {
@Nullable
public abstract Integer getPartition();

@Nullable
public abstract List<ForwardHeader> getHeaders();

public static <K, V> ProduceRecord<K, V> create(
@Nullable K key, @Nullable V value, @Nullable Integer partition) {
return new AutoValue_ProduceRecord<>(key, value, partition);
@Nullable K key, @Nullable V value, @Nullable Integer partition,
@Nullable List<ForwardHeader> headers) {
return new AutoValue_ProduceRecord<>(key, value, partition, headers);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,11 @@
import com.google.protobuf.ByteString;
import io.confluent.kafkarest.entities.ConsumerRecord;
import io.confluent.kafkarest.entities.EntityUtils;
import java.util.Arrays;
import io.confluent.kafkarest.entities.ForwardHeader;

import java.util.List;
import java.util.Objects;
import java.util.Arrays;
import java.util.StringJoiner;
import javax.annotation.Nullable;
import javax.validation.constraints.NotNull;
Expand All @@ -39,6 +42,9 @@ public final class BinaryConsumerRecord {
@Nullable
private final byte[] value;

@Nullable
private final List<ForwardHeader> headers;

@PositiveOrZero
@Nullable
private final Integer partition;
Expand All @@ -53,12 +59,14 @@ private BinaryConsumerRecord(
@JsonProperty("key") @Nullable byte[] key,
@JsonProperty("value") @Nullable byte[] value,
@JsonProperty("partition") @Nullable Integer partition,
@JsonProperty("offset") @Nullable Long offset) {
@JsonProperty("offset") @Nullable Long offset,
@JsonProperty("headers") @Nullable List<ForwardHeader> headers) {
this.topic = topic;
this.key = key;
this.value = value;
this.partition = partition;
this.offset = offset;
this.headers = headers;
}

@JsonProperty
Expand All @@ -79,6 +87,12 @@ public String getValue() {
return value != null ? EntityUtils.encodeBase64Binary(value) : null;
}

@JsonProperty
@Nullable
public List<ForwardHeader> getHeaders() {
return headers;
}

@JsonProperty
@Nullable
public Integer getPartition() {
Expand All @@ -104,7 +118,8 @@ public static BinaryConsumerRecord fromConsumerRecord(
record.getKey() != null ? record.getKey().toByteArray() : null,
record.getValue() != null ? record.getValue().toByteArray() : null,
record.getPartition(),
record.getOffset());
record.getOffset(),
record.getHeaders());
}

public ConsumerRecord<ByteString, ByteString> toConsumerRecord() {
Expand All @@ -122,7 +137,7 @@ public ConsumerRecord<ByteString, ByteString> toConsumerRecord() {
key != null ? ByteString.copyFrom(key) : null,
value != null ? ByteString.copyFrom(value) : null,
partition,
offset);
offset, headers);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,11 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.confluent.kafkarest.entities.EntityUtils;
import io.confluent.kafkarest.entities.ForwardHeader;
import io.confluent.kafkarest.entities.ProduceRecord;
import io.confluent.kafkarest.entities.ProduceRequest;
import io.confluent.rest.validation.ConstraintViolations;

import java.util.Arrays;
import java.util.List;
import java.util.Objects;
Expand Down Expand Up @@ -70,7 +72,7 @@ public ProduceRequest<byte[], byte[]> toProduceRequest() {
}
return ProduceRequest.create(
records.stream()
.map(record -> ProduceRecord.create(record.key, record.value, null))
.map(record -> ProduceRecord.create(record.key, record.value, null, record.headers))
.collect(Collectors.toList()),
/* keySchema= */ null,
/* keySchemaId= */ null,
Expand Down Expand Up @@ -111,10 +113,14 @@ public static final class BinaryPartitionProduceRecord {
@Nullable
private final byte[] value;

@Nullable
private final List<ForwardHeader> headers;

@JsonCreator
public BinaryPartitionProduceRecord(
@JsonProperty("key") @Nullable String key,
@JsonProperty("value") @Nullable String value
@JsonProperty("value") @Nullable String value,
@JsonProperty("headers") @Nullable List<ForwardHeader> headers
) {
try {
this.key = (key != null) ? EntityUtils.parseBase64Binary(key) : null;
Expand All @@ -126,6 +132,7 @@ public BinaryPartitionProduceRecord(
} catch (IllegalArgumentException e) {
throw ConstraintViolations.simpleException("Record value contains invalid base64 encoding");
}
this.headers = headers;
}

@JsonProperty("key")
Expand All @@ -140,6 +147,12 @@ public String getValue() {
return (value == null ? null : EntityUtils.encodeBase64Binary(value));
}

@JsonProperty
@Nullable
public List<ForwardHeader> getHeaders() {
return headers;
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand Down
Loading