mirror of https://github.com/apache/kafka.git
KAFKA-4783: Add ByteArrayConverter (KIP-128)
Author: Ewen Cheslack-Postava <me@ewencp.org> Reviewers: Guozhang Wang <wangguoz@gmail.com> Closes #2599 from ewencp/kafka-4783-byte-array-converter
This commit is contained in:
parent
9e4548df30
commit
52a15d7c0b
|
@ -221,6 +221,10 @@
|
|||
<allow pkg="org.apache.kafka.connect.storage" />
|
||||
</subpackage>
|
||||
|
||||
<subpackage name="converters">
|
||||
<allow pkg="org.apache.kafka.connect.storage" />
|
||||
</subpackage>
|
||||
|
||||
<subpackage name="runtime">
|
||||
<allow pkg="org.apache.kafka.connect" />
|
||||
<allow pkg="org.reflections"/>
|
||||
|
|
|
@ -0,0 +1,52 @@
|
|||
/**
|
||||
* 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.converters;
|
||||
|
||||
import org.apache.kafka.connect.data.Schema;
|
||||
import org.apache.kafka.connect.data.SchemaAndValue;
|
||||
import org.apache.kafka.connect.errors.DataException;
|
||||
import org.apache.kafka.connect.storage.Converter;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Pass-through converter for raw byte data.
|
||||
*/
|
||||
public class ByteArrayConverter implements Converter {
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> configs, boolean isKey) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] fromConnectData(String topic, Schema schema, Object value) {
|
||||
if (schema != null && schema.type() != Schema.Type.BYTES)
|
||||
throw new DataException("Invalid schema type for ByteArrayConverter: " + schema.type().toString());
|
||||
|
||||
if (value != null && !(value instanceof byte[]))
|
||||
throw new DataException("ByteArrayConverter is not compatible with objects of type " + value.getClass());
|
||||
|
||||
return (byte[]) value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SchemaAndValue toConnectData(String topic, byte[] value) {
|
||||
return new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/**
|
||||
* 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.converters;
|
||||
|
||||
import org.apache.kafka.connect.data.Schema;
|
||||
import org.apache.kafka.connect.data.SchemaAndValue;
|
||||
import org.apache.kafka.connect.errors.DataException;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class ByteArrayConverterTest {
|
||||
private static final String TOPIC = "topic";
|
||||
private static final byte[] SAMPLE_BYTES = "sample string".getBytes(StandardCharsets.UTF_8);
|
||||
|
||||
private ByteArrayConverter converter = new ByteArrayConverter();
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
converter.configure(Collections.<String, String>emptyMap(), false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFromConnect() {
|
||||
assertArrayEquals(
|
||||
SAMPLE_BYTES,
|
||||
converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, SAMPLE_BYTES)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFromConnectSchemaless() {
|
||||
assertArrayEquals(
|
||||
SAMPLE_BYTES,
|
||||
converter.fromConnectData(TOPIC, null, SAMPLE_BYTES)
|
||||
);
|
||||
}
|
||||
|
||||
@Test(expected = DataException.class)
|
||||
public void testFromConnectBadSchema() {
|
||||
converter.fromConnectData(TOPIC, Schema.INT32_SCHEMA, SAMPLE_BYTES);
|
||||
}
|
||||
|
||||
@Test(expected = DataException.class)
|
||||
public void testFromConnectInvalidValue() {
|
||||
converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, 12);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFromConnectNull() {
|
||||
assertNull(converter.fromConnectData(TOPIC, Schema.BYTES_SCHEMA, null));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToConnect() {
|
||||
SchemaAndValue data = converter.toConnectData(TOPIC, SAMPLE_BYTES);
|
||||
assertEquals(Schema.OPTIONAL_BYTES_SCHEMA, data.schema());
|
||||
assertTrue(Arrays.equals(SAMPLE_BYTES, (byte[]) data.value()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToConnectNull() {
|
||||
SchemaAndValue data = converter.toConnectData(TOPIC, null);
|
||||
assertEquals(Schema.OPTIONAL_BYTES_SCHEMA, data.schema());
|
||||
assertNull(data.value());
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue