|
| 1 | +/* |
| 2 | + * Copyright 2020 Confluent Inc. |
| 3 | + * |
| 4 | + * Licensed under the Confluent Community License (the "License"); you may not use |
| 5 | + * this file except in compliance with the License. You may obtain a copy of the |
| 6 | + * License at |
| 7 | + * |
| 8 | + * http://www.confluent.io/confluent-community-license |
| 9 | + * |
| 10 | + * Unless required by applicable law or agreed to in writing, software |
| 11 | + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT |
| 12 | + * WARRANTIES OF ANY KIND, either express or implied. See the License for the |
| 13 | + * specific language governing permissions and limitations under the License. |
| 14 | + */ |
| 15 | + |
| 16 | +package io.confluent.connect.elasticsearch.integration; |
| 17 | + |
| 18 | +import io.confluent.connect.avro.AvroConverter; |
| 19 | +import io.confluent.connect.json.JsonSchemaConverter; |
| 20 | +import io.confluent.connect.protobuf.ProtobufConverter; |
| 21 | +import io.confluent.kafka.schemaregistry.CompatibilityLevel; |
| 22 | +import io.confluent.kafka.schemaregistry.RestApp; |
| 23 | +import org.apache.kafka.clients.producer.KafkaProducer; |
| 24 | +import org.apache.kafka.clients.producer.ProducerConfig; |
| 25 | +import org.apache.kafka.clients.producer.ProducerRecord; |
| 26 | +import org.apache.kafka.common.KafkaException; |
| 27 | +import org.apache.kafka.common.serialization.ByteArraySerializer; |
| 28 | +import org.apache.kafka.connect.data.Schema; |
| 29 | +import org.apache.kafka.connect.data.SchemaAndValue; |
| 30 | +import org.apache.kafka.connect.data.SchemaBuilder; |
| 31 | +import org.apache.kafka.connect.data.Struct; |
| 32 | +import org.apache.kafka.connect.storage.Converter; |
| 33 | +import org.apache.kafka.test.TestUtils; |
| 34 | +import org.junit.Test; |
| 35 | +import org.junit.runner.RunWith; |
| 36 | +import org.junit.runners.Parameterized; |
| 37 | +import org.junit.runners.Parameterized.Parameters; |
| 38 | + |
| 39 | +import java.io.IOException; |
| 40 | +import java.net.ServerSocket; |
| 41 | +import java.util.ArrayList; |
| 42 | +import java.util.Arrays; |
| 43 | +import java.util.Collections; |
| 44 | +import java.util.HashMap; |
| 45 | +import java.util.List; |
| 46 | +import java.util.Map; |
| 47 | +import java.util.Properties; |
| 48 | +import java.util.concurrent.TimeUnit; |
| 49 | + |
| 50 | +import static io.confluent.kafka.schemaregistry.ClusterTestHarness.KAFKASTORE_TOPIC; |
| 51 | + |
| 52 | +@RunWith(Parameterized.class) |
| 53 | +public class ElasticsearchConnectorDataFormatIT extends ElasticsearchConnectorBaseIT { |
| 54 | + |
| 55 | + protected void startSchemaRegistry() throws Exception { |
| 56 | + int port = findAvailableOpenPort(); |
| 57 | + restApp = new RestApp(port, null, connect.kafka().bootstrapServers(), |
| 58 | + KAFKASTORE_TOPIC, CompatibilityLevel.NONE.name, true, new Properties()); |
| 59 | + restApp.start(); |
| 60 | + } |
| 61 | + |
| 62 | + protected void stopSchemaRegistry() throws Exception { |
| 63 | + restApp.stop(); |
| 64 | + } |
| 65 | + |
| 66 | + protected void waitForSchemaRegistryToStart() throws InterruptedException { |
| 67 | + TestUtils.waitForCondition( |
| 68 | + () -> restApp.restServer.isRunning(), |
| 69 | + CONNECTOR_STARTUP_DURATION_MS, |
| 70 | + "Schema-registry server did not start in time." |
| 71 | + ); |
| 72 | + } |
| 73 | + |
| 74 | + private Converter converter; |
| 75 | + private Class<? extends Converter> converterClass; |
| 76 | + |
| 77 | + @Override |
| 78 | + public void setup() throws Exception { |
| 79 | + startConnect(); |
| 80 | + startSchemaRegistry(); |
| 81 | + connect.kafka().createTopic(TOPIC); |
| 82 | + |
| 83 | + props = createProps(); |
| 84 | + client = createClient(); |
| 85 | + } |
| 86 | + |
| 87 | + @Override |
| 88 | + public void cleanup() throws Exception { |
| 89 | + stopConnect(); |
| 90 | + stopSchemaRegistry(); |
| 91 | + client.deleteAll(); |
| 92 | + client.close(); |
| 93 | + } |
| 94 | + |
| 95 | + @Parameters |
| 96 | + public static List<Class<? extends Converter>> data() { |
| 97 | + return Arrays.asList(JsonSchemaConverter.class, ProtobufConverter.class, AvroConverter.class); |
| 98 | + } |
| 99 | + |
| 100 | + public ElasticsearchConnectorDataFormatIT(Class<? extends Converter> converter) throws Exception { |
| 101 | + this.converterClass = converter; |
| 102 | + this.converter = converterClass.getConstructor().newInstance(); |
| 103 | + } |
| 104 | + |
| 105 | + @Test |
| 106 | + public void testHappyPathDataFormat() throws Exception { |
| 107 | + // configure configs and converter with schema-registry addr |
| 108 | + props.put("value.converter", converterClass.getSimpleName()); |
| 109 | + props.put("value.converter.schema.registry.url", restApp.restServer.getURI().toString()); |
| 110 | + props.put("value.converter.scrub.invalid.names", "true"); |
| 111 | + converter.configure(Collections.singletonMap( |
| 112 | + "schema.registry.url", restApp.restServer.getURI().toString() |
| 113 | + ), false |
| 114 | + ); |
| 115 | + |
| 116 | + // start the connector |
| 117 | + connect.configureConnector(CONNECTOR_NAME, props); |
| 118 | + |
| 119 | + // wait for schema-registry to spin up |
| 120 | + waitForSchemaRegistryToStart(); |
| 121 | + |
| 122 | + // wait for tasks to spin up |
| 123 | + waitForConnectorToStart(CONNECTOR_NAME, TASKS_MAX); |
| 124 | + |
| 125 | + // run test |
| 126 | + writeRecords(NUM_RECORDS); |
| 127 | + verifySearchResults(NUM_RECORDS); |
| 128 | + } |
| 129 | + |
| 130 | + @Override |
| 131 | + protected void writeRecords(int numRecords) { |
| 132 | + writeRecordsFromIndex(0, numRecords, converter); |
| 133 | + } |
| 134 | + |
| 135 | + protected void writeRecordsFromIndex(int start, int numRecords, Converter converter) { |
| 136 | + // get defined schema for the test |
| 137 | + Schema schema = getRecordSchema(); |
| 138 | + |
| 139 | + // configure producer with default properties |
| 140 | + KafkaProducer<byte[], byte[]> producer = configureProducer(); |
| 141 | + |
| 142 | + List<SchemaAndValue> recordsList = getRecords(schema, start, numRecords); |
| 143 | + |
| 144 | + // produce records into topic |
| 145 | + produceRecords(producer, converter, recordsList, TOPIC); |
| 146 | + } |
| 147 | + |
| 148 | + private Integer findAvailableOpenPort() throws IOException { |
| 149 | + try (ServerSocket socket = new ServerSocket(0)) { |
| 150 | + return socket.getLocalPort(); |
| 151 | + } |
| 152 | + } |
| 153 | + |
| 154 | + protected List<SchemaAndValue> getRecords(Schema schema, int start, int numRecords) { |
| 155 | + List<SchemaAndValue> recordList = new ArrayList<>(); |
| 156 | + for (int i = start; i < start + numRecords; i++) { |
| 157 | + Struct struct = new Struct(schema); |
| 158 | + struct.put("doc_num", i); |
| 159 | + SchemaAndValue schemaAndValue = new SchemaAndValue(schema, struct); |
| 160 | + recordList.add(schemaAndValue); |
| 161 | + } |
| 162 | + return recordList; |
| 163 | + } |
| 164 | + |
| 165 | + protected void produceRecords( |
| 166 | + KafkaProducer<byte[], byte[]> producer, |
| 167 | + Converter converter, |
| 168 | + List<SchemaAndValue> recordsList, |
| 169 | + String topic |
| 170 | + ) { |
| 171 | + for (int i = 0; i < recordsList.size(); i++) { |
| 172 | + SchemaAndValue schemaAndValue = recordsList.get(i); |
| 173 | + byte[] convertedStruct = converter.fromConnectData(topic, schemaAndValue.schema(), schemaAndValue.value()); |
| 174 | + ProducerRecord<byte[], byte[]> msg = new ProducerRecord<>(topic, 0, String.valueOf(i).getBytes(), convertedStruct); |
| 175 | + try { |
| 176 | + producer.send(msg).get(TimeUnit.SECONDS.toMillis(120), TimeUnit.MILLISECONDS); |
| 177 | + } catch (Exception e) { |
| 178 | + throw new KafkaException("Could not produce message: " + msg, e); |
| 179 | + } |
| 180 | + } |
| 181 | + } |
| 182 | + |
| 183 | + protected KafkaProducer<byte[], byte[]> configureProducer() { |
| 184 | + Map<String, Object> producerProps = new HashMap<>(); |
| 185 | + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, connect.kafka().bootstrapServers()); |
| 186 | + return new KafkaProducer<>(producerProps, new ByteArraySerializer(), new ByteArraySerializer()); |
| 187 | + } |
| 188 | + |
| 189 | + protected Schema getRecordSchema() { |
| 190 | + SchemaBuilder schemaBuilder = SchemaBuilder.struct(); |
| 191 | + schemaBuilder.field("doc_num", Schema.INT32_SCHEMA); |
| 192 | + return schemaBuilder.build(); |
| 193 | + } |
| 194 | +} |
0 commit comments