Skip to content
Snippets Groups Projects
Commit 4c48942f authored by Randall Hauch's avatar Randall Hauch Committed by Ewen Cheslack-Postava
Browse files

KAFKA-5142: Add Connect support for message headers (KIP-145)

**[KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect) has been accepted, and this PR implements KIP-145 except without the SMTs.**

Changed the Connect API and runtime to support message headers as described in [KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect).

The new `Header` interface defines an immutable representation of a Kafka header (key-value pair) with support for the Connect value types and schemas. This interface provides methods for easily converting between many of the built-in primitive, structured, and logical data types.

The new `Headers` interface defines an ordered collection of headers and is used to track all headers associated with a `ConnectRecord` (and thus `SourceRecord` and `SinkRecord`). This does allow multiple headers with the same key. The `Headers` contains methods for adding, removing, finding, and modifying headers. Convenience methods allow connectors and transforms to easily use and modify the headers for a record.

A new `HeaderConverter` interface is also defined to enable the Connect runtime framework to be able to serialize and deserialize headers between the in-memory representation and Kafka’s byte[] representation. A new `SimpleHeaderConverter` implementation has been added, and this serializes to strings and deserializes by inferring the schemas (`Struct` header values are serialized without the schemas, so they can only be deserialized as `Map` instances without a schema.) The `StringConverter`, `JsonConverter`, and `ByteArrayConverter` have all been extended to also be `HeaderConverter` implementations. Each connector can be configured with a different header converter, although by default the `SimpleHeaderConverter` is used to serialize header values as strings without schemas.

Unit and integration tests are added for `ConnectHeader` and `ConnectHeaders`, the two implementation classes for headers. Additional test methods are added for the methods added to the `Converter` implementations. Finally, the `ConnectRecord` object is already used heavily, so only limited tests need to be added while quite a few of the existing tests already cover the changes.

Author: Randall Hauch <rhauch@gmail.com>

Reviewers: Arjun Satish <arjun@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4319 from rhauch/kafka-5142-b
parent 530bc59d
No related branches found
No related tags found
No related merge requests found
Showing
with 3705 additions and 21 deletions
......@@ -277,6 +277,7 @@
<allow pkg="org.apache.kafka.common" />
<allow pkg="org.apache.kafka.connect.data" />
<allow pkg="org.apache.kafka.connect.errors" />
<allow pkg="org.apache.kafka.connect.header" />
<allow pkg="org.apache.kafka.clients" />
<allow pkg="org.apache.kafka.test"/>
......
......@@ -57,7 +57,7 @@
files="AbstractRequest.java|KerberosLogin.java|WorkerSinkTaskTest.java"/>
<suppress checks="NPathComplexity"
files="(BufferPool|MetricName|Node|ConfigDef|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|Agent).java"/>
files="(BufferPool|MetricName|Node|ConfigDef|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|Agent|Values).java"/>
<!-- clients tests -->
<suppress checks="ClassDataAbstractionCoupling"
......@@ -102,9 +102,13 @@
files="DistributedHerder.java"/>
<suppress checks="CyclomaticComplexity"
files="KafkaConfigBackingStore.java"/>
<suppress checks="CyclomaticComplexity"
files="(Values|ConnectHeader|ConnectHeaders).java"/>
<suppress checks="JavaNCSS"
files="KafkaConfigBackingStore.java"/>
<suppress checks="JavaNCSS"
files="Values.java"/>
<suppress checks="NPathComplexity"
files="ConnectRecord.java"/>
......@@ -116,6 +120,10 @@
files="JsonConverter.java"/>
<suppress checks="NPathComplexity"
files="DistributedHerder.java"/>
<suppress checks="NPathComplexity"
files="ConnectHeaders.java"/>
<suppress checks="MethodLength"
files="Values.java"/>
<!-- connect tests-->
<suppress checks="ClassDataAbstractionCoupling"
......
......@@ -17,6 +17,11 @@
package org.apache.kafka.connect.connector;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.header.ConnectHeaders;
import org.apache.kafka.connect.header.Header;
import org.apache.kafka.connect.header.Headers;
import java.util.Objects;
/**
* <p>
......@@ -34,11 +39,19 @@ public abstract class ConnectRecord<R extends ConnectRecord<R>> {
private final Schema valueSchema;
private final Object value;
private final Long timestamp;
private final Headers headers;
public ConnectRecord(String topic, Integer kafkaPartition,
Schema keySchema, Object key,
Schema valueSchema, Object value,
Long timestamp) {
this(topic, kafkaPartition, keySchema, key, valueSchema, value, timestamp, new ConnectHeaders());
}
public ConnectRecord(String topic, Integer kafkaPartition,
Schema keySchema, Object key,
Schema valueSchema, Object value,
Long timestamp, Iterable<Header> headers) {
this.topic = topic;
this.kafkaPartition = kafkaPartition;
this.keySchema = keySchema;
......@@ -46,6 +59,11 @@ public abstract class ConnectRecord<R extends ConnectRecord<R>> {
this.valueSchema = valueSchema;
this.value = value;
this.timestamp = timestamp;
if (headers instanceof ConnectHeaders) {
this.headers = (ConnectHeaders) headers;
} else {
this.headers = new ConnectHeaders(headers);
}
}
public String topic() {
......@@ -76,9 +94,46 @@ public abstract class ConnectRecord<R extends ConnectRecord<R>> {
return timestamp;
}
/** Generate a new record of the same type as itself, with the specified parameter values. **/
/**
* Get the headers for this record.
*
* @return the headers; never null
*/
public Headers headers() {
return headers;
}
/**
* Create a new record of the same type as itself, with the specified parameter values. All other fields in this record will be copied
* over to the new record. Since the headers are mutable, the resulting record will have a copy of this record's headers.
*
* @param topic the name of the topic; may be null
* @param kafkaPartition the partition number for the Kafka topic; may be null
* @param keySchema the schema for the key; may be null
* @param key the key; may be null
* @param valueSchema the schema for the value; may be null
* @param value the value; may be null
* @param timestamp the timestamp; may be null
* @return the new record
*/
public abstract R newRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp);
/**
* Create a new record of the same type as itself, with the specified parameter values. All other fields in this record will be copied
* over to the new record.
*
* @param topic the name of the topic; may be null
* @param kafkaPartition the partition number for the Kafka topic; may be null
* @param keySchema the schema for the key; may be null
* @param key the key; may be null
* @param valueSchema the schema for the value; may be null
* @param value the value; may be null
* @param timestamp the timestamp; may be null
* @param headers the headers; may be null or empty
* @return the new record
*/
public abstract R newRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp, Iterable<Header> headers);
@Override
public String toString() {
return "ConnectRecord{" +
......@@ -87,6 +142,7 @@ public abstract class ConnectRecord<R extends ConnectRecord<R>> {
", key=" + key +
", value=" + value +
", timestamp=" + timestamp +
", headers=" + headers +
'}';
}
......@@ -113,6 +169,8 @@ public abstract class ConnectRecord<R extends ConnectRecord<R>> {
return false;
if (timestamp != null ? !timestamp.equals(that.timestamp) : that.timestamp != null)
return false;
if (!Objects.equals(headers, that.headers))
return false;
return true;
}
......@@ -126,6 +184,7 @@ public abstract class ConnectRecord<R extends ConnectRecord<R>> {
result = 31 * result + (valueSchema != null ? valueSchema.hashCode() : 0);
result = 31 * result + (value != null ? value.hashCode() : 0);
result = 31 * result + (timestamp != null ? timestamp.hashCode() : 0);
result = 31 * result + headers.hashCode();
return result;
}
}
This diff is collapsed.
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.header;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.Struct;
import java.util.Objects;
/**
* A {@link Header} implementation.
*/
class ConnectHeader implements Header {
private static final SchemaAndValue NULL_SCHEMA_AND_VALUE = new SchemaAndValue(null, null);
private final String key;
private final SchemaAndValue schemaAndValue;
protected ConnectHeader(String key, SchemaAndValue schemaAndValue) {
Objects.requireNonNull(key, "Null header keys are not permitted");
this.key = key;
this.schemaAndValue = schemaAndValue != null ? schemaAndValue : NULL_SCHEMA_AND_VALUE;
assert this.schemaAndValue != null;
}
@Override
public String key() {
return key;
}
@Override
public Object value() {
return schemaAndValue.value();
}
@Override
public Schema schema() {
Schema schema = schemaAndValue.schema();
if (schema == null && value() instanceof Struct) {
schema = ((Struct) value()).schema();
}
return schema;
}
@Override
public Header rename(String key) {
Objects.requireNonNull(key, "Null header keys are not permitted");
if (this.key.equals(key)) {
return this;
}
return new ConnectHeader(key, schemaAndValue);
}
@Override
public Header with(Schema schema, Object value) {
return new ConnectHeader(key, new SchemaAndValue(schema, value));
}
@Override
public int hashCode() {
return Objects.hash(key, schemaAndValue);
}
@Override
public boolean equals(Object obj) {
if (obj == this) {
return true;
}
if (obj instanceof Header) {
Header that = (Header) obj;
return Objects.equals(this.key, that.key()) && Objects.equals(this.schema(), that.schema()) && Objects.equals(this.value(),
that.value());
}
return false;
}
@Override
public String toString() {
return "ConnectHeader(key=" + key + ", value=" + value() + ", schema=" + schema() + ")";
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.header;
import org.apache.kafka.common.utils.AbstractIterator;
import org.apache.kafka.connect.data.Date;
import org.apache.kafka.connect.data.Decimal;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Schema.Type;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.data.Time;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.errors.DataException;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.Set;
/**
* A basic {@link Headers} implementation.
*/
public class ConnectHeaders implements Headers {
private static final int EMPTY_HASH = Objects.hash(new LinkedList<>());
/**
* An immutable and therefore sharable empty iterator.
*/
private static final Iterator<Header> EMPTY_ITERATOR = new Iterator<Header>() {
@Override
public boolean hasNext() {
return false;
}
@Override
public Header next() {
throw new NoSuchElementException();
}
@Override
public void remove() {
throw new IllegalStateException();
}
};
// This field is set lazily, but once set to a list it is never set back to null
private LinkedList<Header> headers;
public ConnectHeaders() {
}
public ConnectHeaders(Iterable<Header> original) {
if (original == null) {
return;
}
if (original instanceof ConnectHeaders) {
ConnectHeaders originalHeaders = (ConnectHeaders) original;
if (!originalHeaders.isEmpty()) {
headers = new LinkedList<>(originalHeaders.headers);
}
} else {
headers = new LinkedList<>();
for (Header header : original) {
headers.add(header);
}
}
}
@Override
public int size() {
return headers == null ? 0 : headers.size();
}
@Override
public boolean isEmpty() {
return headers == null ? true : headers.isEmpty();
}
@Override
public Headers clear() {
if (headers != null) {
headers.clear();
}
return this;
}
@Override
public Headers add(Header header) {
Objects.requireNonNull(header, "Unable to add a null header.");
if (headers == null) {
headers = new LinkedList<>();
}
headers.add(header);
return this;
}
protected Headers addWithoutValidating(String key, Object value, Schema schema) {
return add(new ConnectHeader(key, new SchemaAndValue(schema, value)));
}
@Override
public Headers add(String key, SchemaAndValue schemaAndValue) {
checkSchemaMatches(schemaAndValue);
return add(new ConnectHeader(key, schemaAndValue != null ? schemaAndValue : SchemaAndValue.NULL));
}
@Override
public Headers add(String key, Object value, Schema schema) {
return add(key, value != null || schema != null ? new SchemaAndValue(schema, value) : SchemaAndValue.NULL);
}
@Override
public Headers addString(String key, String value) {
return addWithoutValidating(key, value, value != null ? Schema.STRING_SCHEMA : Schema.OPTIONAL_STRING_SCHEMA);
}
@Override
public Headers addBytes(String key, byte[] value) {
return addWithoutValidating(key, value, value != null ? Schema.BYTES_SCHEMA : Schema.OPTIONAL_BYTES_SCHEMA);
}
@Override
public Headers addBoolean(String key, boolean value) {
return addWithoutValidating(key, value, Schema.BOOLEAN_SCHEMA);
}
@Override
public Headers addByte(String key, byte value) {
return addWithoutValidating(key, value, Schema.INT8_SCHEMA);
}
@Override
public Headers addShort(String key, short value) {
return addWithoutValidating(key, value, Schema.INT16_SCHEMA);
}
@Override
public Headers addInt(String key, int value) {
return addWithoutValidating(key, value, Schema.INT32_SCHEMA);
}
@Override
public Headers addLong(String key, long value) {
return addWithoutValidating(key, value, Schema.INT64_SCHEMA);
}
@Override
public Headers addFloat(String key, float value) {
return addWithoutValidating(key, value, Schema.FLOAT32_SCHEMA);
}
@Override
public Headers addDouble(String key, double value) {
return addWithoutValidating(key, value, Schema.FLOAT64_SCHEMA);
}
@Override
public Headers addList(String key, List<?> value, Schema schema) {
if (value == null) {
return add(key, null, null);
}
checkSchemaType(schema, Type.ARRAY);
return addWithoutValidating(key, value, schema);
}
@Override
public Headers addMap(String key, Map<?, ?> value, Schema schema) {
if (value == null) {
return add(key, null, null);
}
checkSchemaType(schema, Type.MAP);
return addWithoutValidating(key, value, schema);
}
@Override
public Headers addStruct(String key, Struct value) {
if (value == null) {
return add(key, null, null);
}
checkSchemaType(value.schema(), Type.STRUCT);
return addWithoutValidating(key, value, value.schema());
}
@Override
public Headers addDecimal(String key, BigDecimal value) {
if (value == null) {
return add(key, null, null);
}
// Check that this is a decimal ...
Schema schema = Decimal.schema(value.scale());
Decimal.fromLogical(schema, value);
return addWithoutValidating(key, value, schema);
}
@Override
public Headers addDate(String key, java.util.Date value) {
if (value != null) {
// Check that this is a date ...
Date.fromLogical(Date.SCHEMA, value);
}
return addWithoutValidating(key, value, Date.SCHEMA);
}
@Override
public Headers addTime(String key, java.util.Date value) {
if (value != null) {
// Check that this is a time ...
Time.fromLogical(Time.SCHEMA, value);
}
return addWithoutValidating(key, value, Time.SCHEMA);
}
@Override
public Headers addTimestamp(String key, java.util.Date value) {
if (value != null) {
// Check that this is a timestamp ...
Timestamp.fromLogical(Timestamp.SCHEMA, value);
}
return addWithoutValidating(key, value, Timestamp.SCHEMA);
}
@Override
public Header lastWithName(String key) {
checkKey(key);
if (headers != null) {
ListIterator<Header> iter = headers.listIterator(headers.size());
while (iter.hasPrevious()) {
Header header = iter.previous();
if (key.equals(header.key())) {
return header;
}
}
}
return null;
}
@Override
public Iterator<Header> allWithName(String key) {
return new FilterByKeyIterator(iterator(), key);
}
@Override
public Iterator<Header> iterator() {
if (headers != null) {
return headers.iterator();
}
return EMPTY_ITERATOR;
}
@Override
public Headers remove(String key) {
checkKey(key);
if (!headers.isEmpty()) {
Iterator<Header> iterator = iterator();
while (iterator.hasNext()) {
if (iterator.next().key().equals(key)) {
iterator.remove();
}
}
}
return this;
}
@Override
public Headers retainLatest() {
if (!headers.isEmpty()) {
Set<String> keys = new HashSet<>();
ListIterator<Header> iter = headers.listIterator(headers.size());
while (iter.hasPrevious()) {
Header header = iter.previous();
String key = header.key();
if (!keys.add(key)) {
iter.remove();
}
}
}
return this;
}
@Override
public Headers retainLatest(String key) {
checkKey(key);
if (!headers.isEmpty()) {
boolean found = false;
ListIterator<Header> iter = headers.listIterator(headers.size());
while (iter.hasPrevious()) {
String headerKey = iter.previous().key();
if (key.equals(headerKey)) {
if (found)
iter.remove();
found = true;
}
}
}
return this;
}
@Override
public Headers apply(String key, HeaderTransform transform) {
checkKey(key);
if (!headers.isEmpty()) {
ListIterator<Header> iter = headers.listIterator();
while (iter.hasNext()) {
Header orig = iter.next();
if (orig.key().equals(key)) {
Header updated = transform.apply(orig);
if (updated != null) {
iter.set(updated);
} else {
iter.remove();
}
}
}
}
return this;
}
@Override
public Headers apply(HeaderTransform transform) {
if (!headers.isEmpty()) {
ListIterator<Header> iter = headers.listIterator();
while (iter.hasNext()) {
Header orig = iter.next();
Header updated = transform.apply(orig);
if (updated != null) {
iter.set(updated);
} else {
iter.remove();
}
}
}
return this;
}
@Override
public int hashCode() {
return isEmpty() ? EMPTY_HASH : Objects.hash(headers);
}
@Override
public boolean equals(Object obj) {
if (obj == this) {
return true;
}
if (obj instanceof Headers) {
Headers that = (Headers) obj;
Iterator<Header> thisIter = this.iterator();
Iterator<Header> thatIter = that.iterator();
while (thisIter.hasNext() && thatIter.hasNext()) {
if (!Objects.equals(thisIter.next(), thatIter.next()))
return false;
}
return !thisIter.hasNext() && !thatIter.hasNext();
}
return false;
}
@Override
public String toString() {
return "ConnectHeaders(headers=" + (headers != null ? headers : "") + ")";
}
@Override
public ConnectHeaders duplicate() {
return new ConnectHeaders(this);
}
/**
* Check that the key is not null
*
* @param key the key; may not be null
* @throws NullPointerException if the supplied key is null
*/
private void checkKey(String key) {
Objects.requireNonNull(key, "Header key cannot be null");
}
/**
* Check the {@link Schema#type() schema's type} matches the specified type.
*
* @param schema the schema; never null
* @param type the expected type
* @throws DataException if the schema's type does not match the expected type
*/
private void checkSchemaType(Schema schema, Type type) {
if (schema.type() != type) {
throw new DataException("Expecting " + type + " but instead found " + schema.type());
}
}
/**
* Check that the value and its schema are compatible.
*
* @param schemaAndValue the schema and value pair
* @throws DataException if the schema is not compatible with the value
*/
// visible for testing
void checkSchemaMatches(SchemaAndValue schemaAndValue) {
if (schemaAndValue != null) {
Schema schema = schemaAndValue.schema();
if (schema == null)
return;
schema = schema.schema(); // in case a SchemaBuilder is used
Object value = schemaAndValue.value();
if (value == null && !schema.isOptional()) {
throw new DataException("A null value requires an optional schema but was " + schema);
}
if (value != null) {
switch (schema.type()) {
case BYTES:
if (value instanceof ByteBuffer)
return;
if (value instanceof byte[])
return;
if (value instanceof BigDecimal && Decimal.LOGICAL_NAME.equals(schema.name()))
return;
break;
case STRING:
if (value instanceof String)
return;
break;
case BOOLEAN:
if (value instanceof Boolean)
return;
break;
case INT8:
if (value instanceof Byte)
return;
break;
case INT16:
if (value instanceof Short)
return;
break;
case INT32:
if (value instanceof Integer)
return;
if (value instanceof java.util.Date && Date.LOGICAL_NAME.equals(schema.name()))
return;
if (value instanceof java.util.Date && Time.LOGICAL_NAME.equals(schema.name()))
return;
break;
case INT64:
if (value instanceof Long)
return;
if (value instanceof java.util.Date && Timestamp.LOGICAL_NAME.equals(schema.name()))
return;
break;
case FLOAT32:
if (value instanceof Float)
return;
break;
case FLOAT64:
if (value instanceof Double)
return;
break;
case ARRAY:
if (value instanceof List)
return;
break;
case MAP:
if (value instanceof Map)
return;
break;
case STRUCT:
if (value instanceof Struct)
return;
break;
}
throw new DataException("The value " + value + " is not compatible with the schema " + schema);
}
}
}
private static final class FilterByKeyIterator extends AbstractIterator<Header> {
private final Iterator<Header> original;
private final String key;
private FilterByKeyIterator(Iterator<Header> original, String key) {
this.original = original;
this.key = key;
}
protected Header makeNext() {
while (original.hasNext()) {
Header header = original.next();
if (!header.key().equals(key)) {
continue;
}
return header;
}
return this.allDone();
}
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.header;
import org.apache.kafka.connect.data.Schema;
/**
* A {@link Header} is a key-value pair, and multiple headers can be included with the key, value, and timestamp in each Kafka message.
* If the value contains schema information, then the header will have a non-null {@link #schema() schema}.
* <p>
* This is an immutable interface.
*/
public interface Header {
/**
* The header's key, which is not necessarily unique within the set of headers on a Kafka message.
*
* @return the header's key; never null
*/
String key();
/**
* Return the {@link Schema} associated with this header, if there is one. Not all headers will have schemas.
*
* @return the header's schema, or null if no schema is associated with this header
*/
Schema schema();
/**
* Get the header's value as deserialized by Connect's header converter.
*
* @return the deserialized object representation of the header's value; may be null
*/
Object value();
/**
* Return a new {@link Header} object that has the same key but with the supplied value.
*
* @param schema the schema for the new value; may be null
* @param value the new value
* @return the new {@link Header}; never null
*/
Header with(Schema schema, Object value);
/**
* Return a new {@link Header} object that has the same schema and value but with the supplied key.
*
* @param key the key for the new header; may not be null
* @return the new {@link Header}; never null
*/
Header rename(String key);
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.header;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.DataException;
import java.math.BigDecimal;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
* A mutable ordered collection of {@link Header} objects. Note that multiple headers may have the same {@link Header#key() key}.
*/
public interface Headers extends Iterable<Header> {
/**
* Get the number of headers in this object.
*
* @return the number of headers; never negative
*/
int size();
/**
* Determine whether this object has no headers.
*
* @return true if there are no headers, or false if there is at least one header
*/
boolean isEmpty();
/**
* Get the collection of {@link Header} objects whose {@link Header#key() keys} all match the specified key.
*
* @param key the key; may not be null
* @return the iterator over headers with the specified key; may be null if there are no headers with the specified key
*/
Iterator<Header> allWithName(String key);
/**
* Return the last {@link Header} with the specified key.
*
* @param key the key for the header; may not be null
* @return the last Header, or null if there are no headers with the specified key
*/
Header lastWithName(String key);
/**
* Add the given {@link Header} to this collection.
*
* @param header the header; may not be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers add(Header header);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param schemaAndValue the {@link SchemaAndValue} for the header; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers add(String key, SchemaAndValue schemaAndValue);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @param schema the schema for the header's value; may not be null if the value is not null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers add(String key, Object value, Schema schema);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addString(String key, String value);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addBoolean(String key, boolean value);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addByte(String key, byte value);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addShort(String key, short value);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addInt(String key, int value);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addLong(String key, long value);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addFloat(String key, float value);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addDouble(String key, double value);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addBytes(String key, byte[] value);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @param schema the schema describing the list value; may not be null
* @return this object to facilitate chaining multiple methods; never null
* @throws DataException if the header's value is invalid
*/
Headers addList(String key, List<?> value, Schema schema);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @param schema the schema describing the map value; may not be null
* @return this object to facilitate chaining multiple methods; never null
* @throws DataException if the header's value is invalid
*/
Headers addMap(String key, Map<?, ?> value, Schema schema);
/**
* Add to this collection a {@link Header} with the given key and value.
*
* @param key the header's key; may not be null
* @param value the header's value; may be null
* @return this object to facilitate chaining multiple methods; never null
* @throws DataException if the header's value is invalid
*/
Headers addStruct(String key, Struct value);
/**
* Add to this collection a {@link Header} with the given key and {@link org.apache.kafka.connect.data.Decimal} value.
*
* @param key the header's key; may not be null
* @param value the header's {@link org.apache.kafka.connect.data.Decimal} value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addDecimal(String key, BigDecimal value);
/**
* Add to this collection a {@link Header} with the given key and {@link org.apache.kafka.connect.data.Date} value.
*
* @param key the header's key; may not be null
* @param value the header's {@link org.apache.kafka.connect.data.Date} value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addDate(String key, java.util.Date value);
/**
* Add to this collection a {@link Header} with the given key and {@link org.apache.kafka.connect.data.Time} value.
*
* @param key the header's key; may not be null
* @param value the header's {@link org.apache.kafka.connect.data.Time} value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addTime(String key, java.util.Date value);
/**
* Add to this collection a {@link Header} with the given key and {@link org.apache.kafka.connect.data.Timestamp} value.
*
* @param key the header's key; may not be null
* @param value the header's {@link org.apache.kafka.connect.data.Timestamp} value; may be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers addTimestamp(String key, java.util.Date value);
/**
* Removes all {@link Header} objects whose {@link Header#key() key} matches the specified key.
*
* @param key the key; may not be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers remove(String key);
/**
* Removes all but the latest {@link Header} objects whose {@link Header#key() key} matches the specified key.
*
* @param key the key; may not be null
* @return this object to facilitate chaining multiple methods; never null
*/
Headers retainLatest(String key);
/**
* Removes all but the last {@Header} object with each key.
*
* @return this object to facilitate chaining multiple methods; never null
*/
Headers retainLatest();
/**
* Removes all headers from this object.
*
* @return this object to facilitate chaining multiple methods; never null
*/
Headers clear();
/**
* Create a copy of this {@link Headers} object. The new copy will contain all of the same {@link Header} objects as this object.
* @return the copy; never null
*/
Headers duplicate();
/**
* Get all {@link Header}s, apply the transform to each and store the result in place of the original.
*
* @param transform the transform to apply; may not be null
* @return this object to facilitate chaining multiple methods; never null
* @throws DataException if the header's value is invalid
*/
Headers apply(HeaderTransform transform);
/**
* Get all {@link Header}s with the given key, apply the transform to each and store the result in place of the original.
*
* @param key the header's key; may not be null
* @param transform the transform to apply; may not be null
* @return this object to facilitate chaining multiple methods; never null
* @throws DataException if the header's value is invalid
*/
Headers apply(String key, HeaderTransform transform);
/**
* A function to transform the supplied {@link Header}. Implementations will likely need to use {@link Header#with(Schema, Object)}
* to create the new instance.
*/
interface HeaderTransform {
/**
* Transform the given {@link Header} and return the updated {@link Header}.
*
* @param header the input header; never null
* @return the new header, or null if the supplied {@link Header} is to be removed
*/
Header apply(Header header);
}
}
......@@ -19,6 +19,7 @@ package org.apache.kafka.connect.sink;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.header.Header;
/**
* SinkRecord is a {@link ConnectRecord} that has been read from Kafka and includes the kafkaOffset of
......@@ -38,7 +39,12 @@ public class SinkRecord extends ConnectRecord<SinkRecord> {
public SinkRecord(String topic, int partition, Schema keySchema, Object key, Schema valueSchema, Object value, long kafkaOffset,
Long timestamp, TimestampType timestampType) {
super(topic, partition, keySchema, key, valueSchema, value, timestamp);
this(topic, partition, keySchema, key, valueSchema, value, kafkaOffset, timestamp, timestampType, null);
}
public SinkRecord(String topic, int partition, Schema keySchema, Object key, Schema valueSchema, Object value, long kafkaOffset,
Long timestamp, TimestampType timestampType, Iterable<Header> headers) {
super(topic, partition, keySchema, key, valueSchema, value, timestamp, headers);
this.kafkaOffset = kafkaOffset;
this.timestampType = timestampType;
}
......@@ -53,7 +59,13 @@ public class SinkRecord extends ConnectRecord<SinkRecord> {
@Override
public SinkRecord newRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp) {
return new SinkRecord(topic, kafkaPartition, keySchema, key, valueSchema, value, kafkaOffset(), timestamp, timestampType);
return newRecord(topic, kafkaPartition, keySchema, key, valueSchema, value, timestamp, headers().duplicate());
}
@Override
public SinkRecord newRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value,
Long timestamp, Iterable<Header> headers) {
return new SinkRecord(topic, kafkaPartition, keySchema, key, valueSchema, value, kafkaOffset(), timestamp, timestampType, headers);
}
@Override
......
......@@ -18,6 +18,7 @@ package org.apache.kafka.connect.source;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.header.Header;
import java.util.Map;
......@@ -69,7 +70,15 @@ public class SourceRecord extends ConnectRecord<SourceRecord> {
Schema keySchema, Object key,
Schema valueSchema, Object value,
Long timestamp) {
super(topic, partition, keySchema, key, valueSchema, value, timestamp);
this(sourcePartition, sourceOffset, topic, partition, keySchema, key, valueSchema, value, timestamp, null);
}
public SourceRecord(Map<String, ?> sourcePartition, Map<String, ?> sourceOffset,
String topic, Integer partition,
Schema keySchema, Object key,
Schema valueSchema, Object value,
Long timestamp, Iterable<Header> headers) {
super(topic, partition, keySchema, key, valueSchema, value, timestamp, headers);
this.sourcePartition = sourcePartition;
this.sourceOffset = sourceOffset;
}
......@@ -84,7 +93,13 @@ public class SourceRecord extends ConnectRecord<SourceRecord> {
@Override
public SourceRecord newRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value, Long timestamp) {
return new SourceRecord(sourcePartition, sourceOffset, topic, kafkaPartition, keySchema, key, valueSchema, value, timestamp);
return newRecord(topic, kafkaPartition, keySchema, key, valueSchema, value, timestamp, headers().duplicate());
}
@Override
public SourceRecord newRecord(String topic, Integer kafkaPartition, Schema keySchema, Object key, Schema valueSchema, Object value,
Long timestamp, Iterable<Header> headers) {
return new SourceRecord(sourcePartition, sourceOffset, topic, kafkaPartition, keySchema, key, valueSchema, value, timestamp, headers);
}
@Override
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.storage;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import java.util.Map;
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
/**
* Abstract class that defines the configuration options for {@link Converter} and {@link HeaderConverter} instances.
*/
public abstract class ConverterConfig extends AbstractConfig {
public static final String TYPE_CONFIG = "converter.type";
private static final String TYPE_DOC = "How this converter will be used.";
/**
* Create a new {@link ConfigDef} instance containing the configurations defined by ConverterConfig. This can be called by subclasses.
*
* @return the ConfigDef; never null
*/
public static ConfigDef newConfigDef() {
return new ConfigDef().define(TYPE_CONFIG, Type.STRING, ConfigDef.NO_DEFAULT_VALUE,
in(ConverterType.KEY.getName(), ConverterType.VALUE.getName(), ConverterType.HEADER.getName()),
Importance.LOW, TYPE_DOC);
}
protected ConverterConfig(ConfigDef configDef, Map<String, ?> props) {
super(configDef, props, true);
}
/**
* Get the type of converter as defined by the {@link #TYPE_CONFIG} configuration.
* @return the converter type; never null
*/
public ConverterType type() {
return ConverterType.withName(getString(TYPE_CONFIG));
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.storage;
import java.util.Collections;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
/**
* The type of {@link Converter} and {@link HeaderConverter}.
*/
public enum ConverterType {
KEY,
VALUE,
HEADER;
private static final Map<String, ConverterType> NAME_TO_TYPE;
static {
ConverterType[] types = ConverterType.values();
Map<String, ConverterType> nameToType = new HashMap<>(types.length);
for (ConverterType type : types) {
nameToType.put(type.name, type);
}
NAME_TO_TYPE = Collections.unmodifiableMap(nameToType);
}
/**
* Find the ConverterType with the given name, using a case-insensitive match.
* @param name the name of the converter type; may be null
* @return the matching converter type, or null if the supplied name is null or does not match the name of the known types
*/
public static ConverterType withName(String name) {
if (name == null) {
return null;
}
return NAME_TO_TYPE.get(name.toLowerCase(Locale.getDefault()));
}
private String name;
ConverterType() {
this.name = this.name().toLowerCase(Locale.ROOT);
}
public String getName() {
return name;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.storage;
import org.apache.kafka.common.Configurable;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.header.Header;
import java.io.Closeable;
public interface HeaderConverter extends Configurable, Closeable {
/**
* Convert the header name and byte array value into a {@link Header} object.
* @param topic the name of the topic for the record containing the header
* @param headerKey the header's key; may not be null
* @param value the header's raw value; may be null
* @return the {@link SchemaAndValue}; may not be null
*/
SchemaAndValue toConnectHeader(String topic, String headerKey, byte[] value);
/**
* Convert the {@link Header}'s {@link Header#valueAsBytes() value} into its byte array representation.
* @param topic the name of the topic for the record containing the header
* @param headerKey the header's key; may not be null
* @param schema the schema for the header's value; may be null
* @param value the header's value to convert; may be null
* @return the byte array form of the Header's value; may be null if the value is null
*/
byte[] fromConnectHeader(String topic, String headerKey, Schema schema, Object value);
/**
* Configuration specification for this set of header converters.
* @return the configuration specification; may not be null
*/
ConfigDef config();
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.storage;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.Values;
import org.apache.kafka.connect.errors.DataException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.Map;
import java.util.NoSuchElementException;
/**
* A {@link HeaderConverter} that serializes header values as strings and that deserializes header values to the most appropriate
* numeric, boolean, array, or map representation. Schemas are not serialized, but are inferred upon deserialization when possible.
*/
public class SimpleHeaderConverter implements HeaderConverter {
private static final Logger LOG = LoggerFactory.getLogger(SimpleHeaderConverter.class);
private static final ConfigDef CONFIG_DEF = new ConfigDef();
private static final SchemaAndValue NULL_SCHEMA_AND_VALUE = new SchemaAndValue(null, null);
private static final Charset UTF_8 = StandardCharsets.UTF_8;
@Override
public ConfigDef config() {
return CONFIG_DEF;
}
@Override
public void configure(Map<String, ?> configs) {
// do nothing
}
@Override
public SchemaAndValue toConnectHeader(String topic, String headerKey, byte[] value) {
if (value == null) {
return NULL_SCHEMA_AND_VALUE;
}
try {
String str = new String(value, UTF_8);
if (str.isEmpty()) {
return new SchemaAndValue(Schema.STRING_SCHEMA, str);
}
return Values.parseString(str);
} catch (NoSuchElementException e) {
throw new DataException("Failed to deserialize value for header '" + headerKey + "' on topic '" + topic + "'", e);
} catch (Throwable t) {
LOG.warn("Failed to deserialize value for header '{}' on topic '{}', so using byte array", headerKey, topic, t);
return new SchemaAndValue(Schema.BYTES_SCHEMA, value);
}
}
@Override
public byte[] fromConnectHeader(String topic, String headerKey, Schema schema, Object value) {
if (value == null) {
return null;
}
return Values.convertToString(schema, value).getBytes(UTF_8);
}
@Override
public void close() throws IOException {
// do nothing
}
}
......@@ -16,6 +16,7 @@
*/
package org.apache.kafka.connect.storage;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.serialization.StringSerializer;
......@@ -27,16 +28,19 @@ import java.util.HashMap;
import java.util.Map;
/**
* {@link Converter} implementation that only supports serializing to strings. When converting Kafka Connect data to bytes,
* the schema will be ignored and {@link Object#toString()} will always be invoked to convert the data to a String.
* {@link Converter} and {@link HeaderConverter} implementation that only supports serializing to strings. When converting Kafka Connect
* data to bytes, the schema will be ignored and {@link Object#toString()} will always be invoked to convert the data to a String.
* When converting from bytes to Kafka Connect format, the converter will only ever return an optional string schema and
* a string or null.
*
* Encoding configuration is identical to {@link StringSerializer} and {@link StringDeserializer}, but for convenience
* this class can also be configured to use the same encoding for both encoding and decoding with the converter.encoding
* setting.
* this class can also be configured to use the same encoding for both encoding and decoding with the
* {@link StringConverterConfig#ENCODING_CONFIG converter.encoding} setting.
*
* This implementation currently does nothing with the topic names or header names.
*/
public class StringConverter implements Converter {
public class StringConverter implements Converter, HeaderConverter {
private final StringSerializer serializer = new StringSerializer();
private final StringDeserializer deserializer = new StringDeserializer();
......@@ -44,22 +48,32 @@ public class StringConverter implements Converter {
}
@Override
public void configure(Map<String, ?> configs, boolean isKey) {
Map<String, Object> serializerConfigs = new HashMap<>();
serializerConfigs.putAll(configs);
Map<String, Object> deserializerConfigs = new HashMap<>();
deserializerConfigs.putAll(configs);
public ConfigDef config() {
return StringConverterConfig.configDef();
}
Object encodingValue = configs.get("converter.encoding");
if (encodingValue != null) {
serializerConfigs.put("serializer.encoding", encodingValue);
deserializerConfigs.put("deserializer.encoding", encodingValue);
}
@Override
public void configure(Map<String, ?> configs) {
StringConverterConfig conf = new StringConverterConfig(configs);
String encoding = conf.encoding();
Map<String, Object> serializerConfigs = new HashMap<>(configs);
Map<String, Object> deserializerConfigs = new HashMap<>(configs);
serializerConfigs.put("serializer.encoding", encoding);
deserializerConfigs.put("deserializer.encoding", encoding);
boolean isKey = conf.type() == ConverterType.KEY;
serializer.configure(serializerConfigs, isKey);
deserializer.configure(deserializerConfigs, isKey);
}
@Override
public void configure(Map<String, ?> configs, boolean isKey) {
Map<String, Object> conf = new HashMap<>(configs);
conf.put(StringConverterConfig.TYPE_CONFIG, isKey ? ConverterType.KEY.getName() : ConverterType.VALUE.getName());
configure(conf);
}
@Override
public byte[] fromConnectData(String topic, Schema schema, Object value) {
try {
......@@ -77,4 +91,19 @@ public class StringConverter implements Converter {
throw new DataException("Failed to deserialize string: ", e);
}
}
@Override
public byte[] fromConnectHeader(String topic, String headerKey, Schema schema, Object value) {
return fromConnectData(topic, schema, value);
}
@Override
public SchemaAndValue toConnectHeader(String topic, String headerKey, byte[] value) {
return toConnectData(topic, value);
}
@Override
public void close() {
// do nothing
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.storage;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigDef.Width;
import java.util.Map;
/**
* Configuration options for {@link StringConverter} instances.
*/
public class StringConverterConfig extends ConverterConfig {
public static final String ENCODING_CONFIG = "converter.encoding";
public static final String ENCODING_DEFAULT = "UTF8";
private static final String ENCODING_DOC = "The name of the Java character set to use for encoding strings as byte arrays.";
private static final String ENCODING_DISPLAY = "Encoding";
private final static ConfigDef CONFIG;
static {
CONFIG = ConverterConfig.newConfigDef();
CONFIG.define(ENCODING_CONFIG, Type.STRING, ENCODING_DEFAULT, Importance.HIGH, ENCODING_DOC, null, -1, Width.MEDIUM,
ENCODING_DISPLAY);
}
public static ConfigDef configDef() {
return CONFIG;
}
public StringConverterConfig(Map<String, ?> props) {
super(CONFIG, props);
}
/**
* Get the string encoding.
*
* @return the encoding; never null
*/
public String encoding() {
return getString(ENCODING_CONFIG);
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.data;
import org.apache.kafka.connect.data.Values.Parser;
import org.junit.Test;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class ValuesTest {
private static final Map<String, String> STRING_MAP = new LinkedHashMap<>();
private static final Schema STRING_MAP_SCHEMA = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA).schema();
private static final Map<String, Short> STRING_SHORT_MAP = new LinkedHashMap<>();
private static final Schema STRING_SHORT_MAP_SCHEMA = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT16_SCHEMA).schema();
private static final Map<String, Integer> STRING_INT_MAP = new LinkedHashMap<>();
private static final Schema STRING_INT_MAP_SCHEMA = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT32_SCHEMA).schema();
private static final List<Integer> INT_LIST = new ArrayList<>();
private static final Schema INT_LIST_SCHEMA = SchemaBuilder.array(Schema.INT32_SCHEMA).schema();
private static final List<String> STRING_LIST = new ArrayList<>();
private static final Schema STRING_LIST_SCHEMA = SchemaBuilder.array(Schema.STRING_SCHEMA).schema();
static {
STRING_MAP.put("foo", "123");
STRING_MAP.put("bar", "baz");
STRING_SHORT_MAP.put("foo", (short) 12345);
STRING_SHORT_MAP.put("bar", (short) 0);
STRING_SHORT_MAP.put("baz", (short) -4321);
STRING_INT_MAP.put("foo", 1234567890);
STRING_INT_MAP.put("bar", 0);
STRING_INT_MAP.put("baz", -987654321);
STRING_LIST.add("foo");
STRING_LIST.add("bar");
INT_LIST.add(1234567890);
INT_LIST.add(-987654321);
}
@Test
public void shouldEscapeStringsWithEmbeddedQuotesAndBackslashes() {
String original = "three\"blind\\\"mice";
String expected = "three\\\"blind\\\\\\\"mice";
assertEquals(expected, Values.escape(original));
}
@Test
public void shouldConvertNullValue() {
assertRoundTrip(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA, null);
assertRoundTrip(Schema.OPTIONAL_STRING_SCHEMA, Schema.STRING_SCHEMA, null);
}
@Test
public void shouldConvertSimpleString() {
assertRoundTrip(Schema.STRING_SCHEMA, "simple");
}
@Test
public void shouldConvertEmptyString() {
assertRoundTrip(Schema.STRING_SCHEMA, "");
}
@Test
public void shouldConvertStringWithQuotesAndOtherDelimiterCharacters() {
assertRoundTrip(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA, "three\"blind\\\"mice");
assertRoundTrip(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA, "string with delimiters: <>?,./\\=+-!@#$%^&*(){}[]|;':");
}
@Test
public void shouldConvertMapWithStringKeys() {
assertRoundTrip(STRING_MAP_SCHEMA, STRING_MAP_SCHEMA, STRING_MAP);
}
@Test
public void shouldParseStringOfMapWithStringValuesWithoutWhitespaceAsMap() {
SchemaAndValue result = roundTrip(STRING_MAP_SCHEMA, "{\"foo\":\"123\",\"bar\":\"baz\"}");
assertEquals(STRING_MAP_SCHEMA, result.schema());
assertEquals(STRING_MAP, result.value());
}
@Test
public void shouldParseStringOfMapWithStringValuesWithWhitespaceAsMap() {
SchemaAndValue result = roundTrip(STRING_MAP_SCHEMA, "{ \"foo\" : \"123\", \n\"bar\" : \"baz\" } ");
assertEquals(STRING_MAP_SCHEMA, result.schema());
assertEquals(STRING_MAP, result.value());
}
@Test
public void shouldConvertMapWithStringKeysAndShortValues() {
assertRoundTrip(STRING_SHORT_MAP_SCHEMA, STRING_SHORT_MAP_SCHEMA, STRING_SHORT_MAP);
}
@Test
public void shouldParseStringOfMapWithShortValuesWithoutWhitespaceAsMap() {
SchemaAndValue result = roundTrip(STRING_SHORT_MAP_SCHEMA, "{\"foo\":12345,\"bar\":0,\"baz\":-4321}");
assertEquals(STRING_SHORT_MAP_SCHEMA, result.schema());
assertEquals(STRING_SHORT_MAP, result.value());
}
@Test
public void shouldParseStringOfMapWithShortValuesWithWhitespaceAsMap() {
SchemaAndValue result = roundTrip(STRING_SHORT_MAP_SCHEMA, " { \"foo\" : 12345 , \"bar\" : 0, \"baz\" : -4321 } ");
assertEquals(STRING_SHORT_MAP_SCHEMA, result.schema());
assertEquals(STRING_SHORT_MAP, result.value());
}
@Test
public void shouldConvertMapWithStringKeysAndIntegerValues() {
assertRoundTrip(STRING_INT_MAP_SCHEMA, STRING_INT_MAP_SCHEMA, STRING_INT_MAP);
}
@Test
public void shouldParseStringOfMapWithIntValuesWithoutWhitespaceAsMap() {
SchemaAndValue result = roundTrip(STRING_INT_MAP_SCHEMA, "{\"foo\":1234567890,\"bar\":0,\"baz\":-987654321}");
assertEquals(STRING_INT_MAP_SCHEMA, result.schema());
assertEquals(STRING_INT_MAP, result.value());
}
@Test
public void shouldParseStringOfMapWithIntValuesWithWhitespaceAsMap() {
SchemaAndValue result = roundTrip(STRING_INT_MAP_SCHEMA, " { \"foo\" : 1234567890 , \"bar\" : 0, \"baz\" : -987654321 } ");
assertEquals(STRING_INT_MAP_SCHEMA, result.schema());
assertEquals(STRING_INT_MAP, result.value());
}
@Test
public void shouldConvertListWithStringValues() {
assertRoundTrip(STRING_LIST_SCHEMA, STRING_LIST_SCHEMA, STRING_LIST);
}
@Test
public void shouldConvertListWithIntegerValues() {
assertRoundTrip(INT_LIST_SCHEMA, INT_LIST_SCHEMA, INT_LIST);
}
@Test
public void shouldParseStringsWithoutDelimiters() {
//assertParsed("");
assertParsed(" ");
assertParsed("simple");
assertParsed("simple string");
assertParsed("simple \n\t\bstring");
assertParsed("'simple' string");
assertParsed("si\\mple");
assertParsed("si\\\\mple");
}
@Test
public void shouldParseStringsWithEscapedDelimiters() {
assertParsed("si\\\"mple");
assertParsed("si\\{mple");
assertParsed("si\\}mple");
assertParsed("si\\]mple");
assertParsed("si\\[mple");
assertParsed("si\\:mple");
assertParsed("si\\,mple");
}
@Test
public void shouldParseStringsWithSingleDelimiter() {
assertParsed("a{b", "a", "{", "b");
assertParsed("a}b", "a", "}", "b");
assertParsed("a[b", "a", "[", "b");
assertParsed("a]b", "a", "]", "b");
assertParsed("a:b", "a", ":", "b");
assertParsed("a,b", "a", ",", "b");
assertParsed("a\"b", "a", "\"", "b");
assertParsed("{b", "{", "b");
assertParsed("}b", "}", "b");
assertParsed("[b", "[", "b");
assertParsed("]b", "]", "b");
assertParsed(":b", ":", "b");
assertParsed(",b", ",", "b");
assertParsed("\"b", "\"", "b");
assertParsed("{", "{");
assertParsed("}", "}");
assertParsed("[", "[");
assertParsed("]", "]");
assertParsed(":", ":");
assertParsed(",", ",");
assertParsed("\"", "\"");
}
@Test
public void shouldParseStringsWithMultipleDelimiters() {
assertParsed("\"simple\" string", "\"", "simple", "\"", " string");
assertParsed("a{bc}d", "a", "{", "bc", "}", "d");
assertParsed("a { b c } d", "a ", "{", " b c ", "}", " d");
assertParsed("a { b c } d", "a ", "{", " b c ", "}", " d");
}
@Test
public void canConsume() {
}
protected void assertParsed(String input) {
assertParsed(input, input);
}
protected void assertParsed(String input, String... expectedTokens) {
Parser parser = new Parser(input);
if (!parser.hasNext()) {
assertEquals(1, expectedTokens.length);
assertTrue(expectedTokens[0].isEmpty());
return;
}
for (String expectedToken : expectedTokens) {
assertTrue(parser.hasNext());
int position = parser.mark();
assertEquals(expectedToken, parser.next());
assertEquals(position + expectedToken.length(), parser.position());
assertEquals(expectedToken, parser.previous());
parser.rewindTo(position);
assertEquals(position, parser.position());
assertEquals(expectedToken, parser.next());
int newPosition = parser.mark();
assertEquals(position + expectedToken.length(), newPosition);
assertEquals(expectedToken, parser.previous());
}
assertFalse(parser.hasNext());
// Rewind and try consuming expected tokens ...
parser.rewindTo(0);
assertConsumable(parser, expectedTokens);
// Parse again and try consuming expected tokens ...
parser = new Parser(input);
assertConsumable(parser, expectedTokens);
}
protected void assertConsumable(Parser parser, String ... expectedTokens) {
for (String expectedToken : expectedTokens) {
if (!expectedToken.trim().isEmpty()) {
int position = parser.mark();
assertTrue(parser.canConsume(expectedToken.trim()));
parser.rewindTo(position);
assertTrue(parser.canConsume(expectedToken.trim(), true));
parser.rewindTo(position);
assertTrue(parser.canConsume(expectedToken, false));
}
}
}
protected SchemaAndValue roundTrip(Schema desiredSchema, String currentValue) {
return roundTrip(desiredSchema, new SchemaAndValue(Schema.STRING_SCHEMA, currentValue));
}
protected SchemaAndValue roundTrip(Schema desiredSchema, SchemaAndValue input) {
String serialized = Values.convertToString(input.schema(), input.value());
if (input != null && input.value() != null) {
assertNotNull(serialized);
}
if (desiredSchema == null) {
desiredSchema = Values.inferSchema(input);
assertNotNull(desiredSchema);
}
Object newValue = null;
Schema newSchema = null;
switch (desiredSchema.type()) {
case STRING:
newValue = Values.convertToString(Schema.STRING_SCHEMA, serialized);
break;
case INT8:
newValue = Values.convertToByte(Schema.STRING_SCHEMA, serialized);
break;
case INT16:
newValue = Values.convertToShort(Schema.STRING_SCHEMA, serialized);
break;
case INT32:
newValue = Values.convertToInteger(Schema.STRING_SCHEMA, serialized);
break;
case INT64:
newValue = Values.convertToLong(Schema.STRING_SCHEMA, serialized);
break;
case FLOAT32:
newValue = Values.convertToFloat(Schema.STRING_SCHEMA, serialized);
break;
case FLOAT64:
newValue = Values.convertToDouble(Schema.STRING_SCHEMA, serialized);
break;
case BOOLEAN:
newValue = Values.convertToBoolean(Schema.STRING_SCHEMA, serialized);
break;
case ARRAY:
newValue = Values.convertToList(Schema.STRING_SCHEMA, serialized);
break;
case MAP:
newValue = Values.convertToMap(Schema.STRING_SCHEMA, serialized);
break;
case STRUCT:
newValue = Values.convertToStruct(Schema.STRING_SCHEMA, serialized);
break;
case BYTES:
fail("unexpected schema type");
break;
}
newSchema = Values.inferSchema(newValue);
return new SchemaAndValue(newSchema, newValue);
}
protected void assertRoundTrip(Schema schema, String value) {
assertRoundTrip(schema, Schema.STRING_SCHEMA, value);
}
protected void assertRoundTrip(Schema schema, Schema currentSchema, Object value) {
SchemaAndValue result = roundTrip(schema, new SchemaAndValue(currentSchema, value));
if (value == null) {
assertNull(result.schema());
assertNull(result.value());
} else {
assertEquals(value, result.value());
assertEquals(schema, result.schema());
SchemaAndValue result2 = roundTrip(result.schema(), result);
assertEquals(schema, result2.schema());
assertEquals(value, result2.value());
assertEquals(result, result2);
}
}
}
\ No newline at end of file
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.header;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
public class ConnectHeaderTest {
private String key;
private ConnectHeader header;
@Before
public void beforeEach() {
key = "key";
withString("value");
}
protected Header withValue(Schema schema, Object value) {
header = new ConnectHeader(key, new SchemaAndValue(schema, value));
return header;
}
protected Header withString(String value) {
return withValue(Schema.STRING_SCHEMA, value);
}
@Test
public void shouldAllowNullValues() {
withValue(Schema.OPTIONAL_STRING_SCHEMA, null);
}
@Test
public void shouldAllowNullSchema() {
withValue(null, null);
assertNull(header.schema());
assertNull(header.value());
String value = "non-null value";
withValue(null, value);
assertNull(header.schema());
assertSame(value, header.value());
}
@Test
public void shouldAllowNonNullValue() {
String value = "non-null value";
withValue(Schema.STRING_SCHEMA, value);
assertSame(Schema.STRING_SCHEMA, header.schema());
assertEquals(value, header.value());
withValue(Schema.BOOLEAN_SCHEMA, true);
assertSame(Schema.BOOLEAN_SCHEMA, header.schema());
assertEquals(true, header.value());
}
@Test
public void shouldGetSchemaFromStruct() {
Schema schema = SchemaBuilder.struct()
.field("foo", Schema.STRING_SCHEMA)
.field("bar", Schema.INT32_SCHEMA)
.build();
Struct value = new Struct(schema);
value.put("foo", "value");
value.put("bar", 100);
withValue(null, value);
assertSame(schema, header.schema());
assertSame(value, header.value());
}
@Test
public void shouldSatisfyEquals() {
String value = "non-null value";
Header h1 = withValue(Schema.STRING_SCHEMA, value);
assertSame(Schema.STRING_SCHEMA, header.schema());
assertEquals(value, header.value());
Header h2 = withValue(Schema.STRING_SCHEMA, value);
assertEquals(h1, h2);
assertEquals(h1.hashCode(), h2.hashCode());
Header h3 = withValue(Schema.INT8_SCHEMA, 100);
assertNotEquals(h3, h2);
}
}
\ No newline at end of file
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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 org.apache.kafka.connect.sink;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Values;
import org.apache.kafka.connect.header.ConnectHeaders;
import org.apache.kafka.connect.header.Header;
import org.apache.kafka.connect.header.Headers;
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
public class SinkRecordTest {
private static final String TOPIC_NAME = "myTopic";
private static final Integer PARTITION_NUMBER = 0;
private static final long KAFKA_OFFSET = 0L;
private static final Long KAFKA_TIMESTAMP = 0L;
private static final TimestampType TS_TYPE = TimestampType.CREATE_TIME;
private SinkRecord record;
@Before
public void beforeEach() {
record = new SinkRecord(TOPIC_NAME, PARTITION_NUMBER, Schema.STRING_SCHEMA, "key", Schema.BOOLEAN_SCHEMA, false, KAFKA_OFFSET,
KAFKA_TIMESTAMP, TS_TYPE, null);
}
@Test
public void shouldCreateSinkRecordWithHeaders() {
Headers headers = new ConnectHeaders().addString("h1", "hv1").addBoolean("h2", true);
record = new SinkRecord(TOPIC_NAME, PARTITION_NUMBER, Schema.STRING_SCHEMA, "key", Schema.BOOLEAN_SCHEMA, false, KAFKA_OFFSET,
KAFKA_TIMESTAMP, TS_TYPE, headers);
assertNotNull(record.headers());
assertSame(headers, record.headers());
assertFalse(record.headers().isEmpty());
}
@Test
public void shouldCreateSinkRecordWithEmptyHeaders() {
assertEquals(TOPIC_NAME, record.topic());
assertEquals(PARTITION_NUMBER, record.kafkaPartition());
assertEquals(Schema.STRING_SCHEMA, record.keySchema());
assertEquals("key", record.key());
assertEquals(Schema.BOOLEAN_SCHEMA, record.valueSchema());
assertEquals(false, record.value());
assertEquals(KAFKA_OFFSET, record.kafkaOffset());
assertEquals(KAFKA_TIMESTAMP, record.timestamp());
assertEquals(TS_TYPE, record.timestampType());
assertNotNull(record.headers());
assertTrue(record.headers().isEmpty());
}
@Test
public void shouldDuplicateRecordAndCloneHeaders() {
SinkRecord duplicate = record.newRecord(TOPIC_NAME, PARTITION_NUMBER, Schema.STRING_SCHEMA, "key", Schema.BOOLEAN_SCHEMA, false,
KAFKA_TIMESTAMP);
assertEquals(TOPIC_NAME, duplicate.topic());
assertEquals(PARTITION_NUMBER, duplicate.kafkaPartition());
assertEquals(Schema.STRING_SCHEMA, duplicate.keySchema());
assertEquals("key", duplicate.key());
assertEquals(Schema.BOOLEAN_SCHEMA, duplicate.valueSchema());
assertEquals(false, duplicate.value());
assertEquals(KAFKA_OFFSET, duplicate.kafkaOffset());
assertEquals(KAFKA_TIMESTAMP, duplicate.timestamp());
assertEquals(TS_TYPE, duplicate.timestampType());
assertNotNull(duplicate.headers());
assertTrue(duplicate.headers().isEmpty());
assertNotSame(record.headers(), duplicate.headers());
assertEquals(record.headers(), duplicate.headers());
}
@Test
public void shouldDuplicateRecordUsingNewHeaders() {
Headers newHeaders = new ConnectHeaders().addString("h3", "hv3");
SinkRecord duplicate = record.newRecord(TOPIC_NAME, PARTITION_NUMBER, Schema.STRING_SCHEMA, "key", Schema.BOOLEAN_SCHEMA, false,
KAFKA_TIMESTAMP, newHeaders);
assertEquals(TOPIC_NAME, duplicate.topic());
assertEquals(PARTITION_NUMBER, duplicate.kafkaPartition());
assertEquals(Schema.STRING_SCHEMA, duplicate.keySchema());
assertEquals("key", duplicate.key());
assertEquals(Schema.BOOLEAN_SCHEMA, duplicate.valueSchema());
assertEquals(false, duplicate.value());
assertEquals(KAFKA_OFFSET, duplicate.kafkaOffset());
assertEquals(KAFKA_TIMESTAMP, duplicate.timestamp());
assertEquals(TS_TYPE, duplicate.timestampType());
assertNotNull(duplicate.headers());
assertEquals(newHeaders, duplicate.headers());
assertSame(newHeaders, duplicate.headers());
assertNotSame(record.headers(), duplicate.headers());
assertNotEquals(record.headers(), duplicate.headers());
}
@Test
public void shouldModifyRecordHeader() {
assertTrue(record.headers().isEmpty());
record.headers().addInt("intHeader", 100);
assertEquals(1, record.headers().size());
Header header = record.headers().lastWithName("intHeader");
assertEquals(100, (int) Values.convertToInteger(header.schema(), header.value()));
}
}
\ No newline at end of file
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment